You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by aw...@apache.org on 2015/06/30 20:52:18 UTC

[01/30] hadoop git commit: MAPREDUCE-6410. Fixed MapReduce JobHistory server to use the right (login) UGI to refresh log and cleaner settings. Contributed by Varun Saxena.

Repository: hadoop
Updated Branches:
  refs/heads/HADOOP-12111 1b8040628 -> 084becdd7


MAPREDUCE-6410. Fixed MapReduce JobHistory server to use the right (login) UGI to refresh log and cleaner settings. Contributed by Varun Saxena.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d481684c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d481684c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d481684c

Branch: refs/heads/HADOOP-12111
Commit: d481684c7c9293a94f54ef622a92753531c6acc7
Parents: 41ae776
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Tue Jun 23 11:39:55 2015 -0700
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Tue Jun 23 11:39:55 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../mapreduce/v2/hs/server/HSAdminServer.java   | 43 ++++++++++++++-
 .../v2/hs/server/TestHSAdminServer.java         | 55 ++++++++++++++++++++
 3 files changed, 99 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d481684c/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 0cefde0..5eae44e 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -548,6 +548,9 @@ Release 2.7.1 - UNRELEASED
     MAPREDUCE-6387. Serialize the recently added Task#encryptedSpillKey field at 
     the end. (Arun Suresh via kasha)
 
+    MAPREDUCE-6410. Fixed MapReduce JobHistory server to use the right (login)
+    UGI to refresh log and cleaner settings. (Varun Saxena via vinodkv)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d481684c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
index ef79bf8..3fef5e2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/server/HSAdminServer.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2.hs.server;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.security.PrivilegedExceptionAction;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -52,6 +53,7 @@ import org.apache.hadoop.mapreduce.v2.hs.JobHistory;
 import org.apache.hadoop.mapreduce.v2.hs.proto.HSAdminRefreshProtocolProtos.HSAdminRefreshProtocolService;
 import org.apache.hadoop.mapreduce.v2.hs.protocolPB.HSAdminRefreshProtocolServerSideTranslatorPB;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.protobuf.BlockingService;
 
 @Private
@@ -67,6 +69,8 @@ public class HSAdminServer extends AbstractService implements HSAdminProtocol {
   private static final String HISTORY_ADMIN_SERVER = "HSAdminServer";
   private JobHistory jobHistoryService = null;
 
+  private UserGroupInformation loginUGI;
+
   public HSAdminServer(AggregatedLogDeletionService aggLogDelService,
       JobHistory jobHistoryService) {
     super(HSAdminServer.class.getName());
@@ -125,9 +129,24 @@ public class HSAdminServer extends AbstractService implements HSAdminProtocol {
 
   @Override
   protected void serviceStart() throws Exception {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      loginUGI = UserGroupInformation.getLoginUser();
+    } else {
+      loginUGI = UserGroupInformation.getCurrentUser();
+    }
     clientRpcServer.start();
   }
 
+  @VisibleForTesting
+  UserGroupInformation getLoginUGI() {
+    return loginUGI;
+  }
+
+  @VisibleForTesting
+  void setLoginUGI(UserGroupInformation ugi) {
+    loginUGI = ugi;
+  }
+
   @Override
   protected void serviceStop() throws Exception {
     if (clientRpcServer != null) {
@@ -233,7 +252,17 @@ public class HSAdminServer extends AbstractService implements HSAdminProtocol {
   public void refreshLogRetentionSettings() throws IOException {
     UserGroupInformation user = checkAcls("refreshLogRetentionSettings");
 
-    aggLogDelService.refreshLogRetentionSettings();
+    try {
+      loginUGI.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws IOException {
+          aggLogDelService.refreshLogRetentionSettings();
+          return null;
+        }
+      });
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
 
     HSAuditLogger.logSuccess(user.getShortUserName(),
         "refreshLogRetentionSettings", "HSAdminServer");
@@ -243,7 +272,17 @@ public class HSAdminServer extends AbstractService implements HSAdminProtocol {
   public void refreshJobRetentionSettings() throws IOException {
     UserGroupInformation user = checkAcls("refreshJobRetentionSettings");
 
-    jobHistoryService.refreshJobRetentionSettings();
+    try {
+      loginUGI.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override
+        public Void run() throws IOException {
+          jobHistoryService.refreshJobRetentionSettings();
+          return null;
+        }
+      });
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
 
     HSAuditLogger.logSuccess(user.getShortUserName(),
         "refreshJobRetentionSettings", HISTORY_ADMIN_SERVER);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d481684c/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/server/TestHSAdminServer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/server/TestHSAdminServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/server/TestHSAdminServer.java
index 7ab90f0..d831347 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/server/TestHSAdminServer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/server/TestHSAdminServer.java
@@ -21,6 +21,8 @@ package org.apache.hadoop.mapreduce.v2.hs.server;
 import static org.junit.Assert.*;
 
 import java.io.IOException;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -45,6 +47,9 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.reset;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.verify;
@@ -286,6 +291,56 @@ public class TestHSAdminServer {
     verify(jobHistoryService).refreshJobRetentionSettings();
   }
 
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testUGIForLogAndJobRefresh() throws Exception {
+    UserGroupInformation ugi =
+        UserGroupInformation.createUserForTesting("test", new String[] {"grp"});
+    UserGroupInformation loginUGI = spy(hsAdminServer.getLoginUGI());
+    hsAdminServer.setLoginUGI(loginUGI);
+
+    // Run refresh log retention settings with test user
+    ugi.doAs(new PrivilegedAction<Void>() {
+      @Override
+      public Void run() {
+        String[] args = new String[1];
+        args[0] = "-refreshLogRetentionSettings";
+        try {
+          hsAdminClient.run(args);
+        } catch (Exception e) {
+          fail("refreshLogRetentionSettings should have been successful");
+        }
+        return null;
+      }
+    });
+    // Verify if AggregatedLogDeletionService#refreshLogRetentionSettings was
+    // called with login UGI, instead of the UGI command was run with.
+    verify(loginUGI).doAs(any(PrivilegedExceptionAction.class));
+    verify(alds).refreshLogRetentionSettings();
+
+    // Reset for refresh job retention settings
+    reset(loginUGI);
+
+    // Run refresh job retention settings with test user
+    ugi.doAs(new PrivilegedAction<Void>() {
+      @Override
+      public Void run() {
+        String[] args = new String[1];
+        args[0] = "-refreshJobRetentionSettings";
+        try {
+          hsAdminClient.run(args);
+        } catch (Exception e) {
+          fail("refreshJobRetentionSettings should have been successful");
+        }
+        return null;
+      }
+    });
+    // Verify if JobHistory#refreshJobRetentionSettings was called with
+    // login UGI, instead of the UGI command was run with.
+    verify(loginUGI).doAs(any(PrivilegedExceptionAction.class));
+    verify(jobHistoryService).refreshJobRetentionSettings();
+  }
+
   @After
   public void cleanUp() {
     if (hsAdminServer != null)


[13/30] hadoop git commit: YARN-3360. Add JMX metrics to TimelineDataManager (Jason Lowe via jeagles)

Posted by aw...@apache.org.
YARN-3360. Add JMX metrics to TimelineDataManager (Jason Lowe via jeagles)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/4c659ddb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/4c659ddb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/4c659ddb

Branch: refs/heads/HADOOP-12111
Commit: 4c659ddbf7629aae92e66a5b54893e9c1c68dfb0
Parents: 2236b57
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Wed Jun 24 16:16:52 2015 -0500
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Wed Jun 24 16:16:52 2015 -0500

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../server/timeline/TimelineDataManager.java    | 123 +++++++++++++
 .../timeline/TimelineDataManagerMetrics.java    | 174 +++++++++++++++++++
 .../TestApplicationHistoryClientService.java    |   1 +
 ...pplicationHistoryManagerOnTimelineStore.java |   1 +
 .../webapp/TestAHSWebServices.java              |   1 +
 .../timeline/TestTimelineDataManager.java       |   1 +
 7 files changed, 303 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c659ddb/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f831f0a..9e70ec9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -60,6 +60,8 @@ Release 2.8.0 - UNRELEASED
 
   NEW FEATURES
 
+    YARN-3360. Add JMX metrics to TimelineDataManager (Jason Lowe via jeagles)
+
     YARN-3345. Add non-exclusive node label API. (Wangda Tan via jianhe)
 
     YARN-3365. Enhanced NodeManager to support using the 'tc' tool via

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c659ddb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
index 86aae77..459fd4e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManager.java
@@ -31,6 +31,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
 import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
@@ -56,6 +57,7 @@ public class TimelineDataManager extends AbstractService {
   @VisibleForTesting
   public static final String DEFAULT_DOMAIN_ID = "DEFAULT";
 
+  private TimelineDataManagerMetrics metrics;
   private TimelineStore store;
   private TimelineACLsManager timelineACLsManager;
 
@@ -69,6 +71,7 @@ public class TimelineDataManager extends AbstractService {
 
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
+    metrics = TimelineDataManagerMetrics.create();
     TimelineDomain domain = store.getDomain("DEFAULT");
     // it is okay to reuse an existing domain even if it was created by another
     // user of the timeline server before, because it allows everybody to access.
@@ -130,6 +133,38 @@ public class TimelineDataManager extends AbstractService {
       Long limit,
       EnumSet<Field> fields,
       UserGroupInformation callerUGI) throws YarnException, IOException {
+    long startTime = Time.monotonicNow();
+    metrics.incrGetEntitiesOps();
+    try {
+      TimelineEntities entities = doGetEntities(
+          entityType,
+          primaryFilter,
+          secondaryFilter,
+          windowStart,
+          windowEnd,
+          fromId,
+          fromTs,
+          limit,
+          fields,
+          callerUGI);
+      metrics.incrGetEntitiesTotal(entities.getEntities().size());
+      return entities;
+    } finally {
+      metrics.addGetEntitiesTime(Time.monotonicNow() - startTime);
+    }
+  }
+
+  private TimelineEntities doGetEntities(
+      String entityType,
+      NameValuePair primaryFilter,
+      Collection<NameValuePair> secondaryFilter,
+      Long windowStart,
+      Long windowEnd,
+      String fromId,
+      Long fromTs,
+      Long limit,
+      EnumSet<Field> fields,
+      UserGroupInformation callerUGI) throws YarnException, IOException {
     TimelineEntities entities = null;
     entities = store.getEntities(
         entityType,
@@ -161,6 +196,20 @@ public class TimelineDataManager extends AbstractService {
       String entityId,
       EnumSet<Field> fields,
       UserGroupInformation callerUGI) throws YarnException, IOException {
+    long startTime = Time.monotonicNow();
+    metrics.incrGetEntityOps();
+    try {
+      return doGetEntity(entityType, entityId, fields, callerUGI);
+    } finally {
+      metrics.addGetEntityTime(Time.monotonicNow() - startTime);
+    }
+  }
+
+  private TimelineEntity doGetEntity(
+      String entityType,
+      String entityId,
+      EnumSet<Field> fields,
+      UserGroupInformation callerUGI) throws YarnException, IOException {
     TimelineEntity entity = null;
     entity =
         store.getEntity(entityId, entityType, fields);
@@ -190,6 +239,32 @@ public class TimelineDataManager extends AbstractService {
       Long windowEnd,
       Long limit,
       UserGroupInformation callerUGI) throws YarnException, IOException {
+    long startTime = Time.monotonicNow();
+    metrics.incrGetEventsOps();
+    try {
+      TimelineEvents events = doGetEvents(
+          entityType,
+          entityIds,
+          eventTypes,
+          windowStart,
+          windowEnd,
+          limit,
+          callerUGI);
+      metrics.incrGetEventsTotal(events.getAllEvents().size());
+      return events;
+    } finally {
+      metrics.addGetEventsTime(Time.monotonicNow() - startTime);
+    }
+  }
+
+  private TimelineEvents doGetEvents(
+      String entityType,
+      SortedSet<String> entityIds,
+      SortedSet<String> eventTypes,
+      Long windowStart,
+      Long windowEnd,
+      Long limit,
+      UserGroupInformation callerUGI) throws YarnException, IOException {
     TimelineEvents events = null;
     events = store.getEntityTimelines(
         entityType,
@@ -236,9 +311,22 @@ public class TimelineDataManager extends AbstractService {
   public TimelinePutResponse postEntities(
       TimelineEntities entities,
       UserGroupInformation callerUGI) throws YarnException, IOException {
+    long startTime = Time.monotonicNow();
+    metrics.incrPostEntitiesOps();
+    try {
+      return doPostEntities(entities, callerUGI);
+    } finally {
+      metrics.addPostEntitiesTime(Time.monotonicNow() - startTime);
+    }
+  }
+
+  private TimelinePutResponse doPostEntities(
+      TimelineEntities entities,
+      UserGroupInformation callerUGI) throws YarnException, IOException {
     if (entities == null) {
       return new TimelinePutResponse();
     }
+    metrics.incrPostEntitiesTotal(entities.getEntities().size());
     TimelineEntities entitiesToPut = new TimelineEntities();
     List<TimelinePutResponse.TimelinePutError> errors =
         new ArrayList<TimelinePutResponse.TimelinePutError>();
@@ -303,6 +391,17 @@ public class TimelineDataManager extends AbstractService {
    */
   public void putDomain(TimelineDomain domain,
       UserGroupInformation callerUGI) throws YarnException, IOException {
+    long startTime = Time.monotonicNow();
+    metrics.incrPutDomainOps();
+    try {
+      doPutDomain(domain, callerUGI);
+    } finally {
+      metrics.addPutDomainTime(Time.monotonicNow() - startTime);
+    }
+  }
+
+  private void doPutDomain(TimelineDomain domain,
+      UserGroupInformation callerUGI) throws YarnException, IOException {
     TimelineDomain existingDomain =
         store.getDomain(domain.getId());
     if (existingDomain != null) {
@@ -329,6 +428,17 @@ public class TimelineDataManager extends AbstractService {
    */
   public TimelineDomain getDomain(String domainId,
       UserGroupInformation callerUGI) throws YarnException, IOException {
+    long startTime = Time.monotonicNow();
+    metrics.incrGetDomainOps();
+    try {
+      return doGetDomain(domainId, callerUGI);
+    } finally {
+      metrics.addGetDomainTime(Time.monotonicNow() - startTime);
+    }
+  }
+
+  private TimelineDomain doGetDomain(String domainId,
+      UserGroupInformation callerUGI) throws YarnException, IOException {
     TimelineDomain domain = store.getDomain(domainId);
     if (domain != null) {
       if (timelineACLsManager.checkAccess(callerUGI, domain)) {
@@ -344,6 +454,19 @@ public class TimelineDataManager extends AbstractService {
    */
   public TimelineDomains getDomains(String owner,
       UserGroupInformation callerUGI) throws YarnException, IOException {
+    long startTime = Time.monotonicNow();
+    metrics.incrGetDomainsOps();
+    try {
+      TimelineDomains domains = doGetDomains(owner, callerUGI);
+      metrics.incrGetDomainsTotal(domains.getDomains().size());
+      return domains;
+    } finally {
+      metrics.addGetDomainsTime(Time.monotonicNow() - startTime);
+    }
+  }
+
+  private TimelineDomains doGetDomains(String owner,
+      UserGroupInformation callerUGI) throws YarnException, IOException {
     TimelineDomains domains = store.getDomains(owner);
     boolean hasAccess = true;
     if (domains.getDomains().size() > 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c659ddb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManagerMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManagerMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManagerMetrics.java
new file mode 100644
index 0000000..afd5818
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/TimelineDataManagerMetrics.java
@@ -0,0 +1,174 @@
+/*
+ * 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.yarn.server.timeline;
+
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableRate;
+
+/** This class tracks metrics for the TimelineDataManager. */
+@Metrics(about="Metrics for TimelineDataManager", context="yarn")
+public class TimelineDataManagerMetrics {
+  @Metric("getEntities calls")
+  MutableCounterLong getEntitiesOps;
+
+  @Metric("Entities returned via getEntities")
+  MutableCounterLong getEntitiesTotal;
+
+  @Metric("getEntities processing time")
+  MutableRate getEntitiesTime;
+
+  @Metric("getEntity calls")
+  MutableCounterLong getEntityOps;
+
+  @Metric("getEntity processing time")
+  MutableRate getEntityTime;
+
+  @Metric("getEvents calls")
+  MutableCounterLong getEventsOps;
+
+  @Metric("Events returned via getEvents")
+  MutableCounterLong getEventsTotal;
+
+  @Metric("getEvents processing time")
+  MutableRate getEventsTime;
+
+  @Metric("postEntities calls")
+  MutableCounterLong postEntitiesOps;
+
+  @Metric("Entities posted via postEntities")
+  MutableCounterLong postEntitiesTotal;
+
+  @Metric("postEntities processing time")
+  MutableRate postEntitiesTime;
+
+  @Metric("putDomain calls")
+  MutableCounterLong putDomainOps;
+
+  @Metric("putDomain processing time")
+  MutableRate putDomainTime;
+
+  @Metric("getDomain calls")
+  MutableCounterLong getDomainOps;
+
+  @Metric("getDomain processing time")
+  MutableRate getDomainTime;
+
+  @Metric("getDomains calls")
+  MutableCounterLong getDomainsOps;
+
+  @Metric("Domains returned via getDomains")
+  MutableCounterLong getDomainsTotal;
+
+  @Metric("getDomains processing time")
+  MutableRate getDomainsTime;
+
+  @Metric("Total calls")
+  public long totalOps() {
+    return getEntitiesOps.value() +
+        getEntityOps.value() +
+        getEventsOps.value() +
+        postEntitiesOps.value() +
+        putDomainOps.value() +
+        getDomainOps.value() +
+        getDomainsOps.value();
+  }
+
+  TimelineDataManagerMetrics() {
+  }
+
+  public static TimelineDataManagerMetrics create() {
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    return ms.register(new TimelineDataManagerMetrics());
+  }
+
+  public void incrGetEntitiesOps() {
+    getEntitiesOps.incr();
+  }
+
+  public void incrGetEntitiesTotal(long delta) {
+    getEntitiesTotal.incr(delta);
+  }
+
+  public void addGetEntitiesTime(long msec) {
+    getEntitiesTime.add(msec);
+  }
+
+  public void incrGetEntityOps() {
+    getEntityOps.incr();
+  }
+
+  public void addGetEntityTime(long msec) {
+    getEntityTime.add(msec);
+  }
+
+  public void incrGetEventsOps() {
+    getEventsOps.incr();
+  }
+
+  public void incrGetEventsTotal(long delta) {
+    getEventsTotal.incr(delta);
+  }
+
+  public void addGetEventsTime(long msec) {
+    getEventsTime.add(msec);
+  }
+
+  public void incrPostEntitiesOps() {
+    postEntitiesOps.incr();
+  }
+
+  public void incrPostEntitiesTotal(long delta) {
+    postEntitiesTotal.incr(delta);
+  }
+
+  public void addPostEntitiesTime(long msec) {
+    postEntitiesTime.add(msec);
+  }
+
+  public void incrPutDomainOps() {
+    putDomainOps.incr();
+  }
+
+  public void addPutDomainTime(long msec) {
+    putDomainTime.add(msec);
+  }
+
+  public void incrGetDomainOps() {
+    getDomainOps.incr();
+  }
+
+  public void addGetDomainTime(long msec) {
+    getDomainTime.add(msec);
+  }
+
+  public void incrGetDomainsOps() {
+    getDomainsOps.incr();
+  }
+
+  public void incrGetDomainsTotal(long delta) {
+    getDomainsTotal.incr(delta);
+  }
+
+  public void addGetDomainsTime(long msec) {
+    getDomainsTime.add(msec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c659ddb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
index d04903c..1e98e8d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
@@ -67,6 +67,7 @@ public class TestApplicationHistoryClientService {
     TimelineACLsManager aclsManager = new TimelineACLsManager(conf);
     dataManager =
         new TimelineDataManager(store, aclsManager);
+    dataManager.init(conf);
     ApplicationACLsManager appAclsManager = new ApplicationACLsManager(conf);
     ApplicationHistoryManagerOnTimelineStore historyManager =
         new ApplicationHistoryManagerOnTimelineStore(dataManager, appAclsManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c659ddb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
index 1e5dc5d..f553b79 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
@@ -98,6 +98,7 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     TimelineACLsManager aclsManager = new TimelineACLsManager(new YarnConfiguration());
     TimelineDataManager dataManager =
         new TimelineDataManager(store, aclsManager);
+    dataManager.init(conf);
     ApplicationACLsManager appAclsManager = new ApplicationACLsManager(conf);
     historyManager =
         new ApplicationHistoryManagerOnTimelineStore(dataManager, appAclsManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c659ddb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
index b5ff6a4..f2179b4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
@@ -94,6 +94,7 @@ public class TestAHSWebServices extends JerseyTestBase {
         new TimelineDataManager(store, aclsManager);
     conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "foo");
+    dataManager.init(conf);
     ApplicationACLsManager appAclsManager = new ApplicationACLsManager(conf);
     ApplicationHistoryManagerOnTimelineStore historyManager =
         new ApplicationHistoryManagerOnTimelineStore(dataManager, appAclsManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4c659ddb/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
index 87c3b24..ace2eb8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TestTimelineDataManager.java
@@ -65,6 +65,7 @@ public class TestTimelineDataManager extends TimelineStoreTestUtils {
     dataManaer = new TimelineDataManager(store, aclsManager);
     conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "admin");
+    dataManaer.init(conf);
     adminACLsManager = new AdminACLsManager(conf);
   }
 


[05/30] hadoop git commit: HDFS-6440. Support more than 2 NameNodes. Contributed by Jesse Yates.

Posted by aw...@apache.org.
HDFS-6440. Support more than 2 NameNodes. Contributed by Jesse Yates.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/49dfad94
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/49dfad94
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/49dfad94

Branch: refs/heads/HADOOP-12111
Commit: 49dfad942970459297f72632ed8dfd353e0c86de
Parents: 122cad6
Author: Aaron T. Myers <at...@apache.org>
Authored: Tue Jun 23 17:26:11 2015 -0700
Committer: Aaron T. Myers <at...@apache.org>
Committed: Tue Jun 23 17:26:11 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/ha/ZKFailoverController.java  |  61 ++-
 .../org/apache/hadoop/ha/MiniZKFCCluster.java   |  93 +++-
 .../hadoop/ha/TestZKFailoverController.java     |  32 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../bkjournal/TestBookKeeperHACheckpoints.java  |   7 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   4 +
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |  36 +-
 .../token/block/BlockTokenSecretManager.java    |  40 +-
 .../server/blockmanagement/BlockManager.java    |  21 +-
 .../hdfs/server/namenode/CheckpointConf.java    |  14 +-
 .../hdfs/server/namenode/ImageServlet.java      |  88 +++-
 .../server/namenode/NameNodeHttpServer.java     |   7 +-
 .../hdfs/server/namenode/TransferFsImage.java   |  47 +-
 .../server/namenode/ha/BootstrapStandby.java    |  94 ++--
 .../hdfs/server/namenode/ha/EditLogTailer.java  | 162 +++++--
 .../server/namenode/ha/RemoteNameNodeInfo.java  | 133 ++++++
 .../server/namenode/ha/StandbyCheckpointer.java | 182 +++++---
 .../hdfs/tools/DFSZKFailoverController.java     |  13 +
 .../src/main/resources/hdfs-default.xml         |  20 +
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  | 464 ++++++++++++-------
 .../apache/hadoop/hdfs/MiniDFSNNTopology.java   |  18 +-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |   6 +-
 .../apache/hadoop/hdfs/TestRollingUpgrade.java  | 109 +++--
 .../hadoop/hdfs/qjournal/MiniQJMHACluster.java  |  94 ++--
 .../security/token/block/TestBlockToken.java    |  10 +-
 .../hdfs/server/namenode/TestBackupNode.java    |   2 +-
 .../hdfs/server/namenode/TestCheckpoint.java    |   3 +-
 .../server/namenode/ha/HAStressTestHarness.java |  46 +-
 .../hdfs/server/namenode/ha/HATestUtil.java     |  60 ++-
 .../namenode/ha/TestBootstrapStandby.java       | 176 ++++---
 .../ha/TestBootstrapStandbyWithQJM.java         |  47 +-
 .../ha/TestDNFencingWithReplication.java        |   1 +
 .../server/namenode/ha/TestEditLogTailer.java   |  12 +-
 .../ha/TestFailoverWithBlockTokensEnabled.java  |  55 ++-
 .../server/namenode/ha/TestHAConfiguration.java |  49 +-
 .../namenode/ha/TestPipelinesFailover.java      | 110 +++--
 .../namenode/ha/TestRemoteNameNodeInfo.java     |  61 +++
 .../namenode/ha/TestSeveralNameNodes.java       | 179 +++++++
 .../namenode/ha/TestStandbyCheckpoints.java     | 106 +++--
 .../src/test/resources/hadoop-0.23-reserved.tgz | Bin 4558 -> 5590 bytes
 .../src/test/resources/hadoop-1-reserved.tgz    | Bin 2572 -> 3348 bytes
 .../src/test/resources/hadoop-2-reserved.tgz    | Bin 2838 -> 3465 bytes
 .../src/test/resources/hadoop-22-dfs-dir.tgz    | Bin 318180 -> 413239 bytes
 .../src/test/resources/hadoop1-bbw.tgz          | Bin 40234 -> 43294 bytes
 .../src/test/resources/log4j.properties         |   2 +-
 45 files changed, 1926 insertions(+), 740 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
index b1f5920..30ec8f2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.Executors;
@@ -141,6 +142,7 @@ public abstract class ZKFailoverController {
       throws AccessControlException, IOException;
   protected abstract InetSocketAddress getRpcAddressToBindTo();
   protected abstract PolicyProvider getPolicyProvider();
+  protected abstract List<HAServiceTarget> getAllOtherNodes();
 
   /**
    * Return the name of a znode inside the configured parent znode in which
@@ -616,9 +618,11 @@ public abstract class ZKFailoverController {
    * Coordinate a graceful failover. This proceeds in several phases:
    * 1) Pre-flight checks: ensure that the local node is healthy, and
    * thus a candidate for failover.
-   * 2) Determine the current active node. If it is the local node, no
+   * 2a) Determine the current active node. If it is the local node, no
    * need to failover - return success.
-   * 3) Ask that node to yield from the election for a number of seconds.
+   * 2b) Get the other nodes
+   * 3a) Ask the other nodes to yield from election for a number of seconds
+   * 3b) Ask the active node to yield from the election for a number of seconds.
    * 4) Allow the normal election path to run in other threads. Wait until
    * we either become unhealthy or we see an election attempt recorded by
    * the normal code path.
@@ -648,12 +652,27 @@ public abstract class ZKFailoverController {
           "No need to failover. Returning success.");
       return;
     }
-    
-    // Phase 3: ask the old active to yield from the election.
-    LOG.info("Asking " + oldActive + " to cede its active state for " +
-        timeout + "ms");
-    ZKFCProtocol oldZkfc = oldActive.getZKFCProxy(conf, timeout);
-    oldZkfc.cedeActive(timeout);
+
+    // Phase 2b: get the other nodes
+    List<HAServiceTarget> otherNodes = getAllOtherNodes();
+    List<ZKFCProtocol> otherZkfcs = new ArrayList<ZKFCProtocol>(otherNodes.size());
+
+    // Phase 3: ask the other nodes to yield from the election.
+    HAServiceTarget activeNode = null;
+    for (HAServiceTarget remote : otherNodes) {
+      // same location, same node - may not always be == equality
+      if (remote.getAddress().equals(oldActive.getAddress())) {
+        activeNode = remote;
+        continue;
+      }
+      otherZkfcs.add(cedeRemoteActive(remote, timeout));
+    }
+
+    assert
+      activeNode != null : "Active node does not match any known remote node";
+
+    // Phase 3b: ask the old active to yield
+    otherZkfcs.add(cedeRemoteActive(activeNode, timeout));
 
     // Phase 4: wait for the normal election to make the local node
     // active.
@@ -676,8 +695,10 @@ public abstract class ZKFailoverController {
     // Phase 5. At this point, we made some attempt to become active. So we
     // can tell the old active to rejoin if it wants. This allows a quick
     // fail-back if we immediately crash.
-    oldZkfc.cedeActive(-1);
-    
+    for (ZKFCProtocol zkfc : otherZkfcs) {
+      zkfc.cedeActive(-1);
+    }
+
     if (attempt.succeeded) {
       LOG.info("Successfully became active. " + attempt.status);
     } else {
@@ -688,6 +709,23 @@ public abstract class ZKFailoverController {
   }
 
   /**
+   * Ask the remote zkfc to cede its active status and wait for the specified
+   * timeout before attempting to claim leader status.
+   * @param remote node to ask
+   * @param timeout amount of time to cede
+   * @return the {@link ZKFCProtocol} used to talk to the ndoe
+   * @throws IOException
+   */
+  private ZKFCProtocol cedeRemoteActive(HAServiceTarget remote, int timeout)
+    throws IOException {
+    LOG.info("Asking " + remote + " to cede its active state for "
+               + timeout + "ms");
+    ZKFCProtocol oldZkfc = remote.getZKFCProxy(conf, timeout);
+    oldZkfc.cedeActive(timeout);
+    return oldZkfc;
+  }
+
+  /**
    * Ensure that the local node is in a healthy state, and thus
    * eligible for graceful failover.
    * @throws ServiceFailedException if the node is unhealthy
@@ -777,7 +815,8 @@ public abstract class ZKFailoverController {
           break;
           
         default:
-          throw new IllegalArgumentException("Unhandled state:" + lastHealthState);
+          throw new IllegalArgumentException("Unhandled state:"
+                                               + lastHealthState);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
index 5aee611..b496bf9 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/MiniZKFCCluster.java
@@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -49,7 +51,7 @@ public class MiniZKFCCluster {
   private final TestContext ctx;
   private final ZooKeeperServer zks;
 
-  private DummyHAService svcs[];
+  private List<DummyHAService> svcs;
   private DummyZKFCThread thrs[];
   private Configuration conf;
   
@@ -63,38 +65,67 @@ public class MiniZKFCCluster {
     conf.setInt(CommonConfigurationKeys.HA_HM_CHECK_INTERVAL_KEY, 50);
     conf.setInt(CommonConfigurationKeys.HA_HM_CONNECT_RETRY_INTERVAL_KEY, 50);
     conf.setInt(CommonConfigurationKeys.HA_HM_SLEEP_AFTER_DISCONNECT_KEY, 50);
-    svcs = new DummyHAService[2];
-    svcs[0] = new DummyHAService(HAServiceState.INITIALIZING,
-        new InetSocketAddress("svc1", 1234));
-    svcs[0].setSharedResource(sharedResource);
-    svcs[1] = new DummyHAService(HAServiceState.INITIALIZING,
-        new InetSocketAddress("svc2", 1234));
-    svcs[1].setSharedResource(sharedResource);
-    
+    svcs = new ArrayList<DummyHAService>(2);
+    // remove any existing instances we are keeping track of
+    DummyHAService.instances.clear();
+
+    for (int i = 0; i < 2; i++) {
+      addSvcs(svcs, i);
+    }
+
     this.ctx = new TestContext();
     this.zks = zks;
   }
-  
+
+  private void addSvcs(List<DummyHAService> svcs, int i) {
+    svcs.add(new DummyHAService(HAServiceState.INITIALIZING, new InetSocketAddress("svc" + (i + 1),
+        1234)));
+    svcs.get(i).setSharedResource(sharedResource);
+  }
+
   /**
    * Set up two services and their failover controllers. svc1 is started
    * first, so that it enters ACTIVE state, and then svc2 is started,
    * which enters STANDBY
    */
   public void start() throws Exception {
+    start(2);
+  }
+
+  /**
+   * Set up the specified number of services and their failover controllers. svc1 is
+   * started first, so that it enters ACTIVE state, and then svc2...svcN is started, which enters
+   * STANDBY.
+   * <p>
+   * Adds any extra svc needed beyond the first two before starting the rest of the cluster.
+   * @param count number of zkfcs to start
+   */
+  public void start(int count) throws Exception {
+    // setup the expected number of zkfcs, if we need to add more. This seemed the least invasive
+    // way to add the services - otherwise its a large test rewrite or changing a lot of assumptions
+    if (count > 2) {
+      for (int i = 2; i < count; i++) {
+        addSvcs(svcs, i);
+      }
+    }
+
     // Format the base dir, should succeed
-    thrs = new DummyZKFCThread[2];
-    thrs[0] = new DummyZKFCThread(ctx, svcs[0]);
+    thrs = new DummyZKFCThread[count];
+    thrs[0] = new DummyZKFCThread(ctx, svcs.get(0));
     assertEquals(0, thrs[0].zkfc.run(new String[]{"-formatZK"}));
     ctx.addThread(thrs[0]);
     thrs[0].start();
     
     LOG.info("Waiting for svc0 to enter active state");
     waitForHAState(0, HAServiceState.ACTIVE);
-    
-    LOG.info("Adding svc1");
-    thrs[1] = new DummyZKFCThread(ctx, svcs[1]);
-    thrs[1].start();
-    waitForHAState(1, HAServiceState.STANDBY);
+
+    // add the remaining zkfc
+    for (int i = 1; i < count; i++) {
+      LOG.info("Adding svc" + i);
+      thrs[i] = new DummyZKFCThread(ctx, svcs.get(i));
+      thrs[i].start();
+      waitForHAState(i, HAServiceState.STANDBY);
+    }
   }
   
   /**
@@ -122,7 +153,7 @@ public class MiniZKFCCluster {
   }
   
   public DummyHAService getService(int i) {
-    return svcs[i];
+    return svcs.get(i);
   }
 
   public ActiveStandbyElector getElector(int i) {
@@ -134,23 +165,23 @@ public class MiniZKFCCluster {
   }
   
   public void setHealthy(int idx, boolean healthy) {
-    svcs[idx].isHealthy = healthy;
+    svcs.get(idx).isHealthy = healthy;
   }
 
   public void setFailToBecomeActive(int idx, boolean doFail) {
-    svcs[idx].failToBecomeActive = doFail;
+    svcs.get(idx).failToBecomeActive = doFail;
   }
 
   public void setFailToBecomeStandby(int idx, boolean doFail) {
-    svcs[idx].failToBecomeStandby = doFail;
+    svcs.get(idx).failToBecomeStandby = doFail;
   }
   
   public void setFailToFence(int idx, boolean doFail) {
-    svcs[idx].failToFence = doFail;
+    svcs.get(idx).failToFence = doFail;
   }
   
   public void setUnreachable(int idx, boolean unreachable) {
-    svcs[idx].actUnreachable = unreachable;
+    svcs.get(idx).actUnreachable = unreachable;
   }
 
   /**
@@ -204,7 +235,7 @@ public class MiniZKFCCluster {
     byte[] data = zks.getZKDatabase().getData(
         DummyZKFC.LOCK_ZNODE, stat, null);
     
-    assertArrayEquals(Ints.toByteArray(svcs[idx].index), data);
+    assertArrayEquals(Ints.toByteArray(svcs.get(idx).index), data);
     long session = stat.getEphemeralOwner();
     LOG.info("Expiring svc " + idx + "'s zookeeper session " + session);
     zks.closeSession(session);
@@ -218,7 +249,7 @@ public class MiniZKFCCluster {
    */
   public void waitForActiveLockHolder(Integer idx)
       throws Exception {
-    DummyHAService svc = idx == null ? null : svcs[idx];
+    DummyHAService svc = idx == null ? null : svcs.get(idx);
     ActiveStandbyElectorTestUtil.waitForActiveLockData(ctx, zks,
         DummyZKFC.SCOPED_PARENT_ZNODE,
         (idx == null) ? null : Ints.toByteArray(svc.index));
@@ -320,5 +351,17 @@ public class MiniZKFCCluster {
     protected PolicyProvider getPolicyProvider() {
       return null;
     }
+
+    @Override
+    protected List<HAServiceTarget> getAllOtherNodes() {
+      List<HAServiceTarget> services = new ArrayList<HAServiceTarget>(
+          DummyHAService.instances.size());
+      for (DummyHAService service : DummyHAService.instances) {
+        if (service != this.localTarget) {
+          services.add(service);
+        }
+      }
+      return services;
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
index d8271c5..b8d9ce4 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestZKFailoverController.java
@@ -605,6 +605,38 @@ public class TestZKFailoverController extends ClientBaseWithFixes {
     }
   }
 
+  @Test(timeout = 25000)
+  public void testGracefulFailoverMultipleZKfcs() throws Exception {
+    try {
+      cluster.start(3);
+
+      cluster.waitForActiveLockHolder(0);
+
+      // failover to first
+      cluster.getService(1).getZKFCProxy(conf, 5000).gracefulFailover();
+      cluster.waitForActiveLockHolder(1);
+
+      // failover to second
+      cluster.getService(2).getZKFCProxy(conf, 5000).gracefulFailover();
+      cluster.waitForActiveLockHolder(2);
+
+      // failover back to original
+      cluster.getService(0).getZKFCProxy(conf, 5000).gracefulFailover();
+      cluster.waitForActiveLockHolder(0);
+
+      Thread.sleep(10000); // allow to quiesce
+
+      assertEquals(0, cluster.getService(0).fenceCount);
+      assertEquals(0, cluster.getService(1).fenceCount);
+      assertEquals(0, cluster.getService(2).fenceCount);
+      assertEquals(2, cluster.getService(0).activeTransitionCount);
+      assertEquals(1, cluster.getService(1).activeTransitionCount);
+      assertEquals(1, cluster.getService(2).activeTransitionCount);
+    } finally {
+      cluster.stop();
+    }
+  }
+
   private int runFC(DummyHAService target, String ... args) throws Exception {
     DummyZKFC zkfc = new DummyZKFC(conf, target);
     return zkfc.run(args);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 981ca55..09a6891 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -38,6 +38,8 @@ Trunk (Unreleased)
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)
 
+    HDFS-6440. Support more than 2 NameNodes. (Jesse Yates via atm)
+
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java
index b74cd7f..ed53512 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java
@@ -32,6 +32,10 @@ import org.junit.BeforeClass;
  * using a bookkeeper journal manager as the shared directory
  */
 public class TestBookKeeperHACheckpoints extends TestStandbyCheckpoints {
+  //overwrite the nn count
+ static{
+   TestStandbyCheckpoints.NUM_NNS = 2;
+ }
   private static BKJMUtil bkutil = null;
   static int numBookies = 3;
   static int journalCount = 0;
@@ -57,8 +61,7 @@ public class TestBookKeeperHACheckpoints extends TestStandbyCheckpoints {
       .build();
     cluster.waitActive();
 
-    nn0 = cluster.getNameNode(0);
-    nn1 = cluster.getNameNode(1);
+    setNNs();
     fs = HATestUtil.configureFailoverFs(cluster, conf);
 
     cluster.transitionToActive(0);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.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 30540a9..ebd668f 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
@@ -132,6 +132,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50090";
   public static final String  DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_KEY = "dfs.namenode.secondary.https-address";
   public static final String  DFS_NAMENODE_SECONDARY_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:50091";
+  public static final String  DFS_NAMENODE_CHECKPOINT_QUIET_MULTIPLIER_KEY = "dfs.namenode.checkpoint.check.quiet-multiplier";
+  public static final double  DFS_NAMENODE_CHECKPOINT_QUIET_MULTIPLIER_DEFAULT = 1.5;
   public static final String  DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY = "dfs.namenode.checkpoint.check.period";
   public static final long    DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_DEFAULT = 60;
   public static final String  DFS_NAMENODE_CHECKPOINT_PERIOD_KEY = "dfs.namenode.checkpoint.period";
@@ -544,6 +546,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int DFS_HA_LOGROLL_PERIOD_DEFAULT = 2 * 60; // 2m
   public static final String DFS_HA_TAILEDITS_PERIOD_KEY = "dfs.ha.tail-edits.period";
   public static final int DFS_HA_TAILEDITS_PERIOD_DEFAULT = 60; // 1m
+  public static final String DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_KEY = "dfs.ha.tail-edits.namenode-retries";
+  public static final int DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_DEFAULT = 3;
   public static final String DFS_HA_LOGROLL_RPC_TIMEOUT_KEY = "dfs.ha.log-roll.rpc.timeout";
   public static final int DFS_HA_LOGROLL_RPC_TIMEOUT_DEFAULT = 20000; // 20s
   public static final String DFS_HA_FENCE_METHODS_KEY = "dfs.ha.fencing.methods";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
index c967c69..686a0b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HAUtil.java
@@ -143,7 +143,7 @@ public class HAUtil {
    * @param conf the configuration of this node
    * @return the NN ID of the other node in this nameservice
    */
-  public static String getNameNodeIdOfOtherNode(Configuration conf, String nsId) {
+  public static List<String> getNameNodeIdOfOtherNodes(Configuration conf, String nsId) {
     Preconditions.checkArgument(nsId != null,
         "Could not determine namespace id. Please ensure that this " +
         "machine is one of the machines listed as a NN RPC address, " +
@@ -157,20 +157,20 @@ public class HAUtil {
         DFSUtil.addKeySuffixes(DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX,
             nsId),
         nsId);
-    Preconditions.checkArgument(nnIds.size() == 2,
-        "Expected exactly 2 NameNodes in namespace '%s'. " +
-        "Instead, got only %s (NN ids were '%s'",
-        nsId, nnIds.size(), Joiner.on("','").join(nnIds));
+    Preconditions.checkArgument(nnIds.size() >= 2,
+        "Expected at least 2 NameNodes in namespace '%s'. " +
+          "Instead, got only %s (NN ids were '%s')",
+          nsId, nnIds.size(), Joiner.on("','").join(nnIds));
     Preconditions.checkState(myNNId != null && !myNNId.isEmpty(),
         "Could not determine own NN ID in namespace '%s'. Please " +
         "ensure that this node is one of the machines listed as an " +
         "NN RPC address, or configure " + DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY,
         nsId);
 
-    ArrayList<String> nnSet = Lists.newArrayList(nnIds);
-    nnSet.remove(myNNId);
-    assert nnSet.size() == 1;
-    return nnSet.get(0);
+    ArrayList<String> namenodes = Lists.newArrayList(nnIds);
+    namenodes.remove(myNNId);
+    assert namenodes.size() >= 1;
+    return namenodes;
   }
 
   /**
@@ -180,16 +180,20 @@ public class HAUtil {
    * @param myConf the configuration of this node
    * @return the configuration of the other node in an HA setup
    */
-  public static Configuration getConfForOtherNode(
+  public static List<Configuration> getConfForOtherNodes(
       Configuration myConf) {
     
     String nsId = DFSUtil.getNamenodeNameServiceId(myConf);
-    String otherNn = getNameNodeIdOfOtherNode(myConf, nsId);
-    
-    // Look up the address of the active NN.
-    Configuration confForOtherNode = new Configuration(myConf);
-    NameNode.initializeGenericKeys(confForOtherNode, nsId, otherNn);
-    return confForOtherNode;
+    List<String> otherNn = getNameNodeIdOfOtherNodes(myConf, nsId);
+
+    // Look up the address of the other NNs
+    List<Configuration> confs = new ArrayList<Configuration>(otherNn.size());
+    for (String nn : otherNn) {
+      Configuration confForOtherNode = new Configuration(myConf);
+      NameNode.initializeGenericKeys(confForOtherNode, nsId, nn);
+      confs.add(confForOtherNode);
+    }
+    return confs;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
index b103c1a..53da44c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java
@@ -52,17 +52,11 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 public class BlockTokenSecretManager extends
     SecretManager<BlockTokenIdentifier> {
-  public static final Log LOG = LogFactory
-      .getLog(BlockTokenSecretManager.class);
-  
-  // We use these in an HA setup to ensure that the pair of NNs produce block
-  // token serial numbers that are in different ranges.
-  private static final int LOW_MASK  = ~(1 << 31);
-  
+  public static final Log LOG = LogFactory.getLog(BlockTokenSecretManager.class);
+
   public static final Token<BlockTokenIdentifier> DUMMY_TOKEN = new Token<BlockTokenIdentifier>();
 
   private final boolean isMaster;
-  private int nnIndex;
   
   /**
    * keyUpdateInterval is the interval that NN updates its block keys. It should
@@ -77,21 +71,22 @@ public class BlockTokenSecretManager extends
   private final Map<Integer, BlockKey> allKeys;
   private String blockPoolId;
   private final String encryptionAlgorithm;
-  
+
+  private final int intRange;
+  private final int nnRangeStart;
+
   private final SecureRandom nonceGenerator = new SecureRandom();
 
-  ;
-  
   /**
    * Constructor for slaves.
-   * 
+   *
    * @param keyUpdateInterval how often a new key will be generated
    * @param tokenLifetime how long an individual token is valid
    */
   public BlockTokenSecretManager(long keyUpdateInterval,
       long tokenLifetime, String blockPoolId, String encryptionAlgorithm) {
     this(false, keyUpdateInterval, tokenLifetime, blockPoolId,
-        encryptionAlgorithm);
+        encryptionAlgorithm, 0, 1);
   }
   
   /**
@@ -99,23 +94,25 @@ public class BlockTokenSecretManager extends
    * 
    * @param keyUpdateInterval how often a new key will be generated
    * @param tokenLifetime how long an individual token is valid
-   * @param nnIndex namenode index
+   * @param nnIndex namenode index of the namenode for which we are creating the manager
    * @param blockPoolId block pool ID
    * @param encryptionAlgorithm encryption algorithm to use
+   * @param numNNs number of namenodes possible
    */
   public BlockTokenSecretManager(long keyUpdateInterval,
-      long tokenLifetime, int nnIndex, String blockPoolId,
+      long tokenLifetime, int nnIndex, int numNNs,  String blockPoolId,
       String encryptionAlgorithm) {
-    this(true, keyUpdateInterval, tokenLifetime, blockPoolId,
-        encryptionAlgorithm);
-    Preconditions.checkArgument(nnIndex == 0 || nnIndex == 1);
-    this.nnIndex = nnIndex;
+    this(true, keyUpdateInterval, tokenLifetime, blockPoolId, encryptionAlgorithm, nnIndex, numNNs);
+    Preconditions.checkArgument(nnIndex >= 0);
+    Preconditions.checkArgument(numNNs > 0);
     setSerialNo(new SecureRandom().nextInt());
     generateKeys();
   }
   
   private BlockTokenSecretManager(boolean isMaster, long keyUpdateInterval,
-      long tokenLifetime, String blockPoolId, String encryptionAlgorithm) {
+      long tokenLifetime, String blockPoolId, String encryptionAlgorithm, int nnIndex, int numNNs) {
+    this.intRange = Integer.MAX_VALUE / numNNs;
+    this.nnRangeStart = intRange * nnIndex;
     this.isMaster = isMaster;
     this.keyUpdateInterval = keyUpdateInterval;
     this.tokenLifetime = tokenLifetime;
@@ -127,7 +124,8 @@ public class BlockTokenSecretManager extends
   
   @VisibleForTesting
   public synchronized void setSerialNo(int serialNo) {
-    this.serialNo = (serialNo & LOW_MASK) | (nnIndex << 31);
+    // we mod the serial number by the range and then add that times the index
+    this.serialNo = (serialNo % intRange) + (nnRangeStart);
   }
   
   public void setBlockPoolId(String blockPoolId) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 824801f..7d3a678 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
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.IOException;
@@ -42,6 +43,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -399,14 +401,21 @@ public class BlockManager {
     boolean isHaEnabled = HAUtil.isHAEnabled(conf, nsId);
 
     if (isHaEnabled) {
-      String thisNnId = HAUtil.getNameNodeId(conf, nsId);
-      String otherNnId = HAUtil.getNameNodeIdOfOtherNode(conf, nsId);
-      return new BlockTokenSecretManager(updateMin*60*1000L,
-          lifetimeMin*60*1000L, thisNnId.compareTo(otherNnId) < 0 ? 0 : 1, null,
-          encryptionAlgorithm);
+      // figure out which index we are of the nns
+      Collection<String> nnIds = DFSUtilClient.getNameNodeIds(conf, nsId);
+      String nnId = HAUtil.getNameNodeId(conf, nsId);
+      int nnIndex = 0;
+      for (String id : nnIds) {
+        if (id.equals(nnId)) {
+          break;
+        }
+        nnIndex++;
+      }
+      return new BlockTokenSecretManager(updateMin * 60 * 1000L,
+          lifetimeMin * 60 * 1000L, nnIndex, nnIds.size(), null, encryptionAlgorithm);
     } else {
       return new BlockTokenSecretManager(updateMin*60*1000L,
-          lifetimeMin*60*1000L, 0, null, encryptionAlgorithm);
+          lifetimeMin*60*1000L, 0, 1, null, encryptionAlgorithm);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java
index b1636bc..c30730b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointConf.java
@@ -44,7 +44,13 @@ public class CheckpointConf {
 
   /** The output dir for legacy OIV image */
   private final String legacyOivImageDir;
-  
+
+  /**
+  * multiplier on the checkpoint period to allow other nodes to do the checkpointing, when not the
+  * 'primary' checkpoint node
+  */
+  private double quietMultiplier;
+
   public CheckpointConf(Configuration conf) {
     checkpointCheckPeriod = conf.getLong(
         DFS_NAMENODE_CHECKPOINT_CHECK_PERIOD_KEY,
@@ -57,6 +63,8 @@ public class CheckpointConf {
     maxRetriesOnMergeError = conf.getInt(DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_KEY,
                                   DFS_NAMENODE_CHECKPOINT_MAX_RETRIES_DEFAULT);
     legacyOivImageDir = conf.get(DFS_NAMENODE_LEGACY_OIV_IMAGE_DIR_KEY);
+    quietMultiplier = conf.getDouble(DFS_NAMENODE_CHECKPOINT_QUIET_MULTIPLIER_KEY,
+      DFS_NAMENODE_CHECKPOINT_QUIET_MULTIPLIER_DEFAULT);
     warnForDeprecatedConfigs(conf);
   }
   
@@ -91,4 +99,8 @@ public class CheckpointConf {
   public String getLegacyOivImageDir() {
     return legacyOivImageDir;
   }
+
+  public double getQuietPeriod() {
+    return this.checkpointPeriod * this.quietMultiplier;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
index c565eb5..9dc20b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ImageServlet.java
@@ -30,6 +30,7 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.commons.logging.Log;
@@ -81,6 +82,9 @@ public class ImageServlet extends HttpServlet {
   private static final String LATEST_FSIMAGE_VALUE = "latest";
   private static final String IMAGE_FILE_TYPE = "imageFile";
 
+  private SortedSet<ImageUploadRequest> currentlyDownloadingCheckpoints = Collections
+      .<ImageUploadRequest> synchronizedSortedSet(new TreeSet<ImageUploadRequest>());
+
   @Override
   public void doGet(final HttpServletRequest request,
       final HttpServletResponse response) throws ServletException, IOException {
@@ -253,10 +257,12 @@ public class ImageServlet extends HttpServlet {
     }
 
     if (HAUtil.isHAEnabled(conf, DFSUtil.getNamenodeNameServiceId(conf))) {
-      Configuration otherNnConf = HAUtil.getConfForOtherNode(conf);
-      validRequestors.add(SecurityUtil.getServerPrincipal(otherNnConf
-          .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
-          NameNode.getAddress(otherNnConf).getHostName()));
+      List<Configuration> otherNnConfs = HAUtil.getConfForOtherNodes(conf);
+      for (Configuration otherNnConf : otherNnConfs) {
+        validRequestors.add(SecurityUtil.getServerPrincipal(otherNnConf
+                .get(DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY),
+            NameNode.getAddress(otherNnConf).getHostName()));
+      }
     }
 
     for (String v : validRequestors) {
@@ -420,7 +426,6 @@ public class ImageServlet extends HttpServlet {
   /**
    * Set the required parameters for uploading image
    * 
-   * @param httpMethod instance of method to set the parameters
    * @param storage colon separated storageInfo string
    * @param txid txid of the image
    * @param imageFileSize size of the imagefile to be uploaded
@@ -459,12 +464,37 @@ public class ImageServlet extends HttpServlet {
 
             @Override
             public Void run() throws Exception {
+              // if its not the active NN, then we need to notify the caller it was was the wrong
+              // target (regardless of the fact that we got the image)
+              HAServiceProtocol.HAServiceState state = NameNodeHttpServer
+                  .getNameNodeStateFromContext(getServletContext());
+              if (state != HAServiceProtocol.HAServiceState.ACTIVE) {
+                // we need a different response type here so the client can differentiate this
+                // from the failure to upload due to (1) security, or (2) other checkpoints already
+                // present
+                response.sendError(HttpServletResponse.SC_EXPECTATION_FAILED,
+                    "Nameode "+request.getLocalAddr()+" is currently not in a state which can "
+                        + "accept uploads of new fsimages. State: "+state);
+                return null;
+              }
 
               final long txid = parsedParams.getTxId();
+              String remoteAddr = request.getRemoteAddr();
+              ImageUploadRequest imageRequest = new ImageUploadRequest(txid, remoteAddr);
 
               final NameNodeFile nnf = parsedParams.getNameNodeFile();
 
-              if (!nnImage.addToCheckpointing(txid)) {
+              // if the node is attempting to upload an older transaction, we ignore it
+              SortedSet<ImageUploadRequest> larger = currentlyDownloadingCheckpoints.tailSet(imageRequest);
+              if (larger.size() > 0) {
+                response.sendError(HttpServletResponse.SC_CONFLICT,
+                    "Another checkpointer is already in the process of uploading a" +
+                        " checkpoint made up to transaction ID " + larger.last());
+                return null;
+              }
+
+              //make sure no one else has started uploading one
+              if (!currentlyDownloadingCheckpoints.add(imageRequest)) {
                 response.sendError(HttpServletResponse.SC_CONFLICT,
                     "Either current namenode is checkpointing or another"
                         + " checkpointer is already in the process of "
@@ -499,6 +529,10 @@ public class ImageServlet extends HttpServlet {
                   // remove some old ones.
                   nnImage.purgeOldStorage(nnf);
                 } finally {
+                  // remove the request once we've processed it, or it threw an error, so we
+                  // aren't using it either
+                  currentlyDownloadingCheckpoints.remove(imageRequest);
+
                   stream.close();
                 }
               } finally {
@@ -555,4 +589,46 @@ public class ImageServlet extends HttpServlet {
       return nnf;
     }
   }
+
+  private static class ImageUploadRequest implements Comparable<ImageUploadRequest> {
+
+    private final long txId;
+    private final String address;
+
+    public ImageUploadRequest(long txid, String remoteAddr) {
+      this.txId = txid;
+      this.address = remoteAddr;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+
+      ImageUploadRequest that = (ImageUploadRequest) o;
+
+      if (txId != that.txId) return false;
+      if (!address.equals(that.address)) return false;
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = (int) (txId ^ (txId >>> 32));
+      result = 31 * result + address.hashCode();
+      return result;
+    }
+
+    @Override public int compareTo(ImageUploadRequest other) {
+      return Long.compare(txId, other.txId);
+    }
+
+    @Override public String toString() {
+      return "ImageRequest{" +
+          "txId=" + txId +
+          ", address='" + address + '\'' +
+          '}';
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
index 09b6b80..6bd9868 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
@@ -27,6 +27,7 @@ import javax.servlet.ServletContext;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -272,4 +273,8 @@ public class NameNodeHttpServer {
       ServletContext context) {
     return (StartupProgress)context.getAttribute(STARTUP_PROGRESS_ATTRIBUTE_KEY);
   }
-}
+
+  public static HAServiceProtocol.HAServiceState getNameNodeStateFromContext(ServletContext context) {
+    return getNameNodeFromContext(context).getServiceState();
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
index 9783cca..afecf99 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
@@ -70,7 +70,33 @@ import org.mortbay.jetty.EofException;
  */
 @InterfaceAudience.Private
 public class TransferFsImage {
-  
+
+  public enum TransferResult{
+    SUCCESS(HttpServletResponse.SC_OK, false),
+    AUTHENTICATION_FAILURE(HttpServletResponse.SC_FORBIDDEN, true),
+    NOT_ACTIVE_NAMENODE_FAILURE(HttpServletResponse.SC_EXPECTATION_FAILED, false),
+    OLD_TRANSACTION_ID_FAILURE(HttpServletResponse.SC_CONFLICT, false),
+    UNEXPECTED_FAILURE(-1, true);
+
+    private final int response;
+    private final boolean shouldReThrowException;
+
+    private TransferResult(int response, boolean rethrow) {
+      this.response = response;
+      this.shouldReThrowException = rethrow;
+    }
+
+    public static TransferResult getResultForCode(int code){
+      TransferResult ret = UNEXPECTED_FAILURE;
+      for(TransferResult result:TransferResult.values()){
+        if(result.response == code){
+          return result;
+        }
+      }
+      return ret;
+    }
+  }
+
   public final static String CONTENT_LENGTH = "Content-Length";
   public final static String FILE_LENGTH = "File-Length";
   public final static String MD5_HEADER = "X-MD5-Digest";
@@ -198,9 +224,9 @@ public class TransferFsImage {
    * @param txid the transaction ID of the image to be uploaded
    * @throws IOException if there is an I/O error
    */
-  public static void uploadImageFromStorage(URL fsName, Configuration conf,
+  public static TransferResult uploadImageFromStorage(URL fsName, Configuration conf,
       NNStorage storage, NameNodeFile nnf, long txid) throws IOException {
-    uploadImageFromStorage(fsName, conf, storage, nnf, txid, null);
+    return uploadImageFromStorage(fsName, conf, storage, nnf, txid, null);
   }
 
   /**
@@ -215,7 +241,7 @@ public class TransferFsImage {
    * @param canceler optional canceler to check for abort of upload
    * @throws IOException if there is an I/O error or cancellation
    */
-  public static void uploadImageFromStorage(URL fsName, Configuration conf,
+  public static TransferResult uploadImageFromStorage(URL fsName, Configuration conf,
       NNStorage storage, NameNodeFile nnf, long txid, Canceler canceler)
       throws IOException {
     URL url = new URL(fsName, ImageServlet.PATH_SPEC);
@@ -223,21 +249,18 @@ public class TransferFsImage {
     try {
       uploadImage(url, conf, storage, nnf, txid, canceler);
     } catch (HttpPutFailedException e) {
-      if (e.getResponseCode() == HttpServletResponse.SC_CONFLICT) {
-        // this is OK - this means that a previous attempt to upload
-        // this checkpoint succeeded even though we thought it failed.
-        LOG.info("Image upload with txid " + txid + 
-            " conflicted with a previous image upload to the " +
-            "same NameNode. Continuing...", e);
-        return;
-      } else {
+      // translate the error code to a result, which is a bit more obvious in usage
+      TransferResult result = TransferResult.getResultForCode(e.getResponseCode());
+      if (result.shouldReThrowException) {
         throw e;
       }
+      return result;
     }
     double xferSec = Math.max(
         ((float) (Time.monotonicNow() - startTime)) / 1000.0, 0.001);
     LOG.info("Uploaded image with txid " + txid + " to namenode at " + fsName
         + " in " + xferSec + " seconds");
+    return TransferResult.SUCCESS;
   }
 
   /*

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
----------------------------------------------------------------------
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 88d9a6a..c22d7f1 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
@@ -23,8 +23,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIP
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
-import java.net.URL;
 import java.security.PrivilegedAction;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -77,10 +77,8 @@ public class BootstrapStandby implements Tool, Configurable {
   private static final Log LOG = LogFactory.getLog(BootstrapStandby.class);
   private String nsId;
   private String nnId;
-  private String otherNNId;
+  private List<RemoteNameNodeInfo> remoteNNs;
 
-  private URL otherHttpAddr;
-  private InetSocketAddress otherIpcAddr;
   private Collection<URI> dirsToFormat;
   private List<URI> editUrisToFormat;
   private List<URI> sharedEditsUris;
@@ -139,8 +137,8 @@ public class BootstrapStandby implements Tool, Configurable {
     System.err.println("Usage: " + this.getClass().getSimpleName() +
         " [-force] [-nonInteractive] [-skipSharedEditsCheck]");
   }
-  
-  private NamenodeProtocol createNNProtocolProxy()
+
+  private NamenodeProtocol createNNProtocolProxy(InetSocketAddress otherIpcAddr)
       throws IOException {
     return NameNodeProxies.createNonHAProxy(getConf(),
         otherIpcAddr, NamenodeProtocol.class,
@@ -149,18 +147,36 @@ public class BootstrapStandby implements Tool, Configurable {
   }
   
   private int doRun() throws IOException {
-    NamenodeProtocol proxy = createNNProtocolProxy();
-    NamespaceInfo nsInfo;
-    boolean isUpgradeFinalized;
-    try {
-      nsInfo = proxy.versionRequest();
-      isUpgradeFinalized = proxy.isUpgradeFinalized();
-    } catch (IOException ioe) {
-      LOG.fatal("Unable to fetch namespace information from active NN at " +
-          otherIpcAddr + ": " + ioe.getMessage());
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Full exception trace", ioe);
+    // find the active NN
+    NamenodeProtocol proxy = null;
+    NamespaceInfo nsInfo = null;
+    boolean isUpgradeFinalized = false;
+    RemoteNameNodeInfo proxyInfo = null;
+    for (int i = 0; i < remoteNNs.size(); i++) {
+      proxyInfo = remoteNNs.get(i);
+      InetSocketAddress otherIpcAddress = proxyInfo.getIpcAddress();
+      proxy = createNNProtocolProxy(otherIpcAddress);
+      try {
+        // Get the namespace from any active NN. If you just formatted the primary NN and are
+        // bootstrapping the other NNs from that layout, it will only contact the single NN.
+        // However, if there cluster is already running and you are adding a NN later (e.g.
+        // replacing a failed NN), then this will bootstrap from any node in the cluster.
+        nsInfo = proxy.versionRequest();
+        isUpgradeFinalized = proxy.isUpgradeFinalized();
+        break;
+      } catch (IOException ioe) {
+        LOG.warn("Unable to fetch namespace information from remote NN at " + otherIpcAddress
+            + ": " + ioe.getMessage());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Full exception trace", ioe);
+        }
       }
+    }
+
+    if (nsInfo == null) {
+      LOG.fatal(
+          "Unable to fetch namespace information from any remote NN. Possible NameNodes: "
+              + remoteNNs);
       return ERR_CODE_FAILED_CONNECT;
     }
 
@@ -175,9 +191,9 @@ public class BootstrapStandby implements Tool, Configurable {
         "=====================================================\n" +
         "About to bootstrap Standby ID " + nnId + " from:\n" +
         "           Nameservice ID: " + nsId + "\n" +
-        "        Other Namenode ID: " + otherNNId + "\n" +
-        "  Other NN's HTTP address: " + otherHttpAddr + "\n" +
-        "  Other NN's IPC  address: " + otherIpcAddr + "\n" +
+        "        Other Namenode ID: " + proxyInfo.getNameNodeID() + "\n" +
+        "  Other NN's HTTP address: " + proxyInfo.getHttpAddress() + "\n" +
+        "  Other NN's IPC  address: " + proxyInfo.getIpcAddress() + "\n" +
         "             Namespace ID: " + nsInfo.getNamespaceID() + "\n" +
         "            Block pool ID: " + nsInfo.getBlockPoolID() + "\n" +
         "               Cluster ID: " + nsInfo.getClusterID() + "\n" +
@@ -201,7 +217,7 @@ public class BootstrapStandby implements Tool, Configurable {
     }
 
     // download the fsimage from active namenode
-    int download = downloadImage(storage, proxy);
+    int download = downloadImage(storage, proxy, proxyInfo);
     if (download != 0) {
       return download;
     }
@@ -292,7 +308,7 @@ public class BootstrapStandby implements Tool, Configurable {
     }
   }
 
-  private int downloadImage(NNStorage storage, NamenodeProtocol proxy)
+  private int downloadImage(NNStorage storage, NamenodeProtocol proxy, RemoteNameNodeInfo proxyInfo)
       throws IOException {
     // Load the newly formatted image, using all of the directories
     // (including shared edits)
@@ -316,7 +332,7 @@ public class BootstrapStandby implements Tool, Configurable {
 
       // Download that checkpoint into our storage directories.
       MD5Hash hash = TransferFsImage.downloadImageToStorage(
-          otherHttpAddr, imageTxId, storage, true);
+        proxyInfo.getHttpAddress(), imageTxId, storage, true);
       image.saveDigestAndRenameCheckpointImage(NameNodeFile.IMAGE, imageTxId,
           hash);
     } catch (IOException ioe) {
@@ -385,18 +401,26 @@ public class BootstrapStandby implements Tool, Configurable {
       throw new HadoopIllegalArgumentException(
         "Shared edits storage is not enabled for this namenode.");
     }
-    
-    Configuration otherNode = HAUtil.getConfForOtherNode(conf);
-    otherNNId = HAUtil.getNameNodeId(otherNode, nsId);
-    otherIpcAddr = NameNode.getServiceAddress(otherNode, true);
-    Preconditions.checkArgument(otherIpcAddr.getPort() != 0 &&
-        !otherIpcAddr.getAddress().isAnyLocalAddress(),
-        "Could not determine valid IPC address for other NameNode (%s)" +
-        ", got: %s", otherNNId, otherIpcAddr);
-
-    final String scheme = DFSUtil.getHttpClientScheme(conf);
-    otherHttpAddr = DFSUtil.getInfoServerWithDefaultHost(
-        otherIpcAddr.getHostName(), otherNode, scheme).toURL();
+
+
+    remoteNNs = RemoteNameNodeInfo.getRemoteNameNodes(conf, nsId);
+    // validate the configured NNs
+    List<RemoteNameNodeInfo> remove = new ArrayList<RemoteNameNodeInfo>(remoteNNs.size());
+    for (RemoteNameNodeInfo info : remoteNNs) {
+      InetSocketAddress address = info.getIpcAddress();
+      LOG.info("Found nn: " + info.getNameNodeID() + ", ipc: " + info.getIpcAddress());
+      if (address.getPort() == 0 || address.getAddress().isAnyLocalAddress()) {
+        LOG.error("Could not determine valid IPC address for other NameNode ("
+            + info.getNameNodeID() + ") , got: " + address);
+        remove.add(info);
+      }
+    }
+
+    // remove any invalid nns
+    remoteNNs.removeAll(remove);
+
+    // make sure we have at least one left to read
+    Preconditions.checkArgument(!remoteNNs.isEmpty(), "Could not find any valid namenodes!");
 
     dirsToFormat = FSNamesystem.getNamespaceDirs(conf);
     editUrisToFormat = FSNamesystem.getNamespaceEditsDirs(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
index 38aa358..cfca77c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
@@ -23,7 +23,13 @@ import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.Callable;
 
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -67,10 +73,10 @@ public class EditLogTailer {
   
   private final Configuration conf;
   private final FSNamesystem namesystem;
+  private final Iterator<RemoteNameNodeInfo> nnLookup;
   private FSEditLog editLog;
 
-  private InetSocketAddress activeAddr;
-  private NamenodeProtocol cachedActiveProxy = null;
+  private RemoteNameNodeInfo currentNN;
 
   /**
    * The last transaction ID at which an edit log roll was initiated.
@@ -100,7 +106,17 @@ public class EditLogTailer {
    * available to be read from.
    */
   private final long sleepTimeMs;
-  
+
+  private final int nnCount;
+  private NamenodeProtocol cachedActiveProxy = null;
+  // count of the number of NNs we have attempted in the current lookup loop
+  private int nnLoopCount = 0;
+
+  /**
+   * maximum number of retries we should give each of the remote namenodes before giving up
+   */
+  private int maxRetries;
+
   public EditLogTailer(FSNamesystem namesystem, Configuration conf) {
     this.tailerThread = new EditLogTailerThread();
     this.conf = conf;
@@ -111,12 +127,24 @@ public class EditLogTailer {
 
     logRollPeriodMs = conf.getInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY,
         DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_DEFAULT) * 1000;
+    List<RemoteNameNodeInfo> nns = Collections.emptyList();
     if (logRollPeriodMs >= 0) {
-      this.activeAddr = getActiveNodeAddress();
-      Preconditions.checkArgument(activeAddr.getPort() > 0,
-          "Active NameNode must have an IPC port configured. " +
-          "Got address '%s'", activeAddr);
-      LOG.info("Will roll logs on active node at " + activeAddr + " every " +
+      try {
+        nns = RemoteNameNodeInfo.getRemoteNameNodes(conf);
+      } catch (IOException e) {
+        throw new IllegalArgumentException("Remote NameNodes not correctly configured!", e);
+      }
+
+      for (RemoteNameNodeInfo info : nns) {
+        // overwrite the socket address, if we need to
+        InetSocketAddress ipc = NameNode.getServiceAddress(info.getConfiguration(), true);
+        // sanity check the ipc address
+        Preconditions.checkArgument(ipc.getPort() > 0,
+            "Active NameNode must have an IPC port configured. " + "Got address '%s'", ipc);
+        info.setIpcAddress(ipc);
+      }
+
+      LOG.info("Will roll logs on active node every " +
           (logRollPeriodMs / 1000) + " seconds.");
     } else {
       LOG.info("Not going to trigger log rolls on active node because " +
@@ -125,29 +153,24 @@ public class EditLogTailer {
     
     sleepTimeMs = conf.getInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY,
         DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_DEFAULT) * 1000;
-    
+
+    maxRetries = conf.getInt(DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_KEY,
+      DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_DEFAULT);
+    if (maxRetries <= 0) {
+      LOG.error("Specified a non-positive number of retries for the number of retries for the " +
+          "namenode connection when manipulating the edit log (" +
+          DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_KEY + "), setting to default: " +
+          DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_DEFAULT);
+      maxRetries = DFSConfigKeys.DFS_HA_TAILEDITS_ALL_NAMESNODES_RETRY_DEFAULT;
+    }
+
+    nnCount = nns.size();
+    // setup the iterator to endlessly loop the nns
+    this.nnLookup = Iterators.cycle(nns);
+
     LOG.debug("logRollPeriodMs=" + logRollPeriodMs +
         " sleepTime=" + sleepTimeMs);
   }
-  
-  private InetSocketAddress getActiveNodeAddress() {
-    Configuration activeConf = HAUtil.getConfForOtherNode(conf);
-    return NameNode.getServiceAddress(activeConf, true);
-  }
-  
-  private NamenodeProtocol getActiveNodeProxy() throws IOException {
-    if (cachedActiveProxy == null) {
-      int rpcTimeout = conf.getInt(
-          DFSConfigKeys.DFS_HA_LOGROLL_RPC_TIMEOUT_KEY,
-          DFSConfigKeys.DFS_HA_LOGROLL_RPC_TIMEOUT_DEFAULT);
-      NamenodeProtocolPB proxy = RPC.waitForProxy(NamenodeProtocolPB.class,
-          RPC.getProtocolVersion(NamenodeProtocolPB.class), activeAddr, conf,
-          rpcTimeout, Long.MAX_VALUE);
-      cachedActiveProxy = new NamenodeProtocolTranslatorPB(proxy);
-    }
-    assert cachedActiveProxy != null;
-    return cachedActiveProxy;
-  }
 
   public void start() {
     tailerThread.start();
@@ -270,9 +293,15 @@ public class EditLogTailer {
    * Trigger the active node to roll its logs.
    */
   private void triggerActiveLogRoll() {
-    LOG.info("Triggering log roll on remote NameNode " + activeAddr);
+    LOG.info("Triggering log roll on remote NameNode");
     try {
-      getActiveNodeProxy().rollEditLog();
+      new MultipleNameNodeProxy<Void>() {
+        @Override
+        protected Void doWork() throws IOException {
+          cachedActiveProxy.rollEditLog();
+          return null;
+        }
+      }.call();
       lastRollTriggerTxId = lastLoadedTxnId;
     } catch (IOException ioe) {
       if (ioe instanceof RemoteException) {
@@ -362,5 +391,76 @@ public class EditLogTailer {
       }
     }
   }
+  /**
+   * Manage the 'active namenode proxy'. This cannot just be the a single proxy since we could
+   * failover across a number of NameNodes, rather than just between an active and a standby.
+   * <p>
+   * We - lazily - get a proxy to one of the configured namenodes and attempt to make the request
+   * against it. If it doesn't succeed, either because the proxy failed to be created or the request
+   * failed, we try the next NN in the list. We try this up to the configuration maximum number of
+   * retries before throwing up our hands. A working proxy is retained across attempts since we
+   * expect the active NameNode to switch rarely.
+   * <p>
+   * This mechanism is <b>very bad</b> for cases where we care about being <i>fast</i>; it just
+   * blindly goes and tries namenodes.
+   */
+  private abstract class MultipleNameNodeProxy<T> implements Callable<T> {
+
+    /**
+     * Do the actual work to the remote namenode via the {@link #cachedActiveProxy}.
+     * @return the result of the work, if there is one
+     * @throws IOException if the actions done to the proxy throw an exception.
+     */
+    protected abstract T doWork() throws IOException;
+
+    public T call() throws IOException {
+      while ((cachedActiveProxy = getActiveNodeProxy()) != null) {
+        try {
+          T ret = doWork();
+          // reset the loop count on success
+          nnLoopCount = 0;
+          return ret;
+        } catch (RemoteException e) {
+          Throwable cause = e.unwrapRemoteException(StandbyException.class);
+          // if its not a standby exception, then we need to re-throw it, something bad has happened
+          if (cause == e) {
+            throw e;
+          } else {
+            // it is a standby exception, so we try the other NN
+            LOG.warn("Failed to reach remote node: " + currentNN
+                + ", retrying with remaining remote NNs");
+            cachedActiveProxy = null;
+            // this NN isn't responding to requests, try the next one
+            nnLoopCount++;
+          }
+        }
+      }
+      throw new IOException("Cannot find any valid remote NN to service request!");
+    }
 
-}
+    private NamenodeProtocol getActiveNodeProxy() throws IOException {
+      if (cachedActiveProxy == null) {
+        while (true) {
+          // if we have reached the max loop count, quit by returning null
+          if ((nnLoopCount / nnCount) >= maxRetries) {
+            return null;
+          }
+
+          currentNN = nnLookup.next();
+          try {
+            NamenodeProtocolPB proxy = RPC.waitForProxy(NamenodeProtocolPB.class,
+                RPC.getProtocolVersion(NamenodeProtocolPB.class), currentNN.getIpcAddress(), conf);
+            cachedActiveProxy = new NamenodeProtocolTranslatorPB(proxy);
+            break;
+          } catch (IOException e) {
+            LOG.info("Failed to reach " + currentNN, e);
+            // couldn't even reach this NN, try the next one
+            nnLoopCount++;
+          }
+        }
+      }
+      assert cachedActiveProxy != null;
+      return cachedActiveProxy;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
new file mode 100644
index 0000000..9a51190
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java
@@ -0,0 +1,133 @@
+/**
+ * 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.ha;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Objects;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.HAUtil;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Information about a single remote NameNode
+ */
+public class RemoteNameNodeInfo {
+
+  public static List<RemoteNameNodeInfo> getRemoteNameNodes(Configuration conf) throws IOException {
+    String nsId = DFSUtil.getNamenodeNameServiceId(conf);
+    return getRemoteNameNodes(conf, nsId);
+  }
+
+  public static List<RemoteNameNodeInfo> getRemoteNameNodes(Configuration conf, String nsId)
+      throws IOException {
+    // there is only a single NN configured (and no federation) so we don't have any more NNs
+    if (nsId == null) {
+      return Collections.emptyList();
+    }
+    List<Configuration> otherNodes = HAUtil.getConfForOtherNodes(conf);
+    List<RemoteNameNodeInfo> nns = new ArrayList<RemoteNameNodeInfo>();
+
+    for (Configuration otherNode : otherNodes) {
+      String otherNNId = HAUtil.getNameNodeId(otherNode, nsId);
+      // don't do any validation here as in some cases, it can be overwritten later
+      InetSocketAddress otherIpcAddr = NameNode.getServiceAddress(otherNode, true);
+
+
+      final String scheme = DFSUtil.getHttpClientScheme(conf);
+      URL otherHttpAddr = DFSUtil.getInfoServerWithDefaultHost(otherIpcAddr.getHostName(),
+          otherNode, scheme).toURL();
+
+      nns.add(new RemoteNameNodeInfo(otherNode, otherNNId, otherIpcAddr, otherHttpAddr));
+    }
+    return nns;
+  }
+
+  private final Configuration conf;
+  private final String nnId;
+  private InetSocketAddress ipcAddress;
+  private final URL httpAddress;
+
+  private RemoteNameNodeInfo(Configuration conf, String nnId, InetSocketAddress ipcAddress,
+      URL httpAddress) {
+    this.conf = conf;
+    this.nnId = nnId;
+    this.ipcAddress = ipcAddress;
+    this.httpAddress = httpAddress;
+  }
+
+  public InetSocketAddress getIpcAddress() {
+    return this.ipcAddress;
+  }
+
+  public String getNameNodeID() {
+    return this.nnId;
+  }
+
+  public URL getHttpAddress() {
+    return this.httpAddress;
+  }
+
+  public Configuration getConfiguration() {
+    return this.conf;
+  }
+
+  public void setIpcAddress(InetSocketAddress ipc) {
+    this.ipcAddress = ipc;
+  }
+
+  @Override
+  public String toString() {
+    return "RemoteNameNodeInfo [nnId=" + nnId + ", ipcAddress=" + ipcAddress
+        + ", httpAddress=" + httpAddress + "]";
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    RemoteNameNodeInfo that = (RemoteNameNodeInfo) o;
+
+    if (!nnId.equals(that.nnId)) return false;
+    if (!ipcAddress.equals(that.ipcAddress)) return false;
+    // convert to the standard strings since URL.equals does address resolution, which is a
+    // blocking call and a a FindBugs issue.
+    String httpString = httpAddress.toString();
+    String thatHttpString  = that.httpAddress.toString();
+    return httpString.equals(thatHttpString);
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = nnId.hashCode();
+    result = 31 * result + ipcAddress.hashCode();
+    // toString rather than hashCode b/c Url.hashCode is a blocking call.
+    result = 31 * result + httpAddress.toString().hashCode();
+    return result;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
index 1e40368..f5ecbec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
@@ -23,12 +23,10 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URL;
 import java.security.PrivilegedAction;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadFactory;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -45,6 +43,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SaveNamespaceCancelledException;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage;
 import org.apache.hadoop.hdfs.util.Canceler;
+import org.apache.hadoop.io.MultipleIOException;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -68,12 +67,13 @@ public class StandbyCheckpointer {
   private long lastCheckpointTime;
   private final CheckpointerThread thread;
   private final ThreadFactory uploadThreadFactory;
-  private URL activeNNAddress;
+  private List<URL> activeNNAddresses;
   private URL myNNAddress;
 
   private final Object cancelLock = new Object();
   private Canceler canceler;
-  
+  private boolean isPrimaryCheckPointer = true;
+
   // Keep track of how many checkpoints were canceled.
   // This is for use in tests.
   private static int canceledCount = 0;
@@ -100,14 +100,21 @@ public class StandbyCheckpointer {
     myNNAddress = getHttpAddress(conf);
 
     // Look up the active node's address
-    Configuration confForActive = HAUtil.getConfForOtherNode(conf);
-    activeNNAddress = getHttpAddress(confForActive);
-    
+    List<Configuration> confForActive = HAUtil.getConfForOtherNodes(conf);
+    activeNNAddresses = new ArrayList<URL>(confForActive.size());
+    for (Configuration activeConf : confForActive) {
+      URL activeNNAddress = getHttpAddress(activeConf);
+
+      // sanity check each possible active NN
+      Preconditions.checkArgument(checkAddress(activeNNAddress),
+          "Bad address for active NN: %s", activeNNAddress);
+
+      activeNNAddresses.add(activeNNAddress);
+    }
+
     // Sanity-check.
-    Preconditions.checkArgument(checkAddress(activeNNAddress),
-        "Bad address for active NN: %s", activeNNAddress);
-    Preconditions.checkArgument(checkAddress(myNNAddress),
-        "Bad address for standby NN: %s", myNNAddress);
+    Preconditions.checkArgument(checkAddress(myNNAddress), "Bad address for standby NN: %s",
+        myNNAddress);
   }
   
   private URL getHttpAddress(Configuration conf) throws IOException {
@@ -127,7 +134,7 @@ public class StandbyCheckpointer {
 
   public void start() {
     LOG.info("Starting standby checkpoint thread...\n" +
-        "Checkpointing active NN at " + activeNNAddress + "\n" +
+        "Checkpointing active NN to possible NNs: " + activeNNAddresses + "\n" +
         "Serving checkpoints at " + myNNAddress);
     thread.start();
   }
@@ -148,11 +155,10 @@ public class StandbyCheckpointer {
     thread.interrupt();
   }
 
-  private void doCheckpoint() throws InterruptedException, IOException {
+  private void doCheckpoint(boolean sendCheckpoint) throws InterruptedException, IOException {
     assert canceler != null;
     final long txid;
     final NameNodeFile imageType;
-    
     // Acquire cpLock to make sure no one is modifying the name system.
     // It does not need the full namesystem write lock, since the only thing
     // that modifies namesystem on standby node is edit log replaying.
@@ -161,9 +167,9 @@ public class StandbyCheckpointer {
       assert namesystem.getEditLog().isOpenForRead() :
         "Standby Checkpointer should only attempt a checkpoint when " +
         "NN is in standby mode, but the edit logs are in an unexpected state";
-      
+
       FSImage img = namesystem.getFSImage();
-      
+
       long prevCheckpointTxId = img.getStorage().getMostRecentCheckpointTxId();
       long thisCheckpointTxId = img.getLastAppliedOrWrittenTxId();
       assert thisCheckpointTxId >= prevCheckpointTxId;
@@ -185,7 +191,7 @@ public class StandbyCheckpointer {
       img.saveNamespace(namesystem, imageType, canceler);
       txid = img.getStorage().getMostRecentCheckpointTxId();
       assert txid == thisCheckpointTxId : "expected to save checkpoint at txid=" +
-        thisCheckpointTxId + " but instead saved at txid=" + txid;
+          thisCheckpointTxId + " but instead saved at txid=" + txid;
 
       // Save the legacy OIV image, if the output dir is defined.
       String outputDir = checkpointConf.getLegacyOivImageDir();
@@ -195,31 +201,85 @@ public class StandbyCheckpointer {
     } finally {
       namesystem.cpUnlock();
     }
-    
+
+    //early exit if we shouldn't actually send the checkpoint to the ANN
+    if(!sendCheckpoint){
+      return;
+    }
+
     // Upload the saved checkpoint back to the active
-    // Do this in a separate thread to avoid blocking transition to active
+    // Do this in a separate thread to avoid blocking transition to active, but don't allow more
+    // than the expected number of tasks to run or queue up
     // See HDFS-4816
-    ExecutorService executor =
-        Executors.newSingleThreadExecutor(uploadThreadFactory);
-    Future<Void> upload = executor.submit(new Callable<Void>() {
-      @Override
-      public Void call() throws IOException {
-        TransferFsImage.uploadImageFromStorage(activeNNAddress, conf,
-            namesystem.getFSImage().getStorage(), imageType, txid, canceler);
-        return null;
+    ExecutorService executor = new ThreadPoolExecutor(0, activeNNAddresses.size(), 100,
+        TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(activeNNAddresses.size()),
+        uploadThreadFactory);
+    // for right now, just match the upload to the nn address by convention. There is no need to
+    // directly tie them together by adding a pair class.
+    List<Future<TransferFsImage.TransferResult>> uploads =
+        new ArrayList<Future<TransferFsImage.TransferResult>>();
+    for (final URL activeNNAddress : activeNNAddresses) {
+      Future<TransferFsImage.TransferResult> upload =
+          executor.submit(new Callable<TransferFsImage.TransferResult>() {
+            @Override
+            public TransferFsImage.TransferResult call() throws IOException {
+              return TransferFsImage.uploadImageFromStorage(activeNNAddress, conf, namesystem
+                  .getFSImage().getStorage(), imageType, txid, canceler);
+            }
+          });
+      uploads.add(upload);
+    }
+    InterruptedException ie = null;
+    IOException ioe= null;
+    int i = 0;
+    boolean success = false;
+    for (; i < uploads.size(); i++) {
+      Future<TransferFsImage.TransferResult> upload = uploads.get(i);
+      try {
+        // TODO should there be some smarts here about retries nodes that are not the active NN?
+        if (upload.get() == TransferFsImage.TransferResult.SUCCESS) {
+          success = true;
+          //avoid getting the rest of the results - we don't care since we had a successful upload
+          break;
+        }
+
+      } catch (ExecutionException e) {
+        ioe = new IOException("Exception during image upload: " + e.getMessage(),
+            e.getCause());
+        break;
+      } catch (InterruptedException e) {
+        ie = e;
+        break;
+      }
+    }
+
+    // we are primary if we successfully updated the ANN
+    this.isPrimaryCheckPointer = success;
+
+    // cleaner than copying code for multiple catch statements and better than catching all
+    // exceptions, so we just handle the ones we expect.
+    if (ie != null || ioe != null) {
+
+      // cancel the rest of the tasks, and close the pool
+      for (; i < uploads.size(); i++) {
+        Future<TransferFsImage.TransferResult> upload = uploads.get(i);
+        // The background thread may be blocked waiting in the throttler, so
+        // interrupt it.
+        upload.cancel(true);
+      }
+
+      // shutdown so we interrupt anything running and don't start anything new
+      executor.shutdownNow();
+      // this is a good bit longer than the thread timeout, just to make sure all the threads
+      // that are not doing any work also stop
+      executor.awaitTermination(500, TimeUnit.MILLISECONDS);
+
+      // re-throw the exception we got, since one of these two must be non-null
+      if (ie != null) {
+        throw ie;
+      } else if (ioe != null) {
+        throw ioe;
       }
-    });
-    executor.shutdown();
-    try {
-      upload.get();
-    } catch (InterruptedException e) {
-      // The background thread may be blocked waiting in the throttler, so
-      // interrupt it.
-      upload.cancel(true);
-      throw e;
-    } catch (ExecutionException e) {
-      throw new IOException("Exception during image upload: " + e.getMessage(),
-          e.getCause());
     }
   }
   
@@ -322,8 +382,10 @@ public class StandbyCheckpointer {
           final long now = monotonicNow();
           final long uncheckpointed = countUncheckpointedTxns();
           final long secsSinceLast = (now - lastCheckpointTime) / 1000;
-          
+
+          // if we need a rollback checkpoint, always attempt to checkpoint
           boolean needCheckpoint = needRollbackCheckpoint;
+
           if (needCheckpoint) {
             LOG.info("Triggering a rollback fsimage for rolling upgrade.");
           } else if (uncheckpointed >= checkpointConf.getTxnCount()) {
@@ -338,19 +400,23 @@ public class StandbyCheckpointer {
                 "exceeds the configured interval " + checkpointConf.getPeriod());
             needCheckpoint = true;
           }
-          
-          synchronized (cancelLock) {
-            if (now < preventCheckpointsUntil) {
-              LOG.info("But skipping this checkpoint since we are about to failover!");
-              canceledCount++;
-              continue;
-            }
-            assert canceler == null;
-            canceler = new Canceler();
-          }
-          
+
           if (needCheckpoint) {
-            doCheckpoint();
+            synchronized (cancelLock) {
+              if (now < preventCheckpointsUntil) {
+                LOG.info("But skipping this checkpoint since we are about to failover!");
+                canceledCount++;
+                continue;
+              }
+              assert canceler == null;
+              canceler = new Canceler();
+            }
+
+            // on all nodes, we build the checkpoint. However, we only ship the checkpoint if have a
+            // rollback request, are the checkpointer, are outside the quiet period.
+            boolean sendRequest = isPrimaryCheckPointer || secsSinceLast >= checkpointConf.getQuietPeriod();
+            doCheckpoint(sendRequest);
+
             // reset needRollbackCheckpoint to false only when we finish a ckpt
             // for rollback image
             if (needRollbackCheckpoint
@@ -379,7 +445,7 @@ public class StandbyCheckpointer {
   }
 
   @VisibleForTesting
-  URL getActiveNNAddress() {
-    return activeNNAddress;
+  List<URL> getActiveNNAddresses() {
+    return activeNNAddresses;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
index f125a27..24e5bef 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSZKFailoverController.java
@@ -25,6 +25,8 @@ import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
 import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.logging.Log;
@@ -261,4 +263,15 @@ public class DFSZKFailoverController extends ZKFailoverController {
     return isThreadDumpCaptured;
   }
 
+  @Override
+  public List<HAServiceTarget> getAllOtherNodes() {
+    String nsId = DFSUtil.getNamenodeNameServiceId(conf);
+    List<String> otherNn = HAUtil.getNameNodeIdOfOtherNodes(conf, nsId);
+
+    List<HAServiceTarget> targets = new ArrayList<HAServiceTarget>(otherNn.size());
+    for (String nnId : otherNn) {
+      targets.add(new NNHAServiceTarget(conf, nsId, nnId));
+    }
+    return targets;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index aaa1c2f..76161a5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -914,6 +914,18 @@
 </property>
 
 <property>
+  <name>dfs.namenode.checkpoint.check.quiet-multiplier</name>
+  <value>1.5</value>
+  <description>
+    Used to calculate the amount of time between retries when in the 'quiet' period
+    for creating checkpoints (active namenode already has an up-to-date image from another
+    checkpointer), so we wait a multiplier of the dfs.namenode.checkpoint.check.period before
+    retrying the checkpoint because another node likely is already managing the checkpoints,
+    allowing us to save bandwidth to transfer checkpoints that don't need to be used.
+  </description>
+</property>
+
+<property>
   <name>dfs.namenode.num.checkpoints.retained</name>
   <value>2</value>
   <description>The number of image checkpoint files (fsimage_*) that will be retained by
@@ -1288,6 +1300,14 @@
 </property>
 
 <property>
+  <name>dfs.ha.tail-edits.namenode-retries</name>
+  <value>3</value>
+  <description>
+    Number of retries to use when contacting the namenode when tailing the log.
+  </description>
+</property>
+
+<property>
   <name>dfs.ha.automatic-failover.enabled</name>
   <value>false</value>
   <description>


[27/30] hadoop git commit: HDFS-8656. Preserve compatibility of ClientProtocol#rollingUpgrade after finalization.

Posted by aw...@apache.org.
HDFS-8656. Preserve compatibility of ClientProtocol#rollingUpgrade after finalization.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/60b858bf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/60b858bf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/60b858bf

Branch: refs/heads/HADOOP-12111
Commit: 60b858bfa65e0feb665e1a84784a3d45e9091c66
Parents: de480d6
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 26 11:30:59 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Fri Jun 26 11:30:59 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/ClientProtocol.java    |  3 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/namenode/FSNamesystem.java      |  8 ++-
 .../hdfs/server/namenode/NameNodeMXBean.java    |  5 +-
 .../apache/hadoop/hdfs/TestRollingUpgrade.java  | 63 ++++++++++++++++----
 .../server/namenode/TestNameNodeMXBean.java     |  5 +-
 6 files changed, 68 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 7e27078..ab41911 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -853,7 +853,8 @@ public interface ClientProtocol {
   /**
    * Rolling upgrade operations.
    * @param action either query, prepare or finalize.
-   * @return rolling upgrade information.
+   * @return rolling upgrade information. On query, if no upgrade is in
+   * progress, returns null.
    */
   @Idempotent
   public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index bb1b3ff..e287ea4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1110,6 +1110,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8626. Reserved RBW space is not released if creation of RBW File
     fails. (kanaka kumar avvaru via Arpit Agarwal)
 
+    HDFS08656. Preserve compatibility of ClientProtocol#rollingUpgrade after
+    finalization. (wang)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d0f4e08..e95007b 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
@@ -6757,10 +6757,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     readLock();
     try {
-      if (rollingUpgradeInfo != null) {
-        boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
-        rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage);
+      if (!isRollingUpgrade()) {
+        return null;
       }
+      Preconditions.checkNotNull(rollingUpgradeInfo);
+      boolean hasRollbackImage = this.getFSImage().hasRollbackFSImage();
+      rollingUpgradeInfo.setCreatedRollbackImages(hasRollbackImage);
       return rollingUpgradeInfo;
     } finally {
       readUnlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
index 3c317fd..0e4d445 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java
@@ -81,9 +81,10 @@ public interface NameNodeMXBean {
   public boolean isUpgradeFinalized();
 
   /**
-   * Gets the RollingUpgrade information
+   * Gets the RollingUpgrade information.
    *
-   * @return Rolling upgrade information
+   * @return Rolling upgrade information if an upgrade is in progress. Else
+   * (e.g. if there is no upgrade or the upgrade is finalized), returns null.
    */
   public RollingUpgradeInfo.Bean getRollingUpgradeStatus();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
index b50b1cb..72e16e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
@@ -19,8 +19,18 @@ package org.apache.hadoop.hdfs;
 
 import java.io.File;
 import java.io.IOException;
+import java.lang.management.ManagementFactory;
 import java.util.concurrent.ThreadLocalRandom;
 
+import javax.management.AttributeNotFoundException;
+import javax.management.InstanceNotFoundException;
+import javax.management.MBeanException;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+import javax.management.openmbean.CompositeDataSupport;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -45,7 +55,9 @@ import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
-import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNull;
 
 /**
  * This class tests rolling upgrade.
@@ -56,7 +68,7 @@ public class TestRollingUpgrade {
   public static void runCmd(DFSAdmin dfsadmin, boolean success,
       String... args) throws  Exception {
     if (success) {
-      Assert.assertEquals(0, dfsadmin.run(args));
+      assertEquals(0, dfsadmin.run(args));
     } else {
       Assert.assertTrue(dfsadmin.run(args) != 0);
     }
@@ -86,6 +98,7 @@ public class TestRollingUpgrade {
         //illegal argument "abc" to rollingUpgrade option
         runCmd(dfsadmin, false, "-rollingUpgrade", "abc");
 
+        checkMxBeanIsNull();
         //query rolling upgrade
         runCmd(dfsadmin, true, "-rollingUpgrade");
 
@@ -96,11 +109,16 @@ public class TestRollingUpgrade {
 
         //query rolling upgrade
         runCmd(dfsadmin, true, "-rollingUpgrade", "query");
+        checkMxBean();
 
         dfs.mkdirs(bar);
 
         //finalize rolling upgrade
         runCmd(dfsadmin, true, "-rollingUpgrade", "finalize");
+        // RollingUpgradeInfo should be null after finalization, both via
+        // Java API and in JMX
+        assertNull(dfs.rollingUpgrade(RollingUpgradeAction.QUERY));
+        checkMxBeanIsNull();
 
         dfs.mkdirs(baz);
 
@@ -197,7 +215,7 @@ public class TestRollingUpgrade {
         LOG.info("START\n" + info1);
 
         //query rolling upgrade
-        Assert.assertEquals(info1, dfs.rollingUpgrade(RollingUpgradeAction.QUERY));
+        assertEquals(info1, dfs.rollingUpgrade(RollingUpgradeAction.QUERY));
 
         dfs.mkdirs(bar);
         cluster.shutdown();
@@ -218,13 +236,13 @@ public class TestRollingUpgrade {
       Assert.assertFalse(dfs2.exists(baz));
 
       //query rolling upgrade in cluster2
-      Assert.assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
+      assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
 
       dfs2.mkdirs(baz);
 
       LOG.info("RESTART cluster 2");
       cluster2.restartNameNode();
-      Assert.assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
+      assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
       Assert.assertTrue(dfs2.exists(foo));
       Assert.assertTrue(dfs2.exists(bar));
       Assert.assertTrue(dfs2.exists(baz));
@@ -238,7 +256,7 @@ public class TestRollingUpgrade {
 
       LOG.info("RESTART cluster 2 again");
       cluster2.restartNameNode();
-      Assert.assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
+      assertEquals(info1, dfs2.rollingUpgrade(RollingUpgradeAction.QUERY));
       Assert.assertTrue(dfs2.exists(foo));
       Assert.assertTrue(dfs2.exists(bar));
       Assert.assertTrue(dfs2.exists(baz));
@@ -259,8 +277,30 @@ public class TestRollingUpgrade {
     }
   }
 
+  private static CompositeDataSupport getBean()
+      throws MalformedObjectNameException, MBeanException,
+      AttributeNotFoundException, InstanceNotFoundException,
+      ReflectionException {
+    MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+    ObjectName mxbeanName =
+        new ObjectName("Hadoop:service=NameNode,name=NameNodeInfo");
+    return (CompositeDataSupport)mbs.getAttribute(mxbeanName,
+        "RollingUpgradeStatus");
+  }
+
+  private static void checkMxBeanIsNull() throws Exception {
+    CompositeDataSupport ruBean = getBean();
+    assertNull(ruBean);
+  }
+
+  private static void checkMxBean() throws Exception {
+    CompositeDataSupport ruBean = getBean();
+    assertNotEquals(0l, ruBean.get("startTime"));
+    assertEquals(0l, ruBean.get("finalizeTime"));
+  }
+
   @Test
-  public void testRollback() throws IOException {
+  public void testRollback() throws Exception {
     // start a cluster
     final Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
@@ -279,10 +319,13 @@ public class TestRollingUpgrade {
       out.write(data, 0, data.length);
       out.close();
 
+      checkMxBeanIsNull();
       startRollingUpgrade(foo, bar, file, data, cluster);
+      checkMxBean();
       cluster.getFileSystem().rollEdits();
       cluster.getFileSystem().rollEdits();
       rollbackRollingUpgrade(foo, bar, file, data, cluster);
+      checkMxBeanIsNull();
 
       startRollingUpgrade(foo, bar, file, data, cluster);
       cluster.getFileSystem().rollEdits();
@@ -356,18 +399,18 @@ public class TestRollingUpgrade {
       // check the datanode
       final String dnAddr = dn.getDatanodeId().getIpcAddr(false);
       final String[] args1 = {"-getDatanodeInfo", dnAddr};
-      Assert.assertEquals(0, dfsadmin.run(args1));
+      runCmd(dfsadmin, true, args1);
 
       // issue shutdown to the datanode.
       final String[] args2 = {"-shutdownDatanode", dnAddr, "upgrade" };
-      Assert.assertEquals(0, dfsadmin.run(args2));
+      runCmd(dfsadmin, true, args2);
 
       // the datanode should be down.
       Thread.sleep(2000);
       Assert.assertFalse("DataNode should exit", dn.isDatanodeUp());
 
       // ping should fail.
-      Assert.assertEquals(-1, dfsadmin.run(args1));
+      assertEquals(-1, dfsadmin.run(args1));
     } finally {
       if (cluster != null) cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/60b858bf/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
index 681e8a4..2087c44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeMXBean.java
@@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.top.TopConf;
-import org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIO.POSIX.NoMlockCacheManipulator;
 import org.apache.hadoop.util.VersionInfo;
@@ -46,8 +45,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager.Op;
-import static org.apache.hadoop.hdfs.server.namenode.top.window.RollingWindowManager.TopWindow;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -197,6 +194,8 @@ public class TestNameNodeMXBean {
       assertEquals(NativeIO.POSIX.getCacheManipulator().getMemlockLimit() *
           cluster.getDataNodes().size(),
               mbs.getAttribute(mxbeanName, "CacheCapacity"));
+      assertNull("RollingUpgradeInfo should be null when there is no rolling"
+          + " upgrade", mbs.getAttribute(mxbeanName, "RollingUpgradeStatus"));
     } finally {
       if (cluster != null) {
         for (URI dir : cluster.getNameDirs(0)) {


[08/30] hadoop git commit: YARN-3809. Failed to launch new attempts because ApplicationMasterLauncher's threads all hang. Contributed by Jun Gong

Posted by aw...@apache.org.
YARN-3809. Failed to launch new attempts because ApplicationMasterLauncher's threads all hang. Contributed by Jun Gong


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2a20dd9b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2a20dd9b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2a20dd9b

Branch: refs/heads/HADOOP-12111
Commit: 2a20dd9b61ba3833460cbda0e8c3e8b6366fc3ab
Parents: 72d08a0
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jun 24 16:23:48 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Jun 24 16:23:48 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../hadoop/yarn/conf/YarnConfiguration.java     | 10 +++++++
 .../src/main/resources/yarn-default.xml         | 12 ++++++++
 .../amlauncher/ApplicationMasterLauncher.java   | 30 ++++++++++++++++++--
 4 files changed, 52 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a20dd9b/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f235338..7ecdee3 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -699,6 +699,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3842. NMProxy should retry on NMNotYetReadyException. 
     (Robert Kanter via kasha)
 
+    YARN-3809. Failed to launch new attempts because
+    ApplicationMasterLauncher's threads all hang (Jun Gong via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a20dd9b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 5d75a21..6b660f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -143,6 +143,16 @@ public class YarnConfiguration extends Configuration {
     RM_PREFIX + "client.thread-count";
   public static final int DEFAULT_RM_CLIENT_THREAD_COUNT = 50;
 
+  /** Number of threads used to launch/cleanup AM.*/
+  public static final String RM_AMLAUNCHER_THREAD_COUNT =
+      RM_PREFIX + "amlauncher.thread-count";
+  public static final int DEFAULT_RM_AMLAUNCHER_THREAD_COUNT = 50;
+
+  /** Retry times to connect with NM.*/
+  public static final String RM_NODEMANAGER_CONNECT_RETIRES =
+      RM_PREFIX + "nodemanager-connect-retries";
+  public static final int DEFAULT_RM_NODEMANAGER_CONNECT_RETIRES = 10;
+
   /** The Kerberos principal for the resource manager.*/
   public static final String RM_PRINCIPAL =
     RM_PREFIX + "principal";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a20dd9b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index d94157c..621198c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -77,6 +77,18 @@
   </property>
 
   <property>
+    <description>Number of threads used to launch/cleanup AM.</description>
+    <name>yarn.resourcemanager.amlauncher.thread-count</name>
+    <value>50</value>
+  </property>
+
+  <property>
+    <description>Retry times to connect with NM.</description>
+    <name>yarn.resourcemanager.nodemanager-connect-retries</name>
+    <value>10</value>
+  </property>
+
+  <property>
     <description>The expiry interval for application master reporting.</description>
     <name>yarn.am.liveness-monitor.expiry-interval-ms</name>
     <value>600000</value>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2a20dd9b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
index 5fc39fd..f606e45 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
@@ -19,12 +19,17 @@
 package org.apache.hadoop.yarn.server.resourcemanager.amlauncher;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -34,7 +39,7 @@ public class ApplicationMasterLauncher extends AbstractService implements
     EventHandler<AMLauncherEvent> {
   private static final Log LOG = LogFactory.getLog(
       ApplicationMasterLauncher.class);
-  private final ThreadPoolExecutor launcherPool;
+  private ThreadPoolExecutor launcherPool;
   private LauncherThread launcherHandlingThread;
   
   private final BlockingQueue<Runnable> masterEvents
@@ -45,12 +50,31 @@ public class ApplicationMasterLauncher extends AbstractService implements
   public ApplicationMasterLauncher(RMContext context) {
     super(ApplicationMasterLauncher.class.getName());
     this.context = context;
-    this.launcherPool = new ThreadPoolExecutor(10, 10, 1, 
-        TimeUnit.HOURS, new LinkedBlockingQueue<Runnable>());
     this.launcherHandlingThread = new LauncherThread();
   }
   
   @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    int threadCount = conf.getInt(
+        YarnConfiguration.RM_AMLAUNCHER_THREAD_COUNT,
+        YarnConfiguration.DEFAULT_RM_AMLAUNCHER_THREAD_COUNT);
+    ThreadFactory tf = new ThreadFactoryBuilder()
+        .setNameFormat("ApplicationMasterLauncher #%d")
+        .build();
+    launcherPool = new ThreadPoolExecutor(threadCount, threadCount, 1,
+        TimeUnit.HOURS, new LinkedBlockingQueue<Runnable>());
+    launcherPool.setThreadFactory(tf);
+
+    Configuration newConf = new YarnConfiguration(conf);
+    newConf.setInt(CommonConfigurationKeysPublic.
+            IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SOCKET_TIMEOUTS_KEY,
+        conf.getInt(YarnConfiguration.RM_NODEMANAGER_CONNECT_RETIRES,
+            YarnConfiguration.DEFAULT_RM_NODEMANAGER_CONNECT_RETIRES));
+    setConfig(newConf);
+    super.serviceInit(newConf);
+  }
+
+  @Override
   protected void serviceStart() throws Exception {
     launcherHandlingThread.start();
     super.serviceStart();


[15/30] hadoop git commit: HADOOP-12049. Control http authentication cookie persistence via configuration. Contributed by Huizhi Lu.

Posted by aw...@apache.org.
HADOOP-12049. Control http authentication cookie persistence via configuration. Contributed by Huizhi Lu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a815cc15
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a815cc15
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a815cc15

Branch: refs/heads/HADOOP-12111
Commit: a815cc157ceb24e02189634a85abed8e874568e0
Parents: afe9ea3
Author: Benoy Antony <be...@apache.org>
Authored: Wed Jun 24 15:59:39 2015 -0700
Committer: Benoy Antony <be...@apache.org>
Committed: Wed Jun 24 15:59:39 2015 -0700

----------------------------------------------------------------------
 .../server/AuthenticationFilter.java            |  28 ++-
 .../http/TestAuthenticationSessionCookie.java   | 187 +++++++++++++++++++
 .../apache/hadoop/http/TestHttpCookieFlag.java  |   2 +-
 3 files changed, 213 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a815cc15/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
index 0f86623..bf44f48 100644
--- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
+++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java
@@ -161,6 +161,12 @@ public class AuthenticationFilter implements Filter {
   public static final String COOKIE_PATH = "cookie.path";
 
   /**
+   * Constant for the configuration property
+   * that indicates the persistence of the HTTP cookie.
+   */
+  public static final String COOKIE_PERSISTENT = "cookie.persistent";
+
+  /**
    * Constant for the configuration property that indicates the name of the
    * SignerSecretProvider class to use.
    * Possible values are: "string", "random", "zookeeper", or a classname.
@@ -187,6 +193,7 @@ public class AuthenticationFilter implements Filter {
   private long validity;
   private String cookieDomain;
   private String cookiePath;
+  private boolean isCookiePersistent;
   private boolean isInitializedByTomcat;
 
   /**
@@ -228,6 +235,9 @@ public class AuthenticationFilter implements Filter {
 
     cookieDomain = config.getProperty(COOKIE_DOMAIN, null);
     cookiePath = config.getProperty(COOKIE_PATH, null);
+    isCookiePersistent = Boolean.parseBoolean(
+            config.getProperty(COOKIE_PERSISTENT, "false"));
+
   }
 
   protected void initializeAuthHandler(String authHandlerClassName, FilterConfig filterConfig)
@@ -372,6 +382,15 @@ public class AuthenticationFilter implements Filter {
   }
 
   /**
+   * Returns the cookie persistence to use for the HTTP cookie.
+   *
+   * @return the cookie persistence to use for the HTTP cookie.
+   */
+  protected boolean isCookiePersistent() {
+    return isCookiePersistent;
+  }
+
+  /**
    * Destroys the filter.
    * <p>
    * It invokes the {@link AuthenticationHandler#destroy()} method to release any resources it may hold.
@@ -549,7 +568,8 @@ public class AuthenticationFilter implements Filter {
           if (newToken && !token.isExpired() && token != AuthenticationToken.ANONYMOUS) {
             String signedToken = signer.sign(token.toString());
             createAuthCookie(httpResponse, signedToken, getCookieDomain(),
-                    getCookiePath(), token.getExpires(), isHttps);
+                    getCookiePath(), token.getExpires(),
+                    isCookiePersistent(), isHttps);
           }
           doFilter(filterChain, httpRequest, httpResponse);
         }
@@ -569,7 +589,7 @@ public class AuthenticationFilter implements Filter {
     if (unauthorizedResponse) {
       if (!httpResponse.isCommitted()) {
         createAuthCookie(httpResponse, "", getCookieDomain(),
-                getCookiePath(), 0, isHttps);
+                getCookiePath(), 0, isCookiePersistent(), isHttps);
         // If response code is 401. Then WWW-Authenticate Header should be
         // present.. reset to 403 if not found..
         if ((errCode == HttpServletResponse.SC_UNAUTHORIZED)
@@ -614,6 +634,7 @@ public class AuthenticationFilter implements Filter {
    * @param isSecure is the cookie secure?
    * @param token the token.
    * @param expires the cookie expiration time.
+   * @param isCookiePersistent whether the cookie is persistent or not.
    *
    * XXX the following code duplicate some logic in Jetty / Servlet API,
    * because of the fact that Hadoop is stuck at servlet 2.5 and jetty 6
@@ -621,6 +642,7 @@ public class AuthenticationFilter implements Filter {
    */
   public static void createAuthCookie(HttpServletResponse resp, String token,
                                       String domain, String path, long expires,
+                                      boolean isCookiePersistent,
                                       boolean isSecure) {
     StringBuilder sb = new StringBuilder(AuthenticatedURL.AUTH_COOKIE)
                            .append("=");
@@ -636,7 +658,7 @@ public class AuthenticationFilter implements Filter {
       sb.append("; Domain=").append(domain);
     }
 
-    if (expires >= 0) {
+    if (expires >= 0 && isCookiePersistent) {
       Date date = new Date(expires);
       SimpleDateFormat df = new SimpleDateFormat("EEE, " +
               "dd-MMM-yyyy HH:mm:ss zzz");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a815cc15/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
new file mode 100644
index 0000000..e435034
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestAuthenticationSessionCookie.java
@@ -0,0 +1,187 @@
+/**
+ * Licensed 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. See accompanying LICENSE file.
+ */
+package org.apache.hadoop.http;
+
+import org.junit.Assert;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.junit.After;
+import org.junit.Test;
+import org.mortbay.log.Log;
+
+import javax.servlet.*;
+import javax.servlet.http.HttpServletResponse;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+import java.net.HttpCookie;
+import java.util.List;
+
+public class TestAuthenticationSessionCookie {
+  private static final String BASEDIR = System.getProperty("test.build.dir",
+          "target/test-dir") + "/" + TestHttpCookieFlag.class.getSimpleName();
+  private static boolean isCookiePersistent;
+  private static final long TOKEN_VALIDITY_SEC = 1000;
+  private static long expires;
+  private static String keystoresDir;
+  private static String sslConfDir;
+  private static HttpServer2 server;
+
+  public static class DummyAuthenticationFilter implements Filter {
+
+    @Override
+    public void init(FilterConfig filterConfig) throws ServletException {
+      isCookiePersistent = false;
+    }
+
+    @Override
+    public void doFilter(ServletRequest request, ServletResponse response,
+                         FilterChain chain) throws IOException,
+                                                   ServletException {
+      HttpServletResponse resp = (HttpServletResponse) response;
+      AuthenticationFilter.createAuthCookie(resp, "token", null, null, expires,
+              isCookiePersistent, true);
+      chain.doFilter(request, resp);
+    }
+
+    @Override
+    public void destroy() {
+    }
+  }
+
+  public static class DummyFilterInitializer extends FilterInitializer {
+    @Override
+    public void initFilter(FilterContainer container, Configuration conf) {
+      container.addFilter("DummyAuth", DummyAuthenticationFilter.class
+              .getName(), null);
+    }
+  }
+
+  public static class Dummy2AuthenticationFilter
+  extends DummyAuthenticationFilter {
+
+    @Override
+    public void init(FilterConfig filterConfig) throws ServletException {
+      isCookiePersistent = true;
+      expires = System.currentTimeMillis() + TOKEN_VALIDITY_SEC;
+    }
+
+    @Override
+    public void destroy() {
+    }
+  }
+
+  public static class Dummy2FilterInitializer extends FilterInitializer {
+    @Override
+    public void initFilter(FilterContainer container, Configuration conf) {
+      container.addFilter("Dummy2Auth", Dummy2AuthenticationFilter.class
+              .getName(), null);
+    }
+  }
+
+  public void startServer(boolean isTestSessionCookie) throws Exception {
+    Configuration conf = new Configuration();
+    if (isTestSessionCookie) {
+      conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
+            DummyFilterInitializer.class.getName());
+    } else {
+      conf.set(HttpServer2.FILTER_INITIALIZER_PROPERTY,
+            Dummy2FilterInitializer.class.getName());
+    }
+
+    File base = new File(BASEDIR);
+    FileUtil.fullyDelete(base);
+    base.mkdirs();
+    keystoresDir = new File(BASEDIR).getAbsolutePath();
+    sslConfDir = KeyStoreTestUtil.getClasspathDir(TestSSLHttpServer.class);
+
+    KeyStoreTestUtil.setupSSLConfig(keystoresDir, sslConfDir, conf, false);
+    Configuration sslConf = new Configuration(false);
+    sslConf.addResource("ssl-server.xml");
+    sslConf.addResource("ssl-client.xml");
+
+
+    server = new HttpServer2.Builder()
+            .setName("test")
+            .addEndpoint(new URI("http://localhost"))
+            .addEndpoint(new URI("https://localhost"))
+            .setConf(conf)
+            .keyPassword(sslConf.get("ssl.server.keystore.keypassword"))
+            .keyStore(sslConf.get("ssl.server.keystore.location"),
+                    sslConf.get("ssl.server.keystore.password"),
+                    sslConf.get("ssl.server.keystore.type", "jks"))
+            .trustStore(sslConf.get("ssl.server.truststore.location"),
+                    sslConf.get("ssl.server.truststore.password"),
+                    sslConf.get("ssl.server.truststore.type", "jks")).build();
+    server.addServlet("echo", "/echo", TestHttpServer.EchoServlet.class);
+    server.start();
+  }
+
+  @Test
+  public void testSessionCookie() throws IOException {
+    try {
+        startServer(true);
+    } catch (Exception e) {
+        // Auto-generated catch block
+        e.printStackTrace();
+    }
+
+    URL base = new URL("http://" + NetUtils.getHostPortString(server
+            .getConnectorAddress(0)));
+    HttpURLConnection conn = (HttpURLConnection) new URL(base,
+            "/echo").openConnection();
+
+    String header = conn.getHeaderField("Set-Cookie");
+    List<HttpCookie> cookies = HttpCookie.parse(header);
+    Assert.assertTrue(!cookies.isEmpty());
+    Log.info(header);
+    Assert.assertFalse(header.contains("; Expires="));
+    Assert.assertTrue("token".equals(cookies.get(0).getValue()));
+  }
+  
+  @Test
+  public void testPersistentCookie() throws IOException {
+    try {
+        startServer(false);
+    } catch (Exception e) {
+        // Auto-generated catch block
+        e.printStackTrace();
+    }
+
+    URL base = new URL("http://" + NetUtils.getHostPortString(server
+            .getConnectorAddress(0)));
+    HttpURLConnection conn = (HttpURLConnection) new URL(base,
+            "/echo").openConnection();
+
+    String header = conn.getHeaderField("Set-Cookie");
+    List<HttpCookie> cookies = HttpCookie.parse(header);
+    Assert.assertTrue(!cookies.isEmpty());
+    Log.info(header);
+    Assert.assertTrue(header.contains("; Expires="));
+    Assert.assertTrue("token".equals(cookies.get(0).getValue()));
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    server.stop();
+    FileUtil.fullyDelete(new File(BASEDIR));
+    KeyStoreTestUtil.cleanupSSLConfig(keystoresDir, sslConfDir);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a815cc15/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
index 75a9480..5c5ed48 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpCookieFlag.java
@@ -60,7 +60,7 @@ public class TestHttpCookieFlag {
       HttpServletResponse resp = (HttpServletResponse) response;
       boolean isHttps = "https".equals(request.getScheme());
       AuthenticationFilter.createAuthCookie(resp, "token", null, null, -1,
-              isHttps);
+              true, isHttps);
       chain.doFilter(request, resp);
     }
 


[10/30] hadoop git commit: YARN-3790. usedResource from rootQueue metrics may get stale data for FS scheduler after recovering the container (Zhihai Xu via rohithsharmaks)

Posted by aw...@apache.org.
YARN-3790. usedResource from rootQueue metrics may get stale data for FS scheduler after recovering the container (Zhihai Xu via rohithsharmaks)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/dd4b387d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dd4b387d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dd4b387d

Branch: refs/heads/HADOOP-12111
Commit: dd4b387d96abc66ddebb569b3775b18b19aed027
Parents: 8d58512
Author: rohithsharmaks <ro...@apache.org>
Authored: Wed Jun 24 23:00:14 2015 +0530
Committer: rohithsharmaks <ro...@apache.org>
Committed: Wed Jun 24 23:00:14 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                          |  3 +++
 .../resourcemanager/scheduler/fair/FairScheduler.java    | 11 +++++++----
 2 files changed, 10 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd4b387d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a5fc86b..9547f0f 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -705,6 +705,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3832. Resource Localization fails on a cluster due to existing cache
     directories (Brahma Reddy Battula via jlowe)
 
+    YARN-3790. usedResource from rootQueue metrics may get stale data for FS
+    scheduler after recovering the container (Zhihai Xu via rohithsharmaks)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dd4b387d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 2ed3b2a..cbc10e7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
+import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationConstants;
@@ -841,11 +842,11 @@ public class FairScheduler extends
         + " with event: " + event);
   }
 
-  private synchronized void addNode(RMNode node) {
+  private synchronized void addNode(List<NMContainerStatus> containerReports,
+      RMNode node) {
     FSSchedulerNode schedulerNode = new FSSchedulerNode(node, usePortForNodeName);
     nodes.put(node.getNodeID(), schedulerNode);
     Resources.addTo(clusterResource, node.getTotalCapability());
-    updateRootQueueMetrics();
     updateMaximumAllocation(schedulerNode, true);
 
     triggerUpdate();
@@ -854,6 +855,9 @@ public class FairScheduler extends
     queueMgr.getRootQueue().recomputeSteadyShares();
     LOG.info("Added node " + node.getNodeAddress() +
         " cluster capacity: " + clusterResource);
+
+    recoverContainersOnNode(containerReports, node);
+    updateRootQueueMetrics();
   }
 
   private synchronized void removeNode(RMNode rmNode) {
@@ -1147,8 +1151,7 @@ public class FairScheduler extends
         throw new RuntimeException("Unexpected event type: " + event);
       }
       NodeAddedSchedulerEvent nodeAddedEvent = (NodeAddedSchedulerEvent)event;
-      addNode(nodeAddedEvent.getAddedRMNode());
-      recoverContainersOnNode(nodeAddedEvent.getContainerReports(),
+      addNode(nodeAddedEvent.getContainerReports(),
           nodeAddedEvent.getAddedRMNode());
       break;
     case NODE_REMOVED:


[06/30] hadoop git commit: HDFS-8639. Add Option for NameNode HTTP port in MiniDFSClusterManager. Contributed by Kai Sasaki.

Posted by aw...@apache.org.
HDFS-8639. Add Option for NameNode HTTP port in MiniDFSClusterManager. Contributed by Kai Sasaki.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2ba64657
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2ba64657
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2ba64657

Branch: refs/heads/HADOOP-12111
Commit: 2ba646572185b91d6db1b09837abdcbadbfbeb49
Parents: 49dfad9
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Jun 23 22:16:16 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Tue Jun 23 22:16:16 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../test/java/org/apache/hadoop/test/MiniDFSClusterManager.java  | 4 ++++
 2 files changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ba64657/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 09a6891..d94a213 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -663,6 +663,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-6564. Use slf4j instead of common-logging in hdfs-client.
     (Rakesh R via wheat9)
 
+    HDFS-8639. Add Option for NameNode HTTP port in MiniDFSClusterManager.
+    (Kai Sasaki via jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2ba64657/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
index 7029f42..f8e931a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/test/MiniDFSClusterManager.java
@@ -65,6 +65,7 @@ public class MiniDFSClusterManager {
   private String writeDetails;
   private int numDataNodes;
   private int nameNodePort;
+  private int nameNodeHttpPort;
   private StartupOption dfsOpts;
   private String writeConfig;
   private Configuration conf;
@@ -84,6 +85,7 @@ public class MiniDFSClusterManager {
         .addOption("cmdport", true,
             "Which port to listen on for commands (default 0--we choose)")
         .addOption("nnport", true, "NameNode port (default 0--we choose)")
+        .addOption("httpport", true, "NameNode http port (default 0--we choose)")
         .addOption("namenode", true, "URL of the namenode (default "
             + "is either the DFS cluster or a temporary dir)")     
         .addOption(OptionBuilder
@@ -137,6 +139,7 @@ public class MiniDFSClusterManager {
    */
   public void start() throws IOException, FileNotFoundException {
     dfs = new MiniDFSCluster.Builder(conf).nameNodePort(nameNodePort)
+                                          .nameNodeHttpPort(nameNodeHttpPort)
                                           .numDataNodes(numDataNodes)
                                           .startupOption(dfsOpts)
                                           .format(format)
@@ -198,6 +201,7 @@ public class MiniDFSClusterManager {
     // HDFS
     numDataNodes = intArgument(cli, "datanodes", 1);
     nameNodePort = intArgument(cli, "nnport", 0);
+    nameNodeHttpPort = intArgument(cli, "httpport", 0);
     if (cli.hasOption("format")) {
       dfsOpts = StartupOption.FORMAT;
       format = true;


[02/30] hadoop git commit: HDFS-6564. Use slf4j instead of common-logging in hdfs-client. Contributed by Rakesh R.

Posted by aw...@apache.org.
HDFS-6564. Use slf4j instead of common-logging in hdfs-client. Contributed by Rakesh R.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/122cad6a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/122cad6a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/122cad6a

Branch: refs/heads/HADOOP-12111
Commit: 122cad6aec5839d8d515c5008425ecb34f2fa56b
Parents: d481684
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Jun 23 11:40:21 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Jun 23 11:41:40 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  | 10 ++++++++
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |  7 +++--
 .../hdfs/protocol/BlockStoragePolicy.java       | 12 ++++-----
 .../hadoop/hdfs/protocol/CachePoolInfo.java     |  3 ---
 .../org/apache/hadoop/hdfs/web/TokenAspect.java | 16 ++++++------
 .../hadoop/hdfs/web/URLConnectionFactory.java   | 13 +++++-----
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      | 27 ++++++++++----------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 8 files changed, 49 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
index 33c2ed9..1b45095 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
@@ -35,6 +35,16 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <scope>provided</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>log4j</groupId>
+          <artifactId>log4j</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index eda135e..0e72b98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -317,10 +317,9 @@ public class DFSUtilClient {
       if (address != null) {
         InetSocketAddress isa = NetUtils.createSocketAddr(address);
         if (isa.isUnresolved()) {
-          LOG.warn("Namenode for " + nsId +
-                       " remains unresolved for ID " + nnId +
-                   ".  Check your hdfs-site.xml file to " +
-                   "ensure namenodes are configured properly.");
+          LOG.warn("Namenode for {} remains unresolved for ID {}. Check your "
+              + "hdfs-site.xml file to ensure namenodes are configured "
+              + "properly.", nsId, nnId);
         }
         ret.put(nnId, isa);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
index 2624960..0225009 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockStoragePolicy.java
@@ -158,13 +158,11 @@ public class BlockStoragePolicy implements BlockStoragePolicySpi {
     // remove excess storage types after fallback replacement.
     diff(storageTypes, excess, null);
     if (storageTypes.size() < expectedSize) {
-      LOG.warn("Failed to place enough replicas: expected size is " + expectedSize
-          + " but only " + storageTypes.size() + " storage types can be selected "
-          + "(replication=" + replication
-          + ", selected=" + storageTypes
-          + ", unavailable=" + unavailables
-          + ", removed=" + removed
-          + ", policy=" + this + ")");
+      LOG.warn("Failed to place enough replicas: expected size is {}"
+          + " but only {} storage types can be selected (replication={},"
+          + " selected={}, unavailable={}" + ", removed={}" + ", policy={}"
+          + ")", expectedSize, storageTypes.size(), replication, storageTypes,
+          unavailables, removed, this);
     }
     return storageTypes;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
index 61bbe38..41ec2f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolInfo.java
@@ -24,8 +24,6 @@ import javax.annotation.Nullable;
 
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.InvalidRequestException;
@@ -41,7 +39,6 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo.Expiration;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
 public class CachePoolInfo {
-  public static final Log LOG = LogFactory.getLog(CachePoolInfo.class);
 
   /**
    * Indicates that the pool does not have a maximum relative expiry.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
index bc3eb4b..a864d37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/TokenAspect.java
@@ -21,8 +21,6 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DelegationTokenRenewer;
@@ -37,6 +35,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -115,11 +115,11 @@ final class TokenAspect<T extends FileSystem & Renewable> {
   private final DTSelecorByKind dtSelector;
   private final T fs;
   private boolean hasInitedToken;
-  private final Log LOG;
+  private final Logger LOG;
   private final Text serviceName;
 
   TokenAspect(T fs, final Text serviceName, final Text kind) {
-    this.LOG = LogFactory.getLog(fs.getClass());
+    this.LOG = LoggerFactory.getLogger(fs.getClass());
     this.fs = fs;
     this.dtSelector = new DTSelecorByKind(kind);
     this.serviceName = serviceName;
@@ -134,8 +134,8 @@ final class TokenAspect<T extends FileSystem & Renewable> {
       if (token != null) {
         fs.setDelegationToken(token);
         addRenewAction(fs);
-        if(LOG.isDebugEnabled()) {
-          LOG.debug("Created new DT for " + token.getService());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Created new DT for {}", token.getService());
         }
       }
       hasInitedToken = true;
@@ -149,8 +149,8 @@ final class TokenAspect<T extends FileSystem & Renewable> {
   synchronized void initDelegationToken(UserGroupInformation ugi) {
     Token<?> token = selectDelegationToken(ugi);
     if (token != null) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("Found existing DT for " + token.getService());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Found existing DT for {}", token.getService());
       }
       fs.setDelegationToken(token);
       hasInitedToken = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
index e330adf..a5e02f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/URLConnectionFactory.java
@@ -28,8 +28,6 @@ import javax.net.ssl.HostnameVerifier;
 import javax.net.ssl.HttpsURLConnection;
 import javax.net.ssl.SSLSocketFactory;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -38,6 +36,8 @@ import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.ConnectionConfigurator;
 import org.apache.hadoop.security.ssl.SSLFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -47,7 +47,8 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceAudience.LimitedPrivate({ "HDFS" })
 @InterfaceStability.Unstable
 public class URLConnectionFactory {
-  private static final Log LOG = LogFactory.getLog(URLConnectionFactory.class);
+  private static final Logger LOG = LoggerFactory
+      .getLogger(URLConnectionFactory.class);
 
   /**
    * Timeout for socket connects and reads
@@ -154,16 +155,14 @@ public class URLConnectionFactory {
       throws IOException, AuthenticationException {
     if (isSpnego) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("open AuthenticatedURL connection" + url);
+        LOG.debug("open AuthenticatedURL connection {}", url);
       }
       UserGroupInformation.getCurrentUser().checkTGTAndReloginFromKeytab();
       final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
       return new AuthenticatedURL(new KerberosUgiAuthenticator(),
           connConfigurator).openConnection(url, authToken);
     } else {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("open URL connection");
-      }
+      LOG.debug("open URL connection");
       URLConnection connection = url.openConnection();
       if (connection instanceof HttpURLConnection) {
         connConfigurator.configure((HttpURLConnection) connection);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index d902738..2650dca 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -36,8 +36,6 @@ import java.util.StringTokenizer;
 
 import javax.ws.rs.core.MediaType;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -81,6 +79,8 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelect
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.StringUtils;
 import org.codehaus.jackson.map.ObjectMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -89,7 +89,8 @@ import com.google.common.collect.Lists;
 /** A FileSystem for HDFS over the web. */
 public class WebHdfsFileSystem extends FileSystem
     implements DelegationTokenRenewer.Renewable, TokenAspect.TokenManagementDelegator {
-  public static final Log LOG = LogFactory.getLog(WebHdfsFileSystem.class);
+  public static final Logger LOG = LoggerFactory
+      .getLogger(WebHdfsFileSystem.class);
   /** WebHdfs version. */
   public static final int VERSION = 1;
   /** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
@@ -221,14 +222,14 @@ public class WebHdfsFileSystem extends FileSystem
       // to get another token to match hdfs/rpc behavior
       if (token != null) {
         if(LOG.isDebugEnabled()) {
-          LOG.debug("Using UGI token: " + token);
+          LOG.debug("Using UGI token: {}", token);
         }
         canRefreshDelegationToken = false;
       } else {
         token = getDelegationToken(null);
         if (token != null) {
           if(LOG.isDebugEnabled()) {
-            LOG.debug("Fetched new token: " + token);
+            LOG.debug("Fetched new token: {}", token);
           }
         } else { // security is disabled
           canRefreshDelegationToken = false;
@@ -245,7 +246,7 @@ public class WebHdfsFileSystem extends FileSystem
     if (canRefreshDelegationToken) {
       Token<?> token = getDelegationToken(null);
       if(LOG.isDebugEnabled()) {
-        LOG.debug("Replaced expired token: " + token);
+        LOG.debug("Replaced expired token: {}", token);
       }
       setDelegationToken(token);
       replaced = (token != null);
@@ -430,7 +431,7 @@ public class WebHdfsFileSystem extends FileSystem
     final URL url = new URL(getTransportScheme(), nnAddr.getHostName(),
           nnAddr.getPort(), path + '?' + query);
     if (LOG.isTraceEnabled()) {
-      LOG.trace("url=" + url);
+      LOG.trace("url={}", url);
     }
     return url;
   }
@@ -467,7 +468,7 @@ public class WebHdfsFileSystem extends FileSystem
         + Param.toSortedString("&", parameters);
     final URL url = getNamenodeURL(path, query);
     if (LOG.isTraceEnabled()) {
-      LOG.trace("url=" + url);
+      LOG.trace("url={}", url);
     }
     return url;
   }
@@ -658,9 +659,9 @@ public class WebHdfsFileSystem extends FileSystem
               a.action == RetryPolicy.RetryAction.RetryDecision.FAILOVER_AND_RETRY;
 
           if (isRetry || isFailoverAndRetry) {
-            LOG.info("Retrying connect to namenode: " + nnAddr
-                + ". Already tried " + retry + " time(s); retry policy is "
-                + retryPolicy + ", delay " + a.delayMillis + "ms.");
+            LOG.info("Retrying connect to namenode: {}. Already tried {}"
+                + " time(s); retry policy is {}, delay {}ms.", nnAddr, retry,
+                retryPolicy, a.delayMillis);
 
             if (isFailoverAndRetry) {
               resetStateToFailOver();
@@ -757,7 +758,7 @@ public class WebHdfsFileSystem extends FileSystem
         final IOException ioe =
             new IOException("Response decoding failure: "+e.toString(), e);
         if (LOG.isDebugEnabled()) {
-          LOG.debug(ioe);
+          LOG.debug("Response decoding failure: {}", e.toString(), e);
         }
         throw ioe;
       } finally {
@@ -1212,7 +1213,7 @@ public class WebHdfsFileSystem extends FileSystem
       }
     } catch (IOException ioe) {
       if (LOG.isDebugEnabled()) {
-        LOG.debug("Token cancel failed: " + ioe);
+        LOG.debug("Token cancel failed: ", ioe);
       }
     } finally {
       super.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/122cad6a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 4bd0e8b..981ca55 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -658,6 +658,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8192. Eviction should key off used locked memory instead of
     ram disk free space. (Arpit Agarwal)
 
+    HDFS-6564. Use slf4j instead of common-logging in hdfs-client.
+    (Rakesh R via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[18/30] hadoop git commit: HDFS-8462. Implement GETXATTRS and LISTXATTRS operations for WebImageViewer. Contributed by Jagadesh Kiran N.

Posted by aw...@apache.org.
HDFS-8462. Implement GETXATTRS and LISTXATTRS operations for WebImageViewer. Contributed by Jagadesh Kiran N.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/bc433908
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bc433908
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bc433908

Branch: refs/heads/HADOOP-12111
Commit: bc433908d35758ff0a7225cd6f5662959ef4d294
Parents: b381f88
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jun 26 00:20:12 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Jun 26 00:20:12 2015 +0900

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../offlineImageViewer/FSImageHandler.java      |  57 ++--
 .../tools/offlineImageViewer/FSImageLoader.java |  75 +++++-
 .../src/site/markdown/HdfsImageViewer.md        |   2 +
 .../TestOfflineImageViewerForXAttr.java         | 262 +++++++++++++++++++
 5 files changed, 380 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc433908/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 4268154..e5c30bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -666,6 +666,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8639. Add Option for NameNode HTTP port in MiniDFSClusterManager.
     (Kai Sasaki via jing9)
 
+    HDFS-8462. Implement GETXATTRS and LISTXATTRS operations for WebImageViewer.
+    (Jagadesh Kiran N via aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc433908/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java
index 37db8b7..da02805 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java
@@ -22,6 +22,7 @@ import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_LENGTH;
 import static io.netty.handler.codec.http.HttpHeaderNames.CONTENT_TYPE;
 import static io.netty.handler.codec.http.HttpHeaderValues.CLOSE;
 import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
+import static io.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
 import static io.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
 import static io.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
 import static io.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
@@ -74,10 +75,10 @@ class FSImageHandler extends SimpleChannelInboundHandler<HttpRequest> {
 
   @Override
   public void channelRead0(ChannelHandlerContext ctx, HttpRequest request)
-          throws Exception {
+      throws Exception {
     if (request.method() != HttpMethod.GET) {
       DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1,
-        METHOD_NOT_ALLOWED);
+          METHOD_NOT_ALLOWED);
       resp.headers().set(CONNECTION, CLOSE);
       ctx.write(resp).addListener(ChannelFutureListener.CLOSE);
       return;
@@ -89,25 +90,33 @@ class FSImageHandler extends SimpleChannelInboundHandler<HttpRequest> {
     final String content;
     String path = getPath(decoder);
     switch (op) {
-      case "GETFILESTATUS":
-        content = image.getFileStatus(path);
-        break;
-      case "LISTSTATUS":
-        content = image.listStatus(path);
-        break;
-      case "GETACLSTATUS":
-        content = image.getAclStatus(path);
-        break;
-      default:
-        throw new IllegalArgumentException(
-            "Invalid value for webhdfs parameter" + " \"op\"");
+    case "GETFILESTATUS":
+      content = image.getFileStatus(path);
+      break;
+    case "LISTSTATUS":
+      content = image.listStatus(path);
+      break;
+    case "GETACLSTATUS":
+      content = image.getAclStatus(path);
+      break;
+    case "GETXATTRS":
+      List<String> names = getXattrNames(decoder);
+      String encoder = getEncoder(decoder);
+      content = image.getXAttrs(path, names, encoder);
+      break;
+    case "LISTXATTRS":
+      content = image.listXAttrs(path);
+      break;
+    default:
+      throw new IllegalArgumentException("Invalid value for webhdfs parameter"
+          + " \"op\"");
     }
 
     LOG.info("op=" + op + " target=" + path);
 
-    DefaultFullHttpResponse resp = new DefaultFullHttpResponse(
-            HTTP_1_1, HttpResponseStatus.OK,
-            Unpooled.wrappedBuffer(content.getBytes(Charsets.UTF_8)));
+    DefaultFullHttpResponse resp = new DefaultFullHttpResponse(HTTP_1_1,
+        HttpResponseStatus.OK, Unpooled.wrappedBuffer(content
+            .getBytes(Charsets.UTF_8)));
     resp.headers().set(CONTENT_TYPE, APPLICATION_JSON_UTF8);
     resp.headers().set(CONTENT_LENGTH, resp.content().readableBytes());
     resp.headers().set(CONNECTION, CLOSE);
@@ -134,8 +143,9 @@ class FSImageHandler extends SimpleChannelInboundHandler<HttpRequest> {
       resp.setStatus(BAD_REQUEST);
     } else if (e instanceof FileNotFoundException) {
       resp.setStatus(NOT_FOUND);
+    } else if (e instanceof IOException) {
+      resp.setStatus(FORBIDDEN);
     }
-
     resp.headers().set(CONTENT_LENGTH, resp.content().readableBytes());
     resp.headers().set(CONNECTION, CLOSE);
     ctx.write(resp).addListener(ChannelFutureListener.CLOSE);
@@ -147,6 +157,17 @@ class FSImageHandler extends SimpleChannelInboundHandler<HttpRequest> {
         ? StringUtils.toUpperCase(parameters.get("op").get(0)) : null;
   }
 
+  private static List<String> getXattrNames(QueryStringDecoder decoder) {
+    Map<String, List<String>> parameters = decoder.parameters();
+    return parameters.get("xattr.name");
+  }
+
+  private static String getEncoder(QueryStringDecoder decoder) {
+    Map<String, List<String>> parameters = decoder.parameters();
+    return parameters.containsKey("encoding") ? parameters.get("encoding").get(
+        0) : null;
+  }
+
   private static String getPath(QueryStringDecoder decoder)
           throws FileNotFoundException {
     String path = decoder.path();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc433908/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
index 351ff03..2a11734 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageLoader.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 
 import java.io.BufferedInputStream;
-import java.io.EOFException;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -38,10 +37,12 @@ import com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSImageUtil;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.hdfs.web.JsonUtil;
+import org.apache.hadoop.hdfs.web.resources.XAttrEncodingParam;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.LimitInputStream;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -309,6 +311,77 @@ class FSImageLoader {
   }
 
   /**
+   * Return the JSON formatted XAttrNames of the specified file.
+   *
+   * @param path
+   *          a path specifies a file
+   * @return JSON formatted XAttrNames
+   * @throws IOException
+   *           if failed to serialize fileStatus to JSON.
+   */
+  String listXAttrs(String path) throws IOException {
+    return JsonUtil.toJsonString(getXAttrList(path));
+  }
+
+  /**
+   * Return the JSON formatted XAttrs of the specified file.
+   *
+   * @param path
+   *          a path specifies a file
+   * @return JSON formatted XAttrs
+   * @throws IOException
+   *           if failed to serialize fileStatus to JSON.
+   */
+  String getXAttrs(String path, List<String> names, String encoder)
+      throws IOException {
+
+    List<XAttr> xAttrs = getXAttrList(path);
+    List<XAttr> filtered;
+    if (names == null || names.size() == 0) {
+      filtered = xAttrs;
+    } else {
+      filtered = Lists.newArrayListWithCapacity(names.size());
+      for (String name : names) {
+        XAttr search = XAttrHelper.buildXAttr(name);
+
+        boolean found = false;
+        for (XAttr aXAttr : xAttrs) {
+          if (aXAttr.getNameSpace() == search.getNameSpace()
+              && aXAttr.getName().equals(search.getName())) {
+
+            filtered.add(aXAttr);
+            found = true;
+            break;
+          }
+        }
+
+        if (!found) {
+          throw new IOException(
+              "At least one of the attributes provided was not found.");
+        }
+      }
+
+    }
+    return JsonUtil.toJsonString(filtered,
+        new XAttrEncodingParam(encoder).getEncoding());
+  }
+
+  private List<XAttr> getXAttrList(String path) throws IOException {
+    long id = lookup(path);
+    FsImageProto.INodeSection.INode inode = fromINodeId(id);
+    switch (inode.getType()) {
+    case FILE:
+      return FSImageFormatPBINode.Loader.loadXAttrs(
+          inode.getFile().getXAttrs(), stringTable);
+    case DIRECTORY:
+      return FSImageFormatPBINode.Loader.loadXAttrs(inode.getDirectory()
+          .getXAttrs(), stringTable);
+    default:
+      return null;
+    }
+  }
+
+  /**
    * Return the JSON formatted ACL status of the specified file.
    * @param path a path specifies a file
    * @return JSON formatted AclStatus

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc433908/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
index e3952cd..9b9d80a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsImageViewer.md
@@ -98,6 +98,8 @@ The Web processor now supports the following operations:
 * [LISTSTATUS](./WebHDFS.html#List_a_Directory)
 * [GETFILESTATUS](./WebHDFS.html#Status_of_a_FileDirectory)
 * [GETACLSTATUS](./WebHDFS.html#Get_ACL_Status)
+* [GETXATTRS](./WebHDFS.html#Get_an_XAttr)
+* [LISTXATTRS](./WebHDFS.html#List_all_XAttrs)
 
 ### XML Processor
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc433908/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForXAttr.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForXAttr.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForXAttr.java
new file mode 100644
index 0000000..3f23f64
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForXAttr.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.hdfs.tools.offlineImageViewer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.XAttrHelper;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.web.JsonUtil;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.net.NetUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Tests OfflineImageViewer if the input fsimage has XAttributes
+ */
+public class TestOfflineImageViewerForXAttr {
+
+  private static final Log LOG = LogFactory
+      .getLog(TestOfflineImageViewerForXAttr.class);
+
+  private static File originalFsimage = null;
+
+  static String attr1JSon;
+
+  /**
+   * Create a populated namespace for later testing. Save its contents to a data
+   * structure and store its fsimage location. We only want to generate the
+   * fsimage file once and use it for multiple tests.
+   */
+  @BeforeClass
+  public static void createOriginalFSImage() throws IOException {
+    MiniDFSCluster cluster = null;
+    Configuration conf = new Configuration();
+
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      DistributedFileSystem hdfs = cluster.getFileSystem();
+      // Create a name space with XAttributes
+      Path dir = new Path("/dir1");
+      hdfs.mkdirs(dir);
+      hdfs.setXAttr(dir, "user.attr1", "value1".getBytes());
+      hdfs.setXAttr(dir, "user.attr2", "value2".getBytes());
+      // Write results to the fsimage file
+      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER, false);
+      hdfs.saveNamespace();
+
+      List<XAttr> attributes = new ArrayList<XAttr>();
+      attributes.add(XAttrHelper.buildXAttr("user.attr1", "value1".getBytes()));
+
+      attr1JSon = JsonUtil.toJsonString(attributes, null);
+
+      attributes.add(XAttrHelper.buildXAttr("user.attr2", "value2".getBytes()));
+
+      // Determine the location of the fsimage file
+      originalFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil
+          .getFSImage(cluster.getNameNode()).getStorage().getStorageDir(0));
+      if (originalFsimage == null) {
+        throw new RuntimeException("Didn't generate or can't find fsimage");
+      }
+      LOG.debug("original FS image file is " + originalFsimage);
+    } finally {
+      if (cluster != null)
+        cluster.shutdown();
+    }
+  }
+
+  @AfterClass
+  public static void deleteOriginalFSImage() throws IOException {
+    if (originalFsimage != null && originalFsimage.exists()) {
+      originalFsimage.delete();
+    }
+  }
+
+  @Test
+  public void testWebImageViewerForListXAttrs() throws Exception {
+    try (WebImageViewer viewer = new WebImageViewer(
+        NetUtils.createSocketAddr("localhost:0"))) {
+      viewer.initServer(originalFsimage.getAbsolutePath());
+      int port = viewer.getPort();
+
+      URL url = new URL("http://localhost:" + port
+          + "/webhdfs/v1/dir1/?op=LISTXATTRS");
+      HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+      connection.setRequestMethod("GET");
+      connection.connect();
+
+      assertEquals(HttpURLConnection.HTTP_OK, connection.getResponseCode());
+
+      String content = IOUtils.toString(connection.getInputStream());
+
+      assertTrue("Missing user.attr1 in response ",
+          content.contains("user.attr1"));
+      assertTrue("Missing user.attr2 in response ",
+          content.contains("user.attr2"));
+
+    }
+  }
+
+  @Test
+  public void testWebImageViewerForGetXAttrsWithOutParameters()
+      throws Exception {
+    try (WebImageViewer viewer = new WebImageViewer(
+        NetUtils.createSocketAddr("localhost:0"))) {
+      viewer.initServer(originalFsimage.getAbsolutePath());
+      int port = viewer.getPort();
+
+      URL url = new URL("http://localhost:" + port
+          + "/webhdfs/v1/dir1/?op=GETXATTRS");
+      HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+      connection.setRequestMethod("GET");
+      connection.connect();
+
+      assertEquals(HttpURLConnection.HTTP_OK, connection.getResponseCode());
+      String content = IOUtils.toString(connection.getInputStream());
+
+      assertTrue("Missing user.attr1 in response ",
+          content.contains("user.attr1"));
+      assertTrue("Missing user.attr2 in response ",
+          content.contains("user.attr2"));
+    }
+  }
+
+  @Test
+  public void testWebImageViewerForGetXAttrsWithParameters() throws Exception {
+    try (WebImageViewer viewer = new WebImageViewer(
+        NetUtils.createSocketAddr("localhost:0"))) {
+
+      viewer.initServer(originalFsimage.getAbsolutePath());
+      int port = viewer.getPort();
+
+      URL url = new URL("http://localhost:" + port
+          + "/webhdfs/v1/dir1/?op=GETXATTRS&xattr.name=attr8");
+      HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+      connection.setRequestMethod("GET");
+      connection.connect();
+
+      assertEquals(HttpURLConnection.HTTP_BAD_REQUEST,
+          connection.getResponseCode());
+
+      url = new URL("http://localhost:" + port
+          + "/webhdfs/v1/dir1/?op=GETXATTRS&xattr.name=user.attr1");
+      connection = (HttpURLConnection) url.openConnection();
+      connection.setRequestMethod("GET");
+      connection.connect();
+
+      assertEquals(HttpURLConnection.HTTP_OK, connection.getResponseCode());
+      String content = IOUtils.toString(connection.getInputStream());
+      assertEquals(attr1JSon, content);
+    }
+  }
+
+  @Test
+  public void testWebImageViewerForGetXAttrsWithCodecParameters()
+      throws Exception {
+    try (WebImageViewer viewer = new WebImageViewer(
+        NetUtils.createSocketAddr("localhost:0"))) {
+      viewer.initServer(originalFsimage.getAbsolutePath());
+      int port = viewer.getPort();
+
+      URL url = new URL(
+          "http://localhost:"
+              + port
+              + "/webhdfs/v1/dir1/?op=GETXATTRS&xattr.name=USER.attr1&encoding=TEXT");
+      HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+      connection.setRequestMethod("GET");
+      connection.connect();
+
+      assertEquals(HttpURLConnection.HTTP_OK, connection.getResponseCode());
+      String content = IOUtils.toString(connection.getInputStream());
+      assertEquals(attr1JSon, content);
+
+    }
+  }
+
+  @Test
+  public void testWithWebHdfsFileSystem() throws Exception {
+    try (WebImageViewer viewer = new WebImageViewer(
+        NetUtils.createSocketAddr("localhost:0"))) {
+      viewer.initServer(originalFsimage.getAbsolutePath());
+      int port = viewer.getPort();
+
+      // create a WebHdfsFileSystem instance
+      URI uri = new URI("webhdfs://localhost:" + String.valueOf(port));
+      Configuration conf = new Configuration();
+      WebHdfsFileSystem webhdfs = (WebHdfsFileSystem) FileSystem.get(uri, conf);
+
+      List<String> names = webhdfs.listXAttrs(new Path("/dir1"));
+      assertTrue(names.contains("user.attr1"));
+      assertTrue(names.contains("user.attr2"));
+
+      String value = new String(webhdfs.getXAttr(new Path("/dir1"),
+          "user.attr1"));
+      assertEquals("value1", value);
+
+      Map<String, byte[]> contentMap = webhdfs.getXAttrs(new Path("/dir1"),
+          names);
+
+      assertEquals("value1", new String(contentMap.get("user.attr1")));
+      assertEquals("value2", new String(contentMap.get("user.attr2")));
+    }
+  }
+
+  @Test
+  public void testResponseCode() throws Exception {
+    try (WebImageViewer viewer = new WebImageViewer(
+        NetUtils.createSocketAddr("localhost:0"))) {
+      viewer.initServer(originalFsimage.getAbsolutePath());
+      int port = viewer.getPort();
+
+      URL url = new URL(
+          "http://localhost:"
+              + port
+              + "/webhdfs/v1/dir1/?op=GETXATTRS&xattr.name=user.notpresent&encoding=TEXT");
+      HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+      connection.setRequestMethod("GET");
+      connection.connect();
+
+      assertEquals(HttpURLConnection.HTTP_FORBIDDEN,
+          connection.getResponseCode());
+
+    }
+  }
+}


[24/30] hadoop git commit: YARN-3850. NM fails to read files from full disks which can lead to container logs being lost and other issues. Contributed by Varun Saxena

Posted by aw...@apache.org.
YARN-3850. NM fails to read files from full disks which can lead to container logs being lost and other issues. Contributed by Varun Saxena


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/40b25694
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/40b25694
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/40b25694

Branch: refs/heads/HADOOP-12111
Commit: 40b256949ad6f6e0dbdd248f2d257b05899f4332
Parents: 8ef07f7
Author: Jason Lowe <jl...@apache.org>
Authored: Fri Jun 26 15:47:07 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Fri Jun 26 15:47:07 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../nodemanager/LocalDirsHandlerService.java    | 24 +++++++++
 .../launcher/RecoveredContainerLaunch.java      |  3 +-
 .../logaggregation/AppLogAggregatorImpl.java    |  4 +-
 .../nodemanager/webapp/ContainerLogsUtils.java  |  2 +-
 .../TestLogAggregationService.java              | 54 +++++++++++++++-----
 .../webapp/TestContainerLogsPage.java           | 22 +++++++-
 7 files changed, 93 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7dc02cd..d74e8a9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -716,6 +716,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3832. Resource Localization fails on a cluster due to existing cache
     directories (Brahma Reddy Battula via jlowe)
 
+    YARN-3850. NM fails to read files from full disks which can lead to
+    container logs being lost and other issues (Varun Saxena via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
index 57d4395..0a61035 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
@@ -238,6 +238,18 @@ public class LocalDirsHandlerService extends AbstractService {
   }
 
   /**
+   * Function to get the local dirs which should be considered for reading
+   * existing files on disk. Contains the good local dirs and the local dirs
+   * that have reached the disk space limit
+   *
+   * @return the local dirs which should be considered for reading
+   */
+  public List<String> getLocalDirsForRead() {
+    return DirectoryCollection.concat(localDirs.getGoodDirs(),
+        localDirs.getFullDirs());
+  }
+
+  /**
    * Function to get the local dirs which should be considered when cleaning up
    * resources. Contains the good local dirs and the local dirs that have reached
    * the disk space limit
@@ -250,6 +262,18 @@ public class LocalDirsHandlerService extends AbstractService {
   }
 
   /**
+   * Function to get the log dirs which should be considered for reading
+   * existing files on disk. Contains the good log dirs and the log dirs that
+   * have reached the disk space limit
+   *
+   * @return the log dirs which should be considered for reading
+   */
+  public List<String> getLogDirsForRead() {
+    return DirectoryCollection.concat(logDirs.getGoodDirs(),
+        logDirs.getFullDirs());
+  }
+
+  /**
    * Function to get the log dirs which should be considered when cleaning up
    * resources. Contains the good log dirs and the log dirs that have reached
    * the disk space limit

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
index fb10f22..d7b9ae2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/RecoveredContainerLaunch.java
@@ -126,7 +126,8 @@ public class RecoveredContainerLaunch extends ContainerLaunch {
 
   private File locatePidFile(String appIdStr, String containerIdStr) {
     String pidSubpath= getPidFileSubpath(appIdStr, containerIdStr);
-    for (String dir : getContext().getLocalDirsHandler().getLocalDirs()) {
+    for (String dir : getContext().getLocalDirsHandler().
+        getLocalDirsForRead()) {
       File pidFile = new File(dir, pidSubpath);
       if (pidFile.exists()) {
         return pidFile;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
index 81be813..4b95a03 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
@@ -595,10 +595,10 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
         boolean appFinished) {
       LOG.info("Uploading logs for container " + containerId
           + ". Current good log dirs are "
-          + StringUtils.join(",", dirsHandler.getLogDirs()));
+          + StringUtils.join(",", dirsHandler.getLogDirsForRead()));
       final LogKey logKey = new LogKey(containerId);
       final LogValue logValue =
-          new LogValue(dirsHandler.getLogDirs(), containerId,
+          new LogValue(dirsHandler.getLogDirsForRead(), containerId,
             userUgi.getShortUserName(), logAggregationContext,
             this.uploadedFileMeta, appFinished);
       try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
index c588a89..319f49b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/ContainerLogsUtils.java
@@ -74,7 +74,7 @@ public class ContainerLogsUtils {
   
   static List<File> getContainerLogDirs(ContainerId containerId,
       LocalDirsHandlerService dirsHandler) throws YarnException {
-    List<String> logDirs = dirsHandler.getLogDirs();
+    List<String> logDirs = dirsHandler.getLogDirsForRead();
     List<File> containerLogDirs = new ArrayList<File>(logDirs.size());
     for (String logDir : logDirs) {
       logDir = new File(logDir).toURI().getPath();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index eb0d055..fd97cef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -177,22 +177,11 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     dispatcher.close();
   }
 
-  @Test
-  public void testLocalFileDeletionAfterUpload() throws Exception {
-    this.delSrvc = new DeletionService(createContainerExecutor());
-    delSrvc = spy(delSrvc);
-    this.delSrvc.init(conf);
-    this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
-    this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
-        this.remoteRootLogDir.getAbsolutePath());
-    
-    LogAggregationService logAggregationService = spy(
-        new LogAggregationService(dispatcher, this.context, this.delSrvc,
-                                  super.dirsHandler));
+  private void verifyLocalFileDeletion(
+      LogAggregationService logAggregationService) throws Exception {
     logAggregationService.init(this.conf);
     logAggregationService.start();
 
-    
     ApplicationId application1 = BuilderUtils.newApplicationId(1234, 1);
 
     // AppLogDir should be created
@@ -252,10 +241,47 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
             ApplicationEventType.APPLICATION_LOG_HANDLING_FINISHED)
     };
 
-    checkEvents(appEventHandler, expectedEvents, true, "getType", "getApplicationID");
+    checkEvents(appEventHandler, expectedEvents, true, "getType",
+        "getApplicationID");
   }
 
   @Test
+  public void testLocalFileDeletionAfterUpload() throws Exception {
+    this.delSrvc = new DeletionService(createContainerExecutor());
+    delSrvc = spy(delSrvc);
+    this.delSrvc.init(conf);
+    this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
+    this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
+        this.remoteRootLogDir.getAbsolutePath());
+
+    LogAggregationService logAggregationService = spy(
+        new LogAggregationService(dispatcher, this.context, this.delSrvc,
+                                  super.dirsHandler));
+    verifyLocalFileDeletion(logAggregationService);
+  }
+
+  @Test
+  public void testLocalFileDeletionOnDiskFull() throws Exception {
+    this.delSrvc = new DeletionService(createContainerExecutor());
+    delSrvc = spy(delSrvc);
+    this.delSrvc.init(conf);
+    this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
+    this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
+        this.remoteRootLogDir.getAbsolutePath());
+    List<String> logDirs = super.dirsHandler.getLogDirs();
+    LocalDirsHandlerService dirsHandler = spy(super.dirsHandler);
+    // Simulate disk being full by returning no good log dirs but having a
+    // directory in full log dirs.
+    when(dirsHandler.getLogDirs()).thenReturn(new ArrayList<String>());
+    when(dirsHandler.getLogDirsForRead()).thenReturn(logDirs);
+    LogAggregationService logAggregationService = spy(
+        new LogAggregationService(dispatcher, this.context, this.delSrvc,
+            dirsHandler));
+    verifyLocalFileDeletion(logAggregationService);
+  }
+
+
+  @Test
   public void testNoContainerOnNode() throws Exception {
     this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
     this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/40b25694/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
index 43100b3..e63f681 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp;
 
 import static org.junit.Assume.assumeTrue;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.verify;
 
@@ -29,6 +30,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -122,6 +124,24 @@ public class TestContainerLogsPage {
     Assert.assertNull(nmContext.getContainers().get(container1));
     files = ContainerLogsUtils.getContainerLogDirs(container1, user, nmContext);
     Assert.assertTrue(!(files.get(0).toString().contains("file:")));
+
+    // Create a new context to check if correct container log dirs are fetched
+    // on full disk.
+    LocalDirsHandlerService dirsHandlerForFullDisk = spy(dirsHandler);
+    // good log dirs are empty and nm log dir is in the full log dir list.
+    when(dirsHandlerForFullDisk.getLogDirs()).
+        thenReturn(new ArrayList<String>());
+    when(dirsHandlerForFullDisk.getLogDirsForRead()).
+        thenReturn(Arrays.asList(new String[] {absLogDir.getAbsolutePath()}));
+    nmContext = new NodeManager.NMContext(null, null, dirsHandlerForFullDisk,
+        new ApplicationACLsManager(conf), new NMNullStateStoreService());
+    nmContext.getApplications().put(appId, app);
+    container.setState(ContainerState.RUNNING);
+    nmContext.getContainers().put(container1, container);
+    List<File> dirs =
+        ContainerLogsUtils.getContainerLogDirs(container1, user, nmContext);
+    File containerLogDir = new File(absLogDir, appId + "/" + container1);
+    Assert.assertTrue(dirs.contains(containerLogDir));
   }
 
   @Test(timeout = 10000)
@@ -231,7 +251,7 @@ public class TestContainerLogsPage {
     LocalDirsHandlerService localDirs = mock(LocalDirsHandlerService.class);
     List<String> logDirs = new ArrayList<String>();
     logDirs.add("F:/nmlogs");
-    when(localDirs.getLogDirs()).thenReturn(logDirs);
+    when(localDirs.getLogDirsForRead()).thenReturn(logDirs);
     
     ApplicationIdPBImpl appId = mock(ApplicationIdPBImpl.class);
     when(appId.toString()).thenReturn("app_id_1");


[29/30] hadoop git commit: YARN-2871. TestRMRestart#testRMRestartGetApplicationList sometime fails in trunk. Contributed by zhihai xu

Posted by aw...@apache.org.
YARN-2871. TestRMRestart#testRMRestartGetApplicationList sometime fails
in trunk. Contributed by zhihai xu


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fe6c1bd7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fe6c1bd7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fe6c1bd7

Branch: refs/heads/HADOOP-12111
Commit: fe6c1bd73aee188ed58df4d33bbc2d2fe0779a97
Parents: aa07dea
Author: Xuan <xg...@apache.org>
Authored: Fri Jun 26 19:43:59 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Fri Jun 26 19:43:59 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                              | 3 +++
 .../hadoop/yarn/server/resourcemanager/TestRMRestart.java    | 8 +++++---
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe6c1bd7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index d74e8a9..e2bf706 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -559,6 +559,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3745. SerializedException should also try to instantiate internal
     exception with the default constructor. (Lavkesh Lahngir via devaraj)
 
+    YARN-2871. TestRMRestart#testRMRestartGetApplicationList sometime fails in trunk.
+    (zhihai xu via xgong)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe6c1bd7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index 628e9be..de17acd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.timeout;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
@@ -976,9 +977,10 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     List<ApplicationReport> appList2 = response2.getApplicationList();
     Assert.assertTrue(3 == appList2.size());
 
-    // check application summary is logged for the completed apps after RM restart.
-    verify(rm2.getRMAppManager(), times(3)).logApplicationSummary(
-      isA(ApplicationId.class));
+    // check application summary is logged for the completed apps with timeout
+    // to make sure APP_COMPLETED events are processed, after RM restart.
+    verify(rm2.getRMAppManager(), timeout(1000).times(3)).
+        logApplicationSummary(isA(ApplicationId.class));
   }
 
   private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)


[03/30] hadoop git commit: HDFS-6440. Support more than 2 NameNodes. Contributed by Jesse Yates.

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java
index c4a2988..62643ae 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAConfiguration.java
@@ -23,10 +23,12 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.net.MalformedURLException;
 import java.net.URI;
 import java.net.URL;
 import java.util.Collection;
 
+import com.google.common.base.Joiner;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -58,19 +60,23 @@ public class TestHAConfiguration {
     }
   }
 
-  private Configuration getHAConf(String nsId, String host1, String host2) {
+  private Configuration getHAConf(String nsId, String ... hosts) {
     Configuration conf = new Configuration();
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, nsId);
-    conf.set(DFSUtil.addKeySuffixes(
-        DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX, nsId),
-        "nn1,nn2");    
     conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, "nn1");
+
+    String[] nnids = new String[hosts.length];
+    for (int i = 0; i < hosts.length; i++) {
+      String nnid = "nn" + (i + 1);
+      nnids[i] = nnid;
+      conf.set(DFSUtil.addKeySuffixes(
+              DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, nnid),
+          hosts[i] + ":12345");
+    }
+
     conf.set(DFSUtil.addKeySuffixes(
-        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, "nn1"),
-        host1 + ":12345");
-    conf.set(DFSUtil.addKeySuffixes(
-        DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY, nsId, "nn2"),
-        host2 + ":12345");
+            DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX, nsId),
+        Joiner.on(',').join(nnids));
     return conf;
   }
 
@@ -87,11 +93,28 @@ public class TestHAConfiguration {
     // 0.0.0.0, it should substitute the address from the RPC configuration
     // above.
     StandbyCheckpointer checkpointer = new StandbyCheckpointer(conf, fsn);
-    assertEquals(new URL("http", "1.2.3.2",
-        DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, ""),
-        checkpointer.getActiveNNAddress());
+    assertAddressMatches("1.2.3.2", checkpointer.getActiveNNAddresses().get(0));
+
+    //test when there are three NNs
+    // Use non-local addresses to avoid host address matching
+    conf = getHAConf("ns1", "1.2.3.1", "1.2.3.2", "1.2.3.3");
+
+    // This is done by the NN before the StandbyCheckpointer is created
+    NameNode.initializeGenericKeys(conf, "ns1", "nn1");
+
+    checkpointer = new StandbyCheckpointer(conf, fsn);
+    assertEquals("Got an unexpected number of possible active NNs", 2, checkpointer
+        .getActiveNNAddresses().size());
+    assertEquals(new URL("http", "1.2.3.2", DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, ""),
+        checkpointer.getActiveNNAddresses().get(0));
+    assertAddressMatches("1.2.3.2", checkpointer.getActiveNNAddresses().get(0));
+    assertAddressMatches("1.2.3.3", checkpointer.getActiveNNAddresses().get(1));
   }
-  
+
+  private void assertAddressMatches(String address, URL url) throws MalformedURLException {
+    assertEquals(new URL("http", address, DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, ""), url);
+  }
+
   /**
    * Tests that the namenode edits dirs and shared edits dirs are gotten with
    * duplicates removed

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
index 76a62ff..3da37f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestPipelinesFailover.java
@@ -24,6 +24,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
+import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
@@ -81,24 +82,33 @@ public class TestPipelinesFailover {
   
   private static final int STRESS_NUM_THREADS = 25;
   private static final int STRESS_RUNTIME = 40000;
-  
+
+  private static final int NN_COUNT = 3;
+  private static final long FAILOVER_SEED = System.currentTimeMillis();
+  private static final Random failoverRandom = new Random(FAILOVER_SEED);
+  static{
+    // log the failover seed so we can reproduce the test exactly
+    LOG.info("Using random seed: " + FAILOVER_SEED
+        + " for selecting active target NN during failover");
+  }
+
   enum TestScenario {
     GRACEFUL_FAILOVER {
       @Override
-      void run(MiniDFSCluster cluster) throws IOException {
-        cluster.transitionToStandby(0);
-        cluster.transitionToActive(1);
+      void run(MiniDFSCluster cluster, int previousActive, int activeIndex) throws IOException {
+        cluster.transitionToStandby(previousActive);
+        cluster.transitionToActive(activeIndex);
       }
     },
     ORIGINAL_ACTIVE_CRASHED {
       @Override
-      void run(MiniDFSCluster cluster) throws IOException {
-        cluster.restartNameNode(0);
-        cluster.transitionToActive(1);
+      void run(MiniDFSCluster cluster, int previousActive, int activeIndex) throws IOException {
+        cluster.restartNameNode(previousActive);
+        cluster.transitionToActive(activeIndex);
       }
     };
 
-    abstract void run(MiniDFSCluster cluster) throws IOException;
+    abstract void run(MiniDFSCluster cluster, int previousActive, int activeIndex) throws IOException;
   }
   
   enum MethodToTestIdempotence {
@@ -135,10 +145,7 @@ public class TestPipelinesFailover {
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1000);
     
     FSDataOutputStream stm = null;
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-      .nnTopology(MiniDFSNNTopology.simpleHATopology())
-      .numDataNodes(3)
-      .build();
+    MiniDFSCluster cluster = newMiniCluster(conf, 3);
     try {
       int sizeWritten = 0;
       
@@ -157,15 +164,15 @@ public class TestPipelinesFailover {
       // Make sure all of the blocks are written out before failover.
       stm.hflush();
 
-      LOG.info("Failing over to NN 1");
-      scenario.run(cluster);
+      LOG.info("Failing over to another NN");
+      int activeIndex = failover(cluster, scenario);
 
       // NOTE: explicitly do *not* make any further metadata calls
       // to the NN here. The next IPC call should be to allocate the next
       // block. Any other call would notice the failover and not test
       // idempotence of the operation (HDFS-3031)
       
-      FSNamesystem ns1 = cluster.getNameNode(1).getNamesystem();
+      FSNamesystem ns1 = cluster.getNameNode(activeIndex).getNamesystem();
       BlockManagerTestUtil.updateState(ns1.getBlockManager());
       assertEquals(0, ns1.getPendingReplicationBlocks());
       assertEquals(0, ns1.getCorruptReplicaBlocks());
@@ -213,10 +220,7 @@ public class TestPipelinesFailover {
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     
     FSDataOutputStream stm = null;
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-      .nnTopology(MiniDFSNNTopology.simpleHATopology())
-      .numDataNodes(5)
-      .build();
+    MiniDFSCluster cluster = newMiniCluster(conf, 5);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
@@ -232,8 +236,7 @@ public class TestPipelinesFailover {
       // Make sure all the blocks are written before failover
       stm.hflush();
 
-      LOG.info("Failing over to NN 1");
-      scenario.run(cluster);
+      int nextActive = failover(cluster, scenario);
 
       assertTrue(fs.exists(TEST_PATH));
       
@@ -242,9 +245,9 @@ public class TestPipelinesFailover {
       // write another block and a half
       AppendTestUtil.write(stm, BLOCK_AND_A_HALF, BLOCK_AND_A_HALF);
       stm.hflush();
-      
-      LOG.info("Failing back to NN 0");
-      cluster.transitionToStandby(1);
+
+      LOG.info("Failing back from NN " + nextActive + " to NN 0");
+      cluster.transitionToStandby(nextActive);
       cluster.transitionToActive(0);
       
       cluster.stopDataNode(1);
@@ -262,7 +265,7 @@ public class TestPipelinesFailover {
       cluster.shutdown();
     }
   }
-  
+
   /**
    * Tests lease recovery if a client crashes. This approximates the
    * use case of HBase WALs being recovered after a NN failover.
@@ -275,10 +278,7 @@ public class TestPipelinesFailover {
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     
     FSDataOutputStream stm = null;
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-      .nnTopology(MiniDFSNNTopology.simpleHATopology())
-      .numDataNodes(3)
-      .build();
+    final MiniDFSCluster cluster = newMiniCluster(conf, 3);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
@@ -329,10 +329,7 @@ public class TestPipelinesFailover {
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
     
     FSDataOutputStream stm = null;
-    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-      .nnTopology(MiniDFSNNTopology.simpleHATopology())
-      .numDataNodes(3)
-      .build();
+    final MiniDFSCluster cluster = newMiniCluster(conf, 3);
     try {
       cluster.waitActive();
       cluster.transitionToActive(0);
@@ -406,7 +403,20 @@ public class TestPipelinesFailover {
       cluster.shutdown();
     }
   }
-  
+
+  /**
+   * Create a MiniCluster with the specified base configuration and the specified number of
+   * DataNodes. Helper method to ensure that the we use the same number of NNs across all the tests.
+   * @return mini cluster ready to use
+   * @throws IOException cluster cannot be started
+   */
+  private MiniDFSCluster newMiniCluster(Configuration conf, int dnCount) throws IOException {
+    return new MiniDFSCluster.Builder(conf)
+             .nnTopology(MiniDFSNNTopology.simpleHATopology(NN_COUNT))
+             .numDataNodes(dnCount)
+             .build();
+  }
+
   /**
    * Stress test for pipeline/lease recovery. Starts a number of
    * threads, each of which creates a file and has another client
@@ -485,6 +495,38 @@ public class TestPipelinesFailover {
   }
 
   /**
+   * Fail-over using the given scenario, assuming NN0 is currently active
+   * @param cluster cluster on which to run the scenario
+   * @param scenario failure scenario to run
+   * @return the index of the new active NN
+   * @throws IOException
+   */
+  private int failover(MiniDFSCluster cluster, TestScenario scenario) throws IOException {
+    return failover(cluster, scenario, 0);
+  }
+
+  /**
+   * Do a fail-over with the given scenario.
+   * @param cluster cluster on which to run the scenario
+   * @param scenario failure scenario to run
+   * @param activeIndex index of the currently active node
+   * @throws IOException on failure
+   * @return the index of the new active NN
+   */
+  private int failover(MiniDFSCluster cluster, TestScenario scenario, int activeIndex)
+      throws IOException {
+    // get index of the next node that should be active, ensuring its not the same as the currently
+    // active node
+    int nextActive = failoverRandom.nextInt(NN_COUNT);
+    if (nextActive == activeIndex) {
+      nextActive = (nextActive + 1) % NN_COUNT;
+    }
+    LOG.info("Failing over to a standby NN:" + nextActive + " from NN " + activeIndex);
+    scenario.run(cluster, activeIndex, nextActive);
+    return nextActive;
+  }
+
+  /**
    * Test thread which creates a file, has another fake user recover
    * the lease on the file, and then ensures that the file's contents
    * are properly readable. If any of these steps fails, propagates

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRemoteNameNodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRemoteNameNodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRemoteNameNodeInfo.java
new file mode 100644
index 0000000..cb2a4fc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRemoteNameNodeInfo.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.hdfs.server.namenode.ha;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test that we correctly obtain remote namenode information
+ */
+public class TestRemoteNameNodeInfo {
+
+  @Test
+  public void testParseMultipleNameNodes() throws Exception {
+    // start with an empty configuration
+    Configuration conf = new Configuration(false);
+
+    // add in keys for each of the NNs
+    String nameservice = "ns1";
+    MiniDFSNNTopology topology = new MiniDFSNNTopology()
+        .addNameservice(new MiniDFSNNTopology.NSConf(nameservice)
+            .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10001))
+            .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10002))
+            .addNN(new MiniDFSNNTopology.NNConf("nn3").setIpcPort(10003)));
+
+    // add the configurations of the NNs to the passed conf, so we can parse it back out
+    MiniDFSCluster.configureNameNodes(topology, false, conf);
+
+    // set the 'local' one as nn1
+    conf.set(DFSConfigKeys.DFS_HA_NAMENODE_ID_KEY, "nn1");
+
+    List<RemoteNameNodeInfo> nns = RemoteNameNodeInfo.getRemoteNameNodes(conf);
+
+    // make sure it matches when we pass in the nameservice
+    List<RemoteNameNodeInfo> nns2 = RemoteNameNodeInfo.getRemoteNameNodes(conf,
+        nameservice);
+    assertEquals(nns, nns2);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestSeveralNameNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestSeveralNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestSeveralNameNodes.java
new file mode 100644
index 0000000..dbe8070
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestSeveralNameNodes.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.namenode.ha;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.test.MultithreadedTestUtil.RepeatingTestThread;
+import org.apache.hadoop.test.MultithreadedTestUtil.TestContext;
+import org.junit.Test;
+
+/**
+ * Test that we can start several and run with namenodes on the same minicluster
+ */
+public class TestSeveralNameNodes {
+
+  private static final Log LOG = LogFactory.getLog(TestSeveralNameNodes.class);
+
+  /** ms between failovers between NNs */
+  private static final int TIME_BETWEEN_FAILOVERS = 200;
+  private static final int NUM_NAMENODES = 3;
+  private static final int NUM_THREADS = 3;
+  private static final int LIST_LENGTH = 50;
+  /** ms for length of test */
+  private static final long RUNTIME = 100000;
+
+  @Test
+  public void testCircularLinkedListWrites() throws Exception {
+    HAStressTestHarness harness = new HAStressTestHarness();
+    // setup the harness
+    harness.setNumberOfNameNodes(NUM_NAMENODES);
+    harness.addFailoverThread(TIME_BETWEEN_FAILOVERS);
+
+    final MiniDFSCluster cluster = harness.startCluster();
+    try {
+      cluster.waitActive();
+      cluster.transitionToActive(0);
+
+      // setup the a circular writer
+      FileSystem fs = harness.getFailoverFs();
+      TestContext context = harness.testCtx;
+      List<CircularWriter> writers = new ArrayList<CircularWriter>();
+      for (int i = 0; i < NUM_THREADS; i++) {
+        Path p = new Path("/test-" + i);
+        fs.mkdirs(p);
+        CircularWriter writer = new CircularWriter(context, LIST_LENGTH, fs, p);
+        writers.add(writer);
+        context.addThread(writer);
+      }
+      harness.startThreads();
+
+      // wait for all the writer threads to finish, or that we exceed the time
+      long start = System.currentTimeMillis();
+      while ((System.currentTimeMillis() - start) < RUNTIME) {
+        for (int i = 0; i < writers.size(); i++) {
+          CircularWriter writer = writers.get(i);
+          // remove the writer from the ones to check
+          if (writer.done.await(10, TimeUnit.MILLISECONDS)) {
+            writers.remove(i--);
+          }
+        }
+      }
+      assertEquals(
+          "Some writers didn't complete in expected runtime! Current writer state:"
+              + writers, 0,
+          writers.size());
+
+      harness.stopThreads();
+    } finally {
+      System.err.println("===========================\n\n\n\n");
+      harness.shutdown();
+    }
+  }
+
+  private static class CircularWriter extends RepeatingTestThread {
+
+    private final int maxLength;
+    private final Path dir;
+    private final FileSystem fs;
+    private int currentListIndex = 0;
+    private CountDownLatch done = new CountDownLatch(1);
+
+    public CircularWriter(TestContext context, int listLength, FileSystem fs,
+        Path parentDir) {
+      super(context);
+      this.fs = fs;
+      this.maxLength = listLength;
+      this.dir = parentDir;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder("Circular Writer:\n");
+      builder.append("\t directory: " + dir + "\n");
+      builder.append("\t target length: " + maxLength + "\n");
+      // might be a little racy, but we just want a close count
+      builder.append("\t current item: " + currentListIndex + "\n");
+      builder.append("\t done: " + (done.getCount() == 0) + "\n");
+      return builder.toString();
+    }
+
+    @Override
+    public void doAnAction() throws Exception {
+      if (currentListIndex == maxLength) {
+        checkList();
+        this.stopTestThread();
+        done.countDown();
+      } else {
+        writeList();
+      }
+    }
+
+    /**
+     * Make sure we can traverse the entire linked list
+     */
+    private void checkList() throws IOException {
+      for (int i = 0; i < maxLength; i++) {
+        Path nextFile = getNextFile(i);
+        if (!fs.exists(nextFile)) {
+          throw new RuntimeException("Next file " + nextFile
+              + " for list does not exist!");
+        }
+        // read the next file name
+        FSDataInputStream in = fs.open(nextFile);
+        nextFile = getNextFile(in.read());
+        in.close();
+      }
+
+    }
+
+    private void cleanup() throws IOException {
+      if (!fs.delete(dir, true)) {
+        throw new RuntimeException("Didn't correctly delete " + dir);
+      }
+      if (!fs.mkdirs(dir)) {
+        throw new RuntimeException("Didn't correctly make directory " + dir);
+      }
+    }
+
+    private void writeList() throws IOException {
+      Path nextPath = getNextFile(currentListIndex++);
+      LOG.info("Writing next file: " + nextPath);
+      FSDataOutputStream file = fs.create(nextPath);
+      file.write(currentListIndex);
+      file.close();
+    }
+
+    private Path getNextFile(int i) {
+      return new Path(dir, Integer.toString(i));
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
index 33af0e2..cd32502 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestStandbyCheckpoints.java
@@ -63,8 +63,9 @@ import static org.junit.Assert.*;
 
 public class TestStandbyCheckpoints {
   private static final int NUM_DIRS_IN_LOG = 200000;
+  protected static int NUM_NNS = 3;
   protected MiniDFSCluster cluster;
-  protected NameNode nn0, nn1;
+  protected NameNode[] nns = new NameNode[NUM_NNS];
   protected FileSystem fs;
   private final Random random = new Random();
   protected File tmpOivImgDir;
@@ -88,7 +89,8 @@ public class TestStandbyCheckpoints {
         MiniDFSNNTopology topology = new MiniDFSNNTopology()
             .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
                 .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(basePort))
-                .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(basePort + 1)));
+                .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(basePort + 1))
+                .addNN(new MiniDFSNNTopology.NNConf("nn3").setHttpPort(basePort + 2)));
 
         cluster = new MiniDFSCluster.Builder(conf)
             .nnTopology(topology)
@@ -96,8 +98,8 @@ public class TestStandbyCheckpoints {
             .build();
         cluster.waitActive();
 
-        nn0 = cluster.getNameNode(0);
-        nn1 = cluster.getNameNode(1);
+        setNNs();
+
         fs = HATestUtil.configureFailoverFs(cluster, conf);
 
         cluster.transitionToActive(0);
@@ -110,6 +112,12 @@ public class TestStandbyCheckpoints {
     }
   }
 
+  protected void setNNs(){
+    for (int i = 0; i < NUM_NNS; i++) {
+      nns[i] = cluster.getNameNode(i);
+    }
+  }
+
   protected Configuration setupCommonConfig() {
     tmpOivImgDir = Files.createTempDir();
 
@@ -136,10 +144,10 @@ public class TestStandbyCheckpoints {
 
   @Test(timeout = 300000)
   public void testSBNCheckpoints() throws Exception {
-    JournalSet standbyJournalSet = NameNodeAdapter.spyOnJournalSet(nn1);
-    
+    JournalSet standbyJournalSet = NameNodeAdapter.spyOnJournalSet(nns[1]);
+
     doEdits(0, 10);
-    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    HATestUtil.waitForStandbyToCatchUp(nns[0], nns[1]);
     // Once the standby catches up, it should notice that it needs to
     // do a checkpoint and save one to its local directories.
     HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
@@ -147,10 +155,9 @@ public class TestStandbyCheckpoints {
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
-        if(tmpOivImgDir.list().length > 0) {
+        if (tmpOivImgDir.list().length > 0) {
           return true;
-        }
-        else {
+        } else {
           return false;
         }
       }
@@ -189,9 +196,9 @@ public class TestStandbyCheckpoints {
     HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(12));
     HATestUtil.waitForCheckpoint(cluster, 0, ImmutableList.of(12));
     
-    assertEquals(12, nn0.getNamesystem().getFSImage()
+    assertEquals(12, nns[0].getNamesystem().getFSImage()
         .getMostRecentCheckpointTxId());
-    assertEquals(12, nn1.getNamesystem().getFSImage()
+    assertEquals(12, nns[1].getNamesystem().getFSImage()
         .getMostRecentCheckpointTxId());
     
     List<File> dirs = Lists.newArrayList();
@@ -214,17 +221,17 @@ public class TestStandbyCheckpoints {
     cluster.getConfiguration(1).setInt(
         DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 0);
     cluster.restartNameNode(1);
-    nn1 = cluster.getNameNode(1);
- 
-    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nn1);
-    
+    nns[1] = cluster.getNameNode(1);
+
+    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nns[1]);
+
     // We shouldn't save any checkpoints at txid=0
     Thread.sleep(1000);
     Mockito.verify(spyImage1, Mockito.never())
       .saveNamespace((FSNamesystem) Mockito.anyObject());
  
     // Roll the primary and wait for the standby to catch up
-    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
+    HATestUtil.waitForStandbyToCatchUp(nns[0], nns[1]);
     Thread.sleep(2000);
     
     // We should make exactly one checkpoint at this new txid. 
@@ -259,7 +266,7 @@ public class TestStandbyCheckpoints {
     cluster.getConfiguration(1).setInt(
         DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 0);
     cluster.restartNameNode(1);
-    nn1 = cluster.getNameNode(1);
+    nns[1] = cluster.getNameNode(1);
 
     cluster.transitionToActive(0);    
     
@@ -284,31 +291,42 @@ public class TestStandbyCheckpoints {
   @Test(timeout=60000)
   public void testCheckpointCancellationDuringUpload() throws Exception {
     // don't compress, we want a big image
-    cluster.getConfiguration(0).setBoolean(
-        DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, false);
-    cluster.getConfiguration(1).setBoolean(
-        DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, false);
+    for (int i = 0; i < NUM_NNS; i++) {
+      cluster.getConfiguration(i).setBoolean(
+          DFSConfigKeys.DFS_IMAGE_COMPRESS_KEY, false);
+    }
+
     // Throttle SBN upload to make it hang during upload to ANN
-    cluster.getConfiguration(1).setLong(
-        DFSConfigKeys.DFS_IMAGE_TRANSFER_RATE_KEY, 100);
-    cluster.restartNameNode(0);
-    cluster.restartNameNode(1);
-    nn0 = cluster.getNameNode(0);
-    nn1 = cluster.getNameNode(1);
+    for (int i = 1; i < NUM_NNS; i++) {
+      cluster.getConfiguration(i).setLong(
+          DFSConfigKeys.DFS_IMAGE_TRANSFER_RATE_KEY, 100);
+    }
+    for (int i = 0; i < NUM_NNS; i++) {
+      cluster.restartNameNode(i);
+    }
+
+    // update references to each of the nns
+    setNNs();
 
     cluster.transitionToActive(0);
 
     doEdits(0, 100);
-    HATestUtil.waitForStandbyToCatchUp(nn0, nn1);
-    HATestUtil.waitForCheckpoint(cluster, 1, ImmutableList.of(104));
+
+    for (int i = 1; i < NUM_NNS; i++) {
+      HATestUtil.waitForStandbyToCatchUp(nns[0], nns[i]);
+      HATestUtil.waitForCheckpoint(cluster, i, ImmutableList.of(104));
+    }
+
     cluster.transitionToStandby(0);
     cluster.transitionToActive(1);
 
+
     // Wait to make sure background TransferFsImageUpload thread was cancelled.
     // This needs to be done before the next test in the suite starts, so that a
     // file descriptor is not held open during the next cluster init.
     cluster.shutdown();
     cluster = null;
+
     GenericTestUtils.waitFor(new Supplier<Boolean>() {
       @Override
       public Boolean get() {
@@ -325,7 +343,7 @@ public class TestStandbyCheckpoints {
     }, 1000, 30000);
 
     // Assert that former active did not accept the canceled checkpoint file.
-    assertEquals(0, nn0.getFSImage().getMostRecentCheckpointTxId());
+    assertEquals(0, nns[0].getFSImage().getMostRecentCheckpointTxId());
   }
   
   /**
@@ -337,7 +355,7 @@ public class TestStandbyCheckpoints {
   public void testStandbyExceptionThrownDuringCheckpoint() throws Exception {
     
     // Set it up so that we know when the SBN checkpoint starts and ends.
-    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nn1);
+    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nns[1]);
     DelayAnswer answerer = new DelayAnswer(LOG);
     Mockito.doAnswer(answerer).when(spyImage1)
         .saveNamespace(Mockito.any(FSNamesystem.class),
@@ -345,7 +363,7 @@ public class TestStandbyCheckpoints {
 
     // Perform some edits and wait for a checkpoint to start on the SBN.
     doEdits(0, 1000);
-    nn0.getRpcServer().rollEditLog();
+    nns[0].getRpcServer().rollEditLog();
     answerer.waitForCall();
     assertTrue("SBN is not performing checkpoint but it should be.",
         answerer.getFireCount() == 1 && answerer.getResultCount() == 0);
@@ -355,7 +373,7 @@ public class TestStandbyCheckpoints {
     ThreadUtil.sleepAtLeastIgnoreInterrupts(1000);
     try {
       // Perform an RPC to the SBN and make sure it throws a StandbyException.
-      nn1.getRpcServer().getFileInfo("/");
+      nns[1].getRpcServer().getFileInfo("/");
       fail("Should have thrown StandbyException, but instead succeeded.");
     } catch (StandbyException se) {
       GenericTestUtils.assertExceptionContains("is not supported", se);
@@ -382,7 +400,7 @@ public class TestStandbyCheckpoints {
   public void testReadsAllowedDuringCheckpoint() throws Exception {
     
     // Set it up so that we know when the SBN checkpoint starts and ends.
-    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nn1);
+    FSImage spyImage1 = NameNodeAdapter.spyOnFsImage(nns[1]);
     DelayAnswer answerer = new DelayAnswer(LOG);
     Mockito.doAnswer(answerer).when(spyImage1)
         .saveNamespace(Mockito.any(FSNamesystem.class),
@@ -391,7 +409,7 @@ public class TestStandbyCheckpoints {
     
     // Perform some edits and wait for a checkpoint to start on the SBN.
     doEdits(0, 1000);
-    nn0.getRpcServer().rollEditLog();
+    nns[0].getRpcServer().rollEditLog();
     answerer.waitForCall();
     assertTrue("SBN is not performing checkpoint but it should be.",
         answerer.getFireCount() == 1 && answerer.getResultCount() == 0);
@@ -405,7 +423,7 @@ public class TestStandbyCheckpoints {
       @Override
       public void run() {
         try {
-          nn1.getRpcServer().restoreFailedStorage("false");
+          nns[1].getRpcServer().restoreFailedStorage("false");
         } catch (IOException e) {
           e.printStackTrace();
         }
@@ -415,16 +433,16 @@ public class TestStandbyCheckpoints {
     
     // Make sure that our thread is waiting for the lock.
     ThreadUtil.sleepAtLeastIgnoreInterrupts(1000);
-    
-    assertFalse(nn1.getNamesystem().getFsLockForTests().hasQueuedThreads());
-    assertFalse(nn1.getNamesystem().getFsLockForTests().isWriteLocked());
-    assertTrue(nn1.getNamesystem().getCpLockForTests().hasQueuedThreads());
-    
+
+    assertFalse(nns[1].getNamesystem().getFsLockForTests().hasQueuedThreads());
+    assertFalse(nns[1].getNamesystem().getFsLockForTests().isWriteLocked());
+    assertTrue(nns[1].getNamesystem().getCpLockForTests().hasQueuedThreads());
+
     // Get /jmx of the standby NN web UI, which will cause the FSNS read lock to
     // be taken.
     String pageContents = DFSTestUtil.urlGet(new URL("http://" +
-        nn1.getHttpAddress().getHostName() + ":" +
-        nn1.getHttpAddress().getPort() + "/jmx"));
+        nns[1].getHttpAddress().getHostName() + ":" +
+        nns[1].getHttpAddress().getPort() + "/jmx"));
     assertTrue(pageContents.contains("NumLiveDataNodes"));
     
     // Make sure that the checkpoint is still going on, implying that the client

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-0.23-reserved.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-0.23-reserved.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-0.23-reserved.tgz
index 0f53f2a..abc7bbd 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-0.23-reserved.tgz and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-0.23-reserved.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1-reserved.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1-reserved.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1-reserved.tgz
index 737ad2d..b3f8b9d 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1-reserved.tgz and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1-reserved.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz
index 3cb2ee6..2256fba 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-22-dfs-dir.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-22-dfs-dir.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-22-dfs-dir.tgz
index b69741c..c4959b4 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-22-dfs-dir.tgz and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-22-dfs-dir.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop1-bbw.tgz
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop1-bbw.tgz b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop1-bbw.tgz
index 2574f8b..e7d3fbd 100644
Binary files a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop1-bbw.tgz and b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop1-bbw.tgz differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
index ef3e249..c671ccc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties
@@ -20,4 +20,4 @@ log4j.rootLogger=info,stdout
 log4j.threshold=ALL
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
-log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n


[23/30] hadoop git commit: HADOOP-11203. Allow ditscp to accept bandwitdh in fraction MegaBytes. Contributed by Raju Bairishetti

Posted by aw...@apache.org.
HADOOP-11203. Allow ditscp to accept bandwitdh in fraction MegaBytes. Contributed by Raju Bairishetti


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8ef07f76
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8ef07f76
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8ef07f76

Branch: refs/heads/HADOOP-12111
Commit: 8ef07f767f0421b006b0fc77e5daf36c7b06abf1
Parents: 1403b84
Author: Amareshwari Sriramadasu <am...@apache.org>
Authored: Fri Jun 26 09:52:06 2015 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Fri Jun 26 09:52:06 2015 +0530

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt    |  3 +++
 .../org/apache/hadoop/tools/DistCpConstants.java   |  2 +-
 .../apache/hadoop/tools/DistCpOptionSwitch.java    |  5 +++--
 .../org/apache/hadoop/tools/DistCpOptions.java     |  6 +++---
 .../org/apache/hadoop/tools/OptionsParser.java     |  2 +-
 .../org/apache/hadoop/tools/mapred/CopyMapper.java | 17 ++++++++++++++++-
 .../tools/mapred/RetriableFileCopyCommand.java     |  2 +-
 .../hadoop/tools/util/ThrottledInputStream.java    |  6 +++---
 .../org/apache/hadoop/tools/TestOptionsParser.java | 16 +++++++++-------
 9 files changed, 40 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index b2975dc..5901794 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -53,6 +53,9 @@ Trunk (Unreleased)
 
   IMPROVEMENTS
 
+    HADOOP-11203. Allow ditscp to accept bandwitdh in fraction MegaBytes
+    (Raju Bairishetti via amareshwari)
+
     HADOOP-8017. Configure hadoop-main pom to get rid of M2E plugin execution
     not covered (Eric Charles via bobby)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
index 21dca62..93d6a62 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpConstants.java
@@ -30,7 +30,7 @@ public class DistCpConstants {
   public static final int DEFAULT_MAPS = 20;
 
   /* Default bandwidth if none specified */
-  public static final int DEFAULT_BANDWIDTH_MB = 100;
+  public static final float DEFAULT_BANDWIDTH_MB = 100;
 
   /* Default strategy for copying. Implementation looked up
      from distcp-default.xml

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
index ed4a0b2..f16a5d2 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptionSwitch.java
@@ -174,10 +174,11 @@ public enum DistCpOptionSwitch {
               "copied to <= n bytes")),
 
   /**
-   * Specify bandwidth per map in MB
+   * Specify bandwidth per map in MB, accepts bandwidth as a fraction
    */
   BANDWIDTH(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
-      new Option("bandwidth", true, "Specify bandwidth per map in MB")),
+      new Option("bandwidth", true, "Specify bandwidth per map in MB,"
+          + " accepts bandwidth as a fraction.")),
 
   /**
    * Path containing a list of strings, which when found in the path of

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
index 302b626..5b4ccf9 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java
@@ -47,7 +47,7 @@ public class DistCpOptions {
   public static final int maxNumListstatusThreads = 40;
   private int numListstatusThreads = 0;  // Indicates that flag is not set.
   private int maxMaps = DistCpConstants.DEFAULT_MAPS;
-  private int mapBandwidth = DistCpConstants.DEFAULT_BANDWIDTH_MB;
+  private float mapBandwidth = DistCpConstants.DEFAULT_BANDWIDTH_MB;
 
   private String sslConfigurationFile;
 
@@ -366,7 +366,7 @@ public class DistCpOptions {
    *
    * @return Bandwidth in MB
    */
-  public int getMapBandwidth() {
+  public float getMapBandwidth() {
     return mapBandwidth;
   }
 
@@ -375,7 +375,7 @@ public class DistCpOptions {
    *
    * @param mapBandwidth - per map bandwidth
    */
-  public void setMapBandwidth(int mapBandwidth) {
+  public void setMapBandwidth(float mapBandwidth) {
     assert mapBandwidth > 0 : "Bandwidth " + mapBandwidth + " is invalid (should be > 0)";
     this.mapBandwidth = mapBandwidth;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
index 37add1e..b414513 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java
@@ -293,7 +293,7 @@ public class OptionsParser {
                                      DistCpOptions option) {
     if (command.hasOption(DistCpOptionSwitch.BANDWIDTH.getSwitch())) {
       try {
-        Integer mapBandwidth = Integer.parseInt(
+        Float mapBandwidth = Float.parseFloat(
             getVal(command, DistCpOptionSwitch.BANDWIDTH.getSwitch()).trim());
         if (mapBandwidth <= 0) {
           throw new IllegalArgumentException("Bandwidth specified is not " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
index cca36df..f75fe76 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/CopyMapper.java
@@ -62,6 +62,8 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     BYTESEXPECTED,// Number of bytes expected to be copied.
     BYTESFAILED,  // Number of bytes that failed to be copied.
     BYTESSKIPPED, // Number of bytes that were skipped from copy.
+    SLEEP_TIME_MS, // Time map slept while trying to honor bandwidth cap.
+    BANDWIDTH_IN_BYTES, // Effective transfer rate in B/s.
   }
 
   /**
@@ -85,7 +87,9 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
   private EnumSet<FileAttribute> preserve = EnumSet.noneOf(FileAttribute.class);
 
   private FileSystem targetFS = null;
-  private Path    targetWorkPath = null;
+  private Path targetWorkPath = null;
+  private long startEpoch;
+  private long totalBytesCopied = 0;
 
   /**
    * Implementation of the Mapper::setup() method. This extracts the DistCp-
@@ -118,6 +122,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     if (conf.get(DistCpConstants.CONF_LABEL_SSL_CONF) != null) {
       initializeSSLConf(context);
     }
+    startEpoch = System.currentTimeMillis();
   }
 
   /**
@@ -288,6 +293,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
     incrementCounter(context, Counter.BYTESEXPECTED, sourceFileStatus.getLen());
     incrementCounter(context, Counter.BYTESCOPIED, bytesCopied);
     incrementCounter(context, Counter.COPY, 1);
+    totalBytesCopied += bytesCopied;
   }
 
   private void createTargetDirsWithRetry(String description,
@@ -373,4 +379,13 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
       return false;
     }
   }
+
+  @Override
+  protected void cleanup(Context context)
+      throws IOException, InterruptedException {
+    super.cleanup(context);
+    long secs = (System.currentTimeMillis() - startEpoch) / 1000;
+    incrementCounter(context, Counter.BANDWIDTH_IN_BYTES,
+        totalBytesCopied / ((secs == 0 ? 1 : secs)));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
index 65d644b..6b5078c 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/mapred/RetriableFileCopyCommand.java
@@ -293,7 +293,7 @@ public class RetriableFileCopyCommand extends RetriableCommand {
       Configuration conf) throws IOException {
     try {
       FileSystem fs = path.getFileSystem(conf);
-      long bandwidthMB = conf.getInt(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
+      float bandwidthMB = conf.getFloat(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
               DistCpConstants.DEFAULT_BANDWIDTH_MB);
       FSDataInputStream in = fs.open(path);
       return new ThrottledInputStream(in, bandwidthMB * 1024 * 1024);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java
index 9e435d9..2be8ef0 100644
--- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java
+++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/util/ThrottledInputStream.java
@@ -39,7 +39,7 @@ import com.google.common.base.Preconditions;
 public class ThrottledInputStream extends InputStream {
 
   private final InputStream rawStream;
-  private final long maxBytesPerSec;
+  private final float maxBytesPerSec;
   private final long startTime = System.currentTimeMillis();
 
   private long bytesRead = 0;
@@ -51,8 +51,8 @@ public class ThrottledInputStream extends InputStream {
     this(rawStream, Long.MAX_VALUE);
   }
 
-  public ThrottledInputStream(InputStream rawStream, long maxBytesPerSec) {
-    assert maxBytesPerSec > 0 : "Bandwidth " + maxBytesPerSec + " is invalid"; 
+  public ThrottledInputStream(InputStream rawStream, float maxBytesPerSec) {
+    assert maxBytesPerSec > 0 : "Bandwidth " + maxBytesPerSec + " is invalid";
     this.rawStream = rawStream;
     this.maxBytesPerSec = maxBytesPerSec;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8ef07f76/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
index b9d9ada..616872b 100644
--- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
+++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestOptionsParser.java
@@ -32,6 +32,8 @@ import java.util.NoSuchElementException;
 
 public class TestOptionsParser {
 
+  private static final float DELTA = 0.001f;
+
   @Test
   public void testParseIgnoreFailure() {
     DistCpOptions options = OptionsParser.parse(new String[] {
@@ -104,14 +106,14 @@ public class TestOptionsParser {
     DistCpOptions options = OptionsParser.parse(new String[] {
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(options.getMapBandwidth(), DistCpConstants.DEFAULT_BANDWIDTH_MB);
+    Assert.assertEquals(options.getMapBandwidth(), DistCpConstants.DEFAULT_BANDWIDTH_MB, DELTA);
 
     options = OptionsParser.parse(new String[] {
         "-bandwidth",
-        "11",
+        "11.2",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
-    Assert.assertEquals(options.getMapBandwidth(), 11);
+    Assert.assertEquals(options.getMapBandwidth(), 11.2, DELTA);
   }
 
   @Test(expected=IllegalArgumentException.class)
@@ -585,8 +587,8 @@ public class TestOptionsParser {
     options.appendToConf(conf);
     Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(), false));
     Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.ATOMIC_COMMIT.getConfigLabel(), false));
-    Assert.assertEquals(conf.getInt(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1),
-        DistCpConstants.DEFAULT_BANDWIDTH_MB);
+    Assert.assertEquals(conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1),
+        DistCpConstants.DEFAULT_BANDWIDTH_MB, DELTA);
 
     conf = new Configuration();
     Assert.assertFalse(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
@@ -597,14 +599,14 @@ public class TestOptionsParser {
         "-delete",
         "-pu",
         "-bandwidth",
-        "11",
+        "11.2",
         "hdfs://localhost:8020/source/first",
         "hdfs://localhost:8020/target/"});
     options.appendToConf(conf);
     Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
     Assert.assertTrue(conf.getBoolean(DistCpOptionSwitch.DELETE_MISSING.getConfigLabel(), false));
     Assert.assertEquals(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel()), "U");
-    Assert.assertEquals(conf.getInt(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), 11);
+    Assert.assertEquals(conf.getFloat(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), 11.2, DELTA);
   }
 
   @Test


[16/30] hadoop git commit: YARN-3826. Race condition in ResourceTrackerService leads to wrong diagnostics messages. Contributed by Chengbing Liu.

Posted by aw...@apache.org.
YARN-3826. Race condition in ResourceTrackerService leads to wrong
diagnostics messages. Contributed by Chengbing Liu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/57f1a01e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/57f1a01e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/57f1a01e

Branch: refs/heads/HADOOP-12111
Commit: 57f1a01eda80f44d3ffcbcb93c4ee290e274946a
Parents: a815cc1
Author: Devaraj K <de...@apache.org>
Authored: Thu Jun 25 16:13:59 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Thu Jun 25 16:13:59 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../server/utils/YarnServerBuilderUtils.java    | 11 ++++++++--
 .../resourcemanager/ResourceTrackerService.java | 23 +++++---------------
 3 files changed, 18 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f1a01e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 9e70ec9..3d608fd 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -553,6 +553,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3790. usedResource from rootQueue metrics may get stale data for FS
     scheduler after recovering the container (Zhihai Xu via rohithsharmaks)
 
+    YARN-3826. Race condition in ResourceTrackerService leads to
+    wrong diagnostics messages. (Chengbing Liu via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f1a01e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerBuilderUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerBuilderUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerBuilderUtils.java
index 8bdff62..f333185 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerBuilderUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerBuilderUtils.java
@@ -22,13 +22,11 @@ import java.util.List;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
-import org.apache.hadoop.yarn.util.Records;
 
 /**
  * Server Builder utilities to construct various objects.
@@ -39,6 +37,15 @@ public class YarnServerBuilderUtils {
   private static final RecordFactory recordFactory = RecordFactoryProvider
       .getRecordFactory(null);
 
+  public static NodeHeartbeatResponse newNodeHeartbeatResponse(
+      NodeAction action, String diagnosticsMessage) {
+    NodeHeartbeatResponse response = recordFactory
+        .newRecordInstance(NodeHeartbeatResponse.class);
+    response.setNodeAction(action);
+    response.setDiagnosticsMessage(diagnosticsMessage);
+    return response;
+  }
+
   public static NodeHeartbeatResponse newNodeHeartbeatResponse(int responseId,
       NodeAction action, List<ContainerId> containersToCleanUp,
       List<ApplicationId> applicationsToCleanUp,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/57f1a01e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index aa37254..3c2c09b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -100,22 +100,11 @@ public class ResourceTrackerService extends AbstractService implements
   private InetSocketAddress resourceTrackerAddress;
   private String minimumNodeManagerVersion;
 
-  private static final NodeHeartbeatResponse resync = recordFactory
-      .newRecordInstance(NodeHeartbeatResponse.class);
-  private static final NodeHeartbeatResponse shutDown = recordFactory
-  .newRecordInstance(NodeHeartbeatResponse.class);
-  
   private int minAllocMb;
   private int minAllocVcores;
 
   private boolean isDistributedNodeLabelsConf;
 
-  static {
-    resync.setNodeAction(NodeAction.RESYNC);
-
-    shutDown.setNodeAction(NodeAction.SHUTDOWN);
-  }
-
   public ResourceTrackerService(RMContext rmContext,
       NodesListManager nodesListManager,
       NMLivelinessMonitor nmLivelinessMonitor,
@@ -414,8 +403,8 @@ public class ResourceTrackerService extends AbstractService implements
           "Disallowed NodeManager nodeId: " + nodeId + " hostname: "
               + nodeId.getHost();
       LOG.info(message);
-      shutDown.setDiagnosticsMessage(message);
-      return shutDown;
+      return YarnServerBuilderUtils.newNodeHeartbeatResponse(
+          NodeAction.SHUTDOWN, message);
     }
 
     // 2. Check if it's a registered node
@@ -424,8 +413,8 @@ public class ResourceTrackerService extends AbstractService implements
       /* node does not exist */
       String message = "Node not found resyncing " + remoteNodeStatus.getNodeId();
       LOG.info(message);
-      resync.setDiagnosticsMessage(message);
-      return resync;
+      return YarnServerBuilderUtils.newNodeHeartbeatResponse(NodeAction.RESYNC,
+          message);
     }
 
     // Send ping
@@ -445,11 +434,11 @@ public class ResourceTrackerService extends AbstractService implements
               + lastNodeHeartbeatResponse.getResponseId() + " nm response id:"
               + remoteNodeStatus.getResponseId();
       LOG.info(message);
-      resync.setDiagnosticsMessage(message);
       // TODO: Just sending reboot is not enough. Think more.
       this.rmContext.getDispatcher().getEventHandler().handle(
           new RMNodeEvent(nodeId, RMNodeEventType.REBOOTING));
-      return resync;
+      return YarnServerBuilderUtils.newNodeHeartbeatResponse(NodeAction.RESYNC,
+          message);
     }
 
     // Heartbeat response


[07/30] hadoop git commit: MAPREDUCE-6400. Multiple shuffle transfer fails because input is closed too early. Contributed by Brahma Reddy Battula, Akira AJISAKA, and Gera Shegalov.

Posted by aw...@apache.org.
MAPREDUCE-6400. Multiple shuffle transfer fails because input is closed too early. Contributed by Brahma Reddy Battula, Akira AJISAKA, and Gera Shegalov.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/72d08a0e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/72d08a0e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/72d08a0e

Branch: refs/heads/HADOOP-12111
Commit: 72d08a0e41efda635baa985d55d67cb059a7c07c
Parents: 2ba6465
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jun 24 15:29:11 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Jun 24 15:29:11 2015 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                 |  4 ++++
 .../apache/hadoop/mapreduce/task/reduce/Fetcher.java |  1 +
 .../mapreduce/task/reduce/IFileWrappedMapOutput.java | 10 ++--------
 .../hadoop/mapreduce/task/reduce/LocalFetcher.java   | 15 +++++----------
 4 files changed, 12 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/72d08a0e/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 5eae44e..6c65032 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -505,6 +505,10 @@ Release 2.8.0 - UNRELEASED
     multibyte record delimiters well (Vinayakumar B, Rushabh Shah, and Akira
     AJISAKA via jlowe)
 
+    MAPREDUCE-6400. Multiple shuffle transfer fails because input is closed
+    too early (Brahma Reddy Battula, Akira AJISAKA, and Gera Shegalov via
+    jlowe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72d08a0e/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
index 1e03387..fb0ac0a 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/Fetcher.java
@@ -335,6 +335,7 @@ class Fetcher<K,V> extends Thread {
         try {
           failedTasks = copyMapOutput(host, input, remaining, fetchRetryEnabled);
         } catch (IOException e) {
+          IOUtils.cleanup(LOG, input);
           //
           // Setup connection again if disconnected by NM
           connection.disconnect();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72d08a0e/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/IFileWrappedMapOutput.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/IFileWrappedMapOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/IFileWrappedMapOutput.java
index 119db15..6051c34 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/IFileWrappedMapOutput.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/IFileWrappedMapOutput.java
@@ -60,13 +60,7 @@ public abstract class IFileWrappedMapOutput<K, V> extends MapOutput<K, V> {
                       long compressedLength, long decompressedLength,
                       ShuffleClientMetrics metrics,
                       Reporter reporter) throws IOException {
-    IFileInputStream iFin =
-        new IFileInputStream(input, compressedLength, conf);
-    try {
-      this.doShuffle(host, iFin, compressedLength,
-                    decompressedLength, metrics, reporter);
-    } finally {
-      iFin.close();
-    }
+    doShuffle(host, new IFileInputStream(input, compressedLength, conf),
+        compressedLength, decompressedLength, metrics, reporter);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/72d08a0e/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/LocalFetcher.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/LocalFetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/LocalFetcher.java
index de2382c..f45742f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/LocalFetcher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/LocalFetcher.java
@@ -30,6 +30,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.mapred.IndexRecord;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MapOutputFile;
@@ -149,19 +150,13 @@ class LocalFetcher<K,V> extends Fetcher<K, V> {
     // now read the file, seek to the appropriate section, and send it.
     FileSystem localFs = FileSystem.getLocal(job).getRaw();
     FSDataInputStream inStream = localFs.open(mapOutputFileName);
-
-    inStream = CryptoUtils.wrapIfNecessary(job, inStream);
-
     try {
+      inStream = CryptoUtils.wrapIfNecessary(job, inStream);
       inStream.seek(ir.startOffset + CryptoUtils.cryptoPadding(job));
-      mapOutput.shuffle(LOCALHOST, inStream, compressedLength, decompressedLength, metrics, reporter);
+      mapOutput.shuffle(LOCALHOST, inStream, compressedLength,
+          decompressedLength, metrics, reporter);
     } finally {
-      try {
-        inStream.close();
-      } catch (IOException ioe) {
-        LOG.warn("IOException closing inputstream from map output: "
-            + ioe.toString());
-      }
+      IOUtils.cleanup(LOG, inStream);
     }
 
     scheduler.copySucceeded(mapTaskId, LOCALHOST, compressedLength, 0, 0,


[28/30] hadoop git commit: HADOOP-12036. Consolidate all of the cmake extensions in one directory (alanburlison via cmccabe)

Posted by aw...@apache.org.
HADOOP-12036. Consolidate all of the cmake extensions in one directory (alanburlison via cmccabe)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/aa07dea3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/aa07dea3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/aa07dea3

Branch: refs/heads/HADOOP-12111
Commit: aa07dea3577158b92a17651d10da20df73f54561
Parents: 60b858b
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Jun 26 12:32:31 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Jun 26 12:32:31 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../hadoop-common/HadoopCommon.cmake            | 207 +++++++++++
 .../hadoop-common/HadoopJNI.cmake               |  97 +++++
 .../hadoop-common/src/CMakeLists.txt            | 366 ++++++++-----------
 4 files changed, 457 insertions(+), 216 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa07dea3/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 5901794..92e1bfa 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -679,6 +679,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11885. hadoop-dist dist-layout-stitching.sh does not work with dash.
     (wang)
 
+    HADOOP-12036. Consolidate all of the cmake extensions in one directory
+    (alanburlison via cmccabe)
+
   BUG FIXES
 
     HADOOP-11802: DomainSocketWatcher thread terminates sometimes after there

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa07dea3/hadoop-common-project/hadoop-common/HadoopCommon.cmake
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/HadoopCommon.cmake b/hadoop-common-project/hadoop-common/HadoopCommon.cmake
new file mode 100644
index 0000000..5a83f3d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/HadoopCommon.cmake
@@ -0,0 +1,207 @@
+#
+# 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.
+#
+
+#
+# Common CMake utilities and configuration, shared by all Native components.
+#
+
+#
+# Platform-specific prerequisite checks.
+#
+
+if(CMAKE_SYSTEM_NAME STREQUAL "SunOS")
+    # Only 64-bit Java is supported.
+    if(NOT JVM_ARCH_DATA_MODEL EQUAL 64)
+        message(FATAL_ERROR "Unrecognised JVM_ARCH_DATA_MODEL '${JVM_ARCH_DATA_MODEL}'. "
+          "A 64-bit JVM must be used on Solaris, make sure that one is installed and, "
+          "if necessary, the MAVEN_OPTS environment variable includes '-d64'")
+    endif()
+
+    # Only gcc is suported for now.
+    if(NOT(CMAKE_COMPILER_IS_GNUCC AND CMAKE_COMPILER_IS_GNUCXX))
+        message(FATAL_ERROR "Only gcc is supported on Solaris")
+    endif()
+endif()
+
+#
+# Helper functions and macros.
+#
+
+# Add flags to all the CMake compiler variables
+macro(hadoop_add_compiler_flags FLAGS)
+    set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAGS}")
+    set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${FLAGS}")
+endmacro()
+
+# Add flags to all the CMake linker variables
+macro(hadoop_add_linker_flags FLAGS)
+    set(CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${FLAGS}")
+    set(CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} ${FLAGS}")
+    set(CMAKE_STATIC_LINKER_FLAGS "${CMAKE_STATIC_LINKER_FLAGS} ${FLAGS}")
+endmacro()
+
+# Compile a library with both shared and static variants.
+function(hadoop_add_dual_library LIBNAME)
+    add_library(${LIBNAME} SHARED ${ARGN})
+    add_library(${LIBNAME}_static STATIC ${ARGN})
+    set_target_properties(${LIBNAME}_static PROPERTIES OUTPUT_NAME ${LIBNAME})
+endfunction()
+
+# Link both a static and a dynamic target against some libraries.
+function(hadoop_target_link_dual_libraries LIBNAME)
+    target_link_libraries(${LIBNAME} ${ARGN})
+    target_link_libraries(${LIBNAME}_static ${ARGN})
+endfunction()
+
+# Set all the output directories to the same place.
+function(hadoop_output_directory TGT DIR)
+    set_target_properties(${TGT} PROPERTIES RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
+    set_target_properties(${TGT} PROPERTIES ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
+    set_target_properties(${TGT} PROPERTIES LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
+endfunction()
+
+# Set the target directories for dynamic and static builds.
+function(hadoop_dual_output_directory TGT DIR)
+    hadoop_output_directory(${TGT} "${DIR}")
+    hadoop_output_directory(${TGT}_static "${DIR}")
+endfunction()
+
+# Alter the behavior of find_package and find_library so that we find only
+# shared libraries with a given version suffix.  You should save
+# CMAKE_FIND_LIBRARY_SUFFIXES before calling this function and restore it
+# afterwards.  On Windows this function is a no-op.  Windows does not encode
+# version number information information into library path names.
+macro(hadoop_set_find_shared_library_version LVERS)
+    if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+        # Mac OS uses .dylib
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".${LVERS}.dylib")
+    elseif(${CMAKE_SYSTEM_NAME} MATCHES "FreeBSD")
+        # FreeBSD has always .so installed.
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
+    elseif(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
+        # Windows doesn't support finding shared libraries by version.
+    else()
+        # Most UNIX variants use .so
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so.${LVERS}")
+    endif()
+endmacro()
+
+# Alter the behavior of find_package and find_library so that we find only
+# shared libraries without any version suffix.  You should save
+# CMAKE_FIND_LIBRARY_SUFFIXES before calling this function and restore it
+# afterwards. On Windows this function is a no-op.  Windows does not encode
+# version number information information into library path names.
+macro(hadoop_set_find_shared_library_without_version)
+    if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
+        # Mac OS uses .dylib
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".dylib")
+    elseif(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
+        # No effect
+    else()
+        # Most UNIX variants use .so
+        set(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
+    endif()
+endmacro()
+
+#
+# Configuration.
+#
+
+# Initialise the shared gcc/g++ flags if they aren't already defined.
+if(NOT DEFINED GCC_SHARED_FLAGS)
+    set(GCC_SHARED_FLAGS "-g -O2 -Wall -pthread -D_FILE_OFFSET_BITS=64")
+endif()
+
+# Add in support other compilers here, if necessary,
+# the assumption is that GCC or a GCC-compatible compiler is being used.
+
+# Set the shared GCC-compatible compiler and linker flags.
+hadoop_add_compiler_flags("${GCC_SHARED_FLAGS}")
+hadoop_add_linker_flags("${LINKER_SHARED_FLAGS}")
+
+#
+# Linux-specific configuration.
+#
+if(CMAKE_SYSTEM_NAME STREQUAL "Linux")
+    # Make GNU extensions available.
+    hadoop_add_compiler_flags("-D_GNU_SOURCE")
+
+    # If JVM_ARCH_DATA_MODEL is 32, compile all binaries as 32-bit.
+    if(JVM_ARCH_DATA_MODEL EQUAL 32)
+        # Force 32-bit code generation on amd64/x86_64, ppc64, sparc64
+        if(CMAKE_COMPILER_IS_GNUCC AND CMAKE_SYSTEM_PROCESSOR MATCHES ".*64")
+            hadoop_add_compiler_flags("-m32")
+            hadoop_add_linker_flags("-m32")
+        endif()
+        # Set CMAKE_SYSTEM_PROCESSOR to ensure that find_package(JNI) will use 32-bit libraries
+        if(CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
+            set(CMAKE_SYSTEM_PROCESSOR "i686")
+        endif()
+    endif()
+
+    # Determine float ABI of JVM on ARM.
+    if(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm")
+        find_program(READELF readelf)
+        if(READELF MATCHES "NOTFOUND")
+            message(WARNING "readelf not found; JVM float ABI detection disabled")
+        else(READELF MATCHES "NOTFOUND")
+            execute_process(
+                COMMAND ${READELF} -A ${JAVA_JVM_LIBRARY}
+                OUTPUT_VARIABLE JVM_ELF_ARCH
+                ERROR_QUIET)
+            if(NOT JVM_ELF_ARCH MATCHES "Tag_ABI_VFP_args: VFP registers")
+                # Test compilation with -mfloat-abi=softfp using an arbitrary libc function
+                # (typically fails with "fatal error: bits/predefs.h: No such file or directory"
+                # if soft-float dev libraries are not installed)
+                message("Soft-float JVM detected")
+                include(CMakePushCheckState)
+                cmake_push_check_state()
+                set(CMAKE_REQUIRED_FLAGS "${CMAKE_REQUIRED_FLAGS} -mfloat-abi=softfp")
+                include(CheckSymbolExists)
+                check_symbol_exists(exit stdlib.h SOFTFP_AVAILABLE)
+                if(NOT SOFTFP_AVAILABLE)
+                    message(FATAL_ERROR "Soft-float dev libraries required (e.g. 'apt-get install libc6-dev-armel' on Debian/Ubuntu)")
+                endif()
+                cmake_pop_check_state()
+                hadoop_add_compiler_flags("-mfloat-abi=softfp")
+            endif()
+        endif()
+    endif()
+
+#
+# Solaris-specific configuration.
+#
+elseif(CMAKE_SYSTEM_NAME STREQUAL "SunOS")
+    # Solaris flags. 64-bit compilation is mandatory, and is checked earlier.
+    hadoop_add_compiler_flags("-m64 -D__EXTENSIONS__ -D_POSIX_PTHREAD_SEMANTICS -D_XOPEN_SOURCE=500")
+    hadoop_add_linker_flags("-m64")
+
+    # CMAKE_SYSTEM_PROCESSOR is set to the output of 'uname -p', which on Solaris is
+    # the 'lowest' ISA supported, i.e. 'i386' or 'sparc'. However in order for the
+    # standard CMake modules to look in the right places it needs to reflect the required
+    # compilation mode, i.e. 64 bit. We therefore force it to either 'amd64' or 'sparcv9'.
+    if(CMAKE_SYSTEM_PROCESSOR STREQUAL "i386")
+        set(CMAKE_SYSTEM_PROCESSOR "amd64")
+        set(CMAKE_LIBRARY_ARCHITECTURE "amd64")
+    elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "sparc")
+        set(CMAKE_SYSTEM_PROCESSOR STREQUAL "sparcv9")
+        set(CMAKE_LIBRARY_ARCHITECTURE "sparcv9")
+    else()
+        message(FATAL_ERROR "Unrecognised CMAKE_SYSTEM_PROCESSOR ${CMAKE_SYSTEM_PROCESSOR}")
+    endif()
+endif()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa07dea3/hadoop-common-project/hadoop-common/HadoopJNI.cmake
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/HadoopJNI.cmake b/hadoop-common-project/hadoop-common/HadoopJNI.cmake
new file mode 100644
index 0000000..78d7ffd
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/HadoopJNI.cmake
@@ -0,0 +1,97 @@
+#
+# 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.
+#
+
+#
+# Common JNI detection for CMake, shared by all Native components.
+#
+
+# Check the JVM_ARCH_DATA_MODEL variable as been set to 32 or 64 by maven.
+if(NOT DEFINED JVM_ARCH_DATA_MODEL)
+    message(FATAL_ERROR "JVM_ARCH_DATA_MODEL is not defined")
+elseif(NOT (JVM_ARCH_DATA_MODEL EQUAL 32 OR JVM_ARCH_DATA_MODEL EQUAL 64))
+    message(FATAL_ERROR "JVM_ARCH_DATA_MODEL is not 32 or 64")
+endif()
+
+#
+# Linux-specific JNI configuration.
+#
+if(CMAKE_SYSTEM_NAME STREQUAL "Linux")
+    # Locate JNI_INCLUDE_DIRS and JNI_LIBRARIES.
+    # Since we were invoked from Maven, we know that the JAVA_HOME environment
+    # variable is valid.  So we ignore system paths here and just use JAVA_HOME.
+    file(TO_CMAKE_PATH "$ENV{JAVA_HOME}" _java_home)
+    if(CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$")
+        set(_java_libarch "i386")
+    elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
+        set(_java_libarch "amd64")
+    elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^arm")
+        set(_java_libarch "arm")
+    elseif(CMAKE_SYSTEM_PROCESSOR MATCHES "^(powerpc|ppc)64le")
+        if(EXISTS "${_java_home}/jre/lib/ppc64le")
+            set(_java_libarch "ppc64le")
+        else()
+            set(_java_libarch "ppc64")
+        endif()
+    else()
+        set(_java_libarch ${CMAKE_SYSTEM_PROCESSOR})
+    endif()
+    set(_JDK_DIRS "${_java_home}/jre/lib/${_java_libarch}/*"
+                  "${_java_home}/jre/lib/${_java_libarch}"
+                  "${_java_home}/jre/lib/*"
+                  "${_java_home}/jre/lib"
+                  "${_java_home}/lib/*"
+                  "${_java_home}/lib"
+                  "${_java_home}/include/*"
+                  "${_java_home}/include"
+                  "${_java_home}"
+    )
+    find_path(JAVA_INCLUDE_PATH
+        NAMES jni.h
+        PATHS ${_JDK_DIRS}
+        NO_DEFAULT_PATH)
+    #In IBM java, it's jniport.h instead of jni_md.h
+    find_path(JAVA_INCLUDE_PATH2
+        NAMES jni_md.h jniport.h
+        PATHS ${_JDK_DIRS}
+        NO_DEFAULT_PATH)
+    set(JNI_INCLUDE_DIRS ${JAVA_INCLUDE_PATH} ${JAVA_INCLUDE_PATH2})
+    find_library(JAVA_JVM_LIBRARY
+        NAMES jvm JavaVM
+        PATHS ${_JDK_DIRS}
+        NO_DEFAULT_PATH)
+    set(JNI_LIBRARIES ${JAVA_JVM_LIBRARY})
+    unset(_java_libarch)
+    unset(_java_home)
+
+    message("JAVA_HOME=${JAVA_HOME}, JAVA_JVM_LIBRARY=${JAVA_JVM_LIBRARY}")
+    message("JAVA_INCLUDE_PATH=${JAVA_INCLUDE_PATH}, JAVA_INCLUDE_PATH2=${JAVA_INCLUDE_PATH2}")
+    if(JAVA_JVM_LIBRARY AND JAVA_INCLUDE_PATH AND JAVA_INCLUDE_PATH2)
+        message("Located all JNI components successfully.")
+    else()
+        message(FATAL_ERROR "Failed to find a viable JVM installation under JAVA_HOME.")
+    endif()
+
+    # Use the standard FindJNI module to locate the JNI components.
+    find_package(JNI REQUIRED)
+
+#
+# Otherwise, use the standard FindJNI module to locate the JNI components.
+#
+else()
+    find_package(JNI REQUIRED)
+endif()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa07dea3/hadoop-common-project/hadoop-common/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/CMakeLists.txt b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
index 7d68fd7..c93bfe7 100644
--- a/hadoop-common-project/hadoop-common/src/CMakeLists.txt
+++ b/hadoop-common-project/hadoop-common/src/CMakeLists.txt
@@ -16,209 +16,149 @@
 # limitations under the License.
 #
 
-cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
-
-# Default to release builds
-set(CMAKE_BUILD_TYPE, Release)
-
-include(JNIFlags.cmake NO_POLICY_SCOPE)
-
-# Compile a library with both shared and static variants
-function(add_dual_library LIBNAME)
-    add_library(${LIBNAME} SHARED ${ARGN})
-    add_library(${LIBNAME}_static STATIC ${ARGN})
-    set_target_properties(${LIBNAME}_static PROPERTIES OUTPUT_NAME ${LIBNAME})
-endfunction(add_dual_library)
+#
+# CMake configuration.
+#
 
-# Link both a static and a dynamic target against some libraries
-function(target_link_dual_libraries LIBNAME)
-    target_link_libraries(${LIBNAME} ${ARGN})
-    target_link_libraries(${LIBNAME}_static ${ARGN})
-endfunction(target_link_dual_libraries)
+cmake_minimum_required(VERSION 2.6 FATAL_ERROR)
 
-function(output_directory TGT DIR)
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        RUNTIME_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        ARCHIVE_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-    SET_TARGET_PROPERTIES(${TGT} PROPERTIES
-        LIBRARY_OUTPUT_DIRECTORY "${CMAKE_BINARY_DIR}/${DIR}")
-endfunction(output_directory TGT DIR)
+list(APPEND CMAKE_MODULE_PATH ${CMAKE_SOURCE_DIR}/..)
+include(HadoopCommon)
 
-function(dual_output_directory TGT DIR)
-    output_directory(${TGT} "${DIR}")
-    output_directory(${TGT}_static "${DIR}")
-endfunction(dual_output_directory TGT DIR)
+# Source and test locations.
+set(SRC main/native/src/org/apache/hadoop)
+set(TST main/native/src/test/org/apache/hadoop)
 
 #
-# This macro alters the behavior of find_package and find_library.
-# It does this by setting the CMAKE_FIND_LIBRARY_SUFFIXES global variable. 
-# You should save that variable before calling this function and restore it
-# after you have accomplished your goal.
+# Main configuration.
 #
-# The behavior is altered in two ways:
-# 1. We always find shared libraries, never static;
-# 2. We find shared libraries with the given version number.
-#
-# On Windows this function is a no-op.  Windows does not encode
-# version number information information into library path names.
-#
-macro(set_find_shared_library_version LVERS)
-    IF(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
-        # Mac OS uses .dylib
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".${LVERS}.dylib")
-    ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "FreeBSD")
-        # FreeBSD has always .so installed.
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
-    ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
-        # Windows doesn't support finding shared libraries by version.
-    ELSE()
-        # Most UNIX variants use .so
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so.${LVERS}")
-    ENDIF()
-endmacro(set_find_shared_library_version LVERS)
 
-#
-# Alter the behavior of find_package and find_library so that we find only
-# shared libraries without any version suffix.  You should save
-# CMAKE_FIND_LIBRARY_SUFFIXES before calling this function and restore it
-# afterwards.
-#
-macro(set_find_shared_library_without_version)
-    IF(${CMAKE_SYSTEM_NAME} MATCHES "Darwin")
-        # Mac OS uses .dylib
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".dylib")
-    ELSEIF(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
-        # No effect
-    ELSE()
-        # Most UNIX variants use .so
-        SET(CMAKE_FIND_LIBRARY_SUFFIXES ".so")
-    ENDIF()
-endmacro(set_find_shared_library_without_version)
+# The caller must specify where the generated headers have been placed.
+if(NOT GENERATED_JAVAH)
+    message(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
+endif()
 
-if (NOT GENERATED_JAVAH)
-    # Must identify where the generated headers have been placed
-    MESSAGE(FATAL_ERROR "You must set the cmake variable GENERATED_JAVAH")
-endif (NOT GENERATED_JAVAH)
-find_package(JNI REQUIRED)
+# Configure JNI.
+include(HadoopJNI)
 
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
-set_find_shared_library_version("1")
+# Require zlib.
+set(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
+hadoop_set_find_shared_library_version("1")
 find_package(ZLIB REQUIRED)
-SET(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
-
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_GNU_SOURCE")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
-set(D main/native/src/org/apache/hadoop)
-set(T main/native/src/test/org/apache/hadoop)
-
-GET_FILENAME_COMPONENT(HADOOP_ZLIB_LIBRARY ${ZLIB_LIBRARIES} NAME)
+set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
+get_filename_component(HADOOP_ZLIB_LIBRARY ${ZLIB_LIBRARIES} NAME)
 
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
-set_find_shared_library_version("1")
+# Look for bzip2.
+set(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
+hadoop_set_find_shared_library_version("1")
 find_package(BZip2 QUIET)
-if (BZIP2_INCLUDE_DIR AND BZIP2_LIBRARIES)
-    GET_FILENAME_COMPONENT(HADOOP_BZIP2_LIBRARY ${BZIP2_LIBRARIES} NAME)
+if(BZIP2_INCLUDE_DIR AND BZIP2_LIBRARIES)
+    get_filename_component(HADOOP_BZIP2_LIBRARY ${BZIP2_LIBRARIES} NAME)
     set(BZIP2_SOURCE_FILES
-          "${D}/io/compress/bzip2/Bzip2Compressor.c"
-          "${D}/io/compress/bzip2/Bzip2Decompressor.c")
-else (BZIP2_INCLUDE_DIR AND BZIP2_LIBRARIES)
+          "${SRC}/io/compress/bzip2/Bzip2Compressor.c"
+          "${SRC}/io/compress/bzip2/Bzip2Decompressor.c")
+    set(REQUIRE_BZIP2 ${REQUIRE_BZIP2}) # Stop warning about unused variable.
+else()
     set(BZIP2_SOURCE_FILES "")
     set(BZIP2_INCLUDE_DIR "")
-    IF(REQUIRE_BZIP2)
-        MESSAGE(FATAL_ERROR "Required bzip2 library and/or header files could not be found.")
-    ENDIF(REQUIRE_BZIP2)
-endif (BZIP2_INCLUDE_DIR AND BZIP2_LIBRARIES)
-SET(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
-
-INCLUDE(CheckFunctionExists)
-INCLUDE(CheckCSourceCompiles)
-INCLUDE(CheckLibraryExists)
-CHECK_FUNCTION_EXISTS(sync_file_range HAVE_SYNC_FILE_RANGE)
-CHECK_FUNCTION_EXISTS(posix_fadvise HAVE_POSIX_FADVISE)
-CHECK_LIBRARY_EXISTS(dl dlopen "" NEED_LINK_DL)
-
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
-set_find_shared_library_version("1")
-find_library(SNAPPY_LIBRARY 
+    if(REQUIRE_BZIP2)
+        message(FATAL_ERROR "Required bzip2 library and/or header files could not be found.")
+    endif()
+endif()
+set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
+
+# Require snappy.
+set(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
+hadoop_set_find_shared_library_version("1")
+find_library(SNAPPY_LIBRARY
     NAMES snappy
     PATHS ${CUSTOM_SNAPPY_PREFIX} ${CUSTOM_SNAPPY_PREFIX}/lib
           ${CUSTOM_SNAPPY_PREFIX}/lib64 ${CUSTOM_SNAPPY_LIB})
-SET(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
-find_path(SNAPPY_INCLUDE_DIR 
+set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
+find_path(SNAPPY_INCLUDE_DIR
     NAMES snappy.h
     PATHS ${CUSTOM_SNAPPY_PREFIX} ${CUSTOM_SNAPPY_PREFIX}/include
           ${CUSTOM_SNAPPY_INCLUDE})
-if (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
-    GET_FILENAME_COMPONENT(HADOOP_SNAPPY_LIBRARY ${SNAPPY_LIBRARY} NAME)
+if(SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
+    get_filename_component(HADOOP_SNAPPY_LIBRARY ${SNAPPY_LIBRARY} NAME)
     set(SNAPPY_SOURCE_FILES
-        "${D}/io/compress/snappy/SnappyCompressor.c"
-        "${D}/io/compress/snappy/SnappyDecompressor.c")
-else (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
+        "${SRC}/io/compress/snappy/SnappyCompressor.c"
+        "${SRC}/io/compress/snappy/SnappyDecompressor.c")
+    set(REQUIRE_SNAPPY ${REQUIRE_SNAPPY}) # Stop warning about unused variable.
+    message(STATUS "Found Snappy: ${SNAPPY_LIBRARY}")
+else()
     set(SNAPPY_INCLUDE_DIR "")
     set(SNAPPY_SOURCE_FILES "")
-    IF(REQUIRE_SNAPPY)
-        MESSAGE(FATAL_ERROR "Required snappy library could not be found.  SNAPPY_LIBRARY=${SNAPPY_LIBRARY}, SNAPPY_INCLUDE_DIR=${SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_INCLUDE_DIR=${CUSTOM_SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_PREFIX=${CUSTOM_SNAPPY_PREFIX}, CUSTOM_SNAPPY_INCLUDE=${CUSTOM_SNAPPY_INCLUDE}")
-    ENDIF(REQUIRE_SNAPPY)
-endif (SNAPPY_LIBRARY AND SNAPPY_INCLUDE_DIR)
-
-IF (CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
-  set(BULK_CRC_ARCH_SOURCE_FIlE "${D}/util/bulk_crc32_x86.c")
-ELSEIF (CMAKE_SYSTEM_PROCESSOR STREQUAL "aarch64")
-  set(BULK_CRC_ARCH_SOURCE_FIlE "${D}/util/bulk_crc32_aarch64.c")
-ELSE()
-  MESSAGE("No HW CRC acceleration for ${CMAKE_SYSTEM_PROCESSOR}, falling back to SW")
-ENDIF()
-
-# Find the no-suffix version of libcrypto.
-# See HADOOP-11216 for details.
-SET(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
-set_find_shared_library_without_version()
-SET(OPENSSL_NAME "crypto")
-IF(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
+    if(REQUIRE_SNAPPY)
+        message(FATAL_ERROR "Required snappy library could not be found.  SNAPPY_LIBRARY=${SNAPPY_LIBRARY}, SNAPPY_INCLUDE_DIR=${SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_INCLUDE_DIR=${CUSTOM_SNAPPY_INCLUDE_DIR}, CUSTOM_SNAPPY_PREFIX=${CUSTOM_SNAPPY_PREFIX}, CUSTOM_SNAPPY_INCLUDE=${CUSTOM_SNAPPY_INCLUDE}")
+    endif()
+endif()
+
+# Build hardware CRC32 acceleration, if supported on the platform.
+if(CMAKE_SYSTEM_PROCESSOR MATCHES "^i.86$" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "x86_64" OR CMAKE_SYSTEM_PROCESSOR STREQUAL "amd64")
+  set(BULK_CRC_ARCH_SOURCE_FIlE "${SRC}/util/bulk_crc32_x86.c")
+elseif(CMAKE_SYSTEM_PROCESSOR STREQUAL "aarch64")
+  set(BULK_CRC_ARCH_SOURCE_FIlE "${SRC}/util/bulk_crc32_aarch64.c")
+else()
+  message("No HW CRC acceleration for ${CMAKE_SYSTEM_PROCESSOR}, falling back to SW")
+endif()
+
+# Find the no-suffix version of libcrypto/openssl. See HADOOP-11216 for details.
+set(STORED_CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES})
+hadoop_set_find_shared_library_without_version()
+set(OPENSSL_NAME "crypto")
+if(${CMAKE_SYSTEM_NAME} MATCHES "Windows")
     SET(OPENSSL_NAME "eay32")
-ENDIF()
-MESSAGE("CUSTOM_OPENSSL_PREFIX = ${CUSTOM_OPENSSL_PREFIX}")
+endif()
+message("CUSTOM_OPENSSL_PREFIX = ${CUSTOM_OPENSSL_PREFIX}")
 find_library(OPENSSL_LIBRARY
     NAMES ${OPENSSL_NAME}
     PATHS ${CUSTOM_OPENSSL_PREFIX} ${CUSTOM_OPENSSL_PREFIX}/lib
           ${CUSTOM_OPENSSL_PREFIX}/lib64 ${CUSTOM_OPENSSL_LIB} NO_DEFAULT_PATH)
 find_library(OPENSSL_LIBRARY NAMES ${OPENSSL_NAME})
-find_path(OPENSSL_INCLUDE_DIR 
+find_path(OPENSSL_INCLUDE_DIR
     NAMES openssl/evp.h
     PATHS ${CUSTOM_OPENSSL_PREFIX} ${CUSTOM_OPENSSL_PREFIX}/include
           ${CUSTOM_OPENSSL_INCLUDE} NO_DEFAULT_PATH)
 find_path(OPENSSL_INCLUDE_DIR NAMES openssl/evp.h)
-SET(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
-SET(USABLE_OPENSSL 0)
-if (OPENSSL_LIBRARY AND OPENSSL_INCLUDE_DIR)
-    INCLUDE(CheckCSourceCompiles)
-    SET(OLD_CMAKE_REQUIRED_INCLUDES ${CMAKE_REQUIRED_INCLUDES})
-    SET(CMAKE_REQUIRED_INCLUDES ${OPENSSL_INCLUDE_DIR})
-    CHECK_C_SOURCE_COMPILES("#include \"${OPENSSL_INCLUDE_DIR}/openssl/evp.h\"\nint main(int argc, char **argv) { return !EVP_aes_256_ctr; }" HAS_NEW_ENOUGH_OPENSSL)
-    SET(CMAKE_REQUIRED_INCLUDES ${OLD_CMAKE_REQUIRED_INCLUDES})
+set(CMAKE_FIND_LIBRARY_SUFFIXES ${STORED_CMAKE_FIND_LIBRARY_SUFFIXES})
+set(USABLE_OPENSSL 0)
+if(OPENSSL_LIBRARY AND OPENSSL_INCLUDE_DIR)
+    include(CheckCSourceCompiles)
+    set(OLD_CMAKE_REQUIRED_INCLUDES ${CMAKE_REQUIRED_INCLUDES})
+    set(CMAKE_REQUIRED_INCLUDES ${OPENSSL_INCLUDE_DIR})
+    check_c_source_compiles("#include \"${OPENSSL_INCLUDE_DIR}/openssl/evp.h\"\nint main(int argc, char **argv) { return !EVP_aes_256_ctr; }" HAS_NEW_ENOUGH_OPENSSL)
+    set(CMAKE_REQUIRED_INCLUDES ${OLD_CMAKE_REQUIRED_INCLUDES})
     if(NOT HAS_NEW_ENOUGH_OPENSSL)
-        MESSAGE("The OpenSSL library installed at ${OPENSSL_LIBRARY} is too old.  You need a version at least new enough to have EVP_aes_256_ctr.")
-    else(NOT HAS_NEW_ENOUGH_OPENSSL)
+        message("The OpenSSL library installed at ${OPENSSL_LIBRARY} is too old.  You need a version at least new enough to have EVP_aes_256_ctr.")
+    else()
         SET(USABLE_OPENSSL 1)
-    endif(NOT HAS_NEW_ENOUGH_OPENSSL)
-endif (OPENSSL_LIBRARY AND OPENSSL_INCLUDE_DIR)
-if (USABLE_OPENSSL)
-    GET_FILENAME_COMPONENT(HADOOP_OPENSSL_LIBRARY ${OPENSSL_LIBRARY} NAME)
-    SET(OPENSSL_SOURCE_FILES
-        "${D}/crypto/OpensslCipher.c"
-        "${D}/crypto/random/OpensslSecureRandom.c")
-else (USABLE_OPENSSL)
-    MESSAGE("Cannot find a usable OpenSSL library.  OPENSSL_LIBRARY=${OPENSSL_LIBRARY}, OPENSSL_INCLUDE_DIR=${OPENSSL_INCLUDE_DIR}, CUSTOM_OPENSSL_LIB=${CUSTOM_OPENSSL_LIB}, CUSTOM_OPENSSL_PREFIX=${CUSTOM_OPENSSL_PREFIX}, CUSTOM_OPENSSL_INCLUDE=${CUSTOM_OPENSSL_INCLUDE}")
-    IF(REQUIRE_OPENSSL)
-        MESSAGE(FATAL_ERROR "Terminating build because require.openssl was specified.")
-    ENDIF(REQUIRE_OPENSSL)
-    SET(OPENSSL_LIBRARY "")
-    SET(OPENSSL_INCLUDE_DIR "")
-    SET(OPENSSL_SOURCE_FILES "")
-endif (USABLE_OPENSSL)
-
+    endif()
+endif()
+if(USABLE_OPENSSL)
+    get_filename_component(HADOOP_OPENSSL_LIBRARY ${OPENSSL_LIBRARY} NAME)
+    set(OPENSSL_SOURCE_FILES
+        "${SRC}/crypto/OpensslCipher.c"
+        "${SRC}/crypto/random/OpensslSecureRandom.c")
+    set(REQUIRE_OPENSSL ${REQUIRE_OPENSSL}) # Stop warning about unused variable.
+else()
+    message("Cannot find a usable OpenSSL library. OPENSSL_LIBRARY=${OPENSSL_LIBRARY}, OPENSSL_INCLUDE_DIR=${OPENSSL_INCLUDE_DIR}, CUSTOM_OPENSSL_LIB=${CUSTOM_OPENSSL_LIB}, CUSTOM_OPENSSL_PREFIX=${CUSTOM_OPENSSL_PREFIX}, CUSTOM_OPENSSL_INCLUDE=${CUSTOM_OPENSSL_INCLUDE}")
+    if(REQUIRE_OPENSSL)
+        message(FATAL_ERROR "Terminating build because require.openssl was specified.")
+    endif()
+    set(OPENSSL_LIBRARY "")
+    set(OPENSSL_INCLUDE_DIR "")
+    set(OPENSSL_SOURCE_FILES "")
+endif()
+
+# Check for platform-specific functions and libraries.
+include(CheckFunctionExists)
+include(CheckLibraryExists)
+check_function_exists(sync_file_range HAVE_SYNC_FILE_RANGE)
+check_function_exists(posix_fadvise HAVE_POSIX_FADVISE)
+check_library_exists(dl dlopen "" NEED_LINK_DL)
+
+# Configure the build.
 include_directories(
     ${GENERATED_JAVAH}
     main/native/src
@@ -230,66 +170,60 @@ include_directories(
     ${BZIP2_INCLUDE_DIR}
     ${SNAPPY_INCLUDE_DIR}
     ${OPENSSL_INCLUDE_DIR}
-    ${D}/util
-)
-CONFIGURE_FILE(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
-
-add_executable(test_bulk_crc32
-    ${D}/util/bulk_crc32.c
-    ${BULK_CRC_ARCH_SOURCE_FIlE}
-    ${T}/util/test_bulk_crc32.c
+    ${SRC}/util
 )
+configure_file(${CMAKE_SOURCE_DIR}/config.h.cmake ${CMAKE_BINARY_DIR}/config.h)
 
-SET(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
-add_dual_library(hadoop
+set(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
+hadoop_add_dual_library(hadoop
     main/native/src/exception.c
-    ${D}/io/compress/lz4/Lz4Compressor.c
-    ${D}/io/compress/lz4/Lz4Decompressor.c
-    ${D}/io/compress/lz4/lz4.c
-    ${D}/io/compress/lz4/lz4hc.c
+    ${SRC}/io/compress/lz4/Lz4Compressor.c
+    ${SRC}/io/compress/lz4/Lz4Decompressor.c
+    ${SRC}/io/compress/lz4/lz4.c
+    ${SRC}/io/compress/lz4/lz4hc.c
     ${SNAPPY_SOURCE_FILES}
     ${OPENSSL_SOURCE_FILES}
-    ${D}/io/compress/zlib/ZlibCompressor.c
-    ${D}/io/compress/zlib/ZlibDecompressor.c
+    ${SRC}/io/compress/zlib/ZlibCompressor.c
+    ${SRC}/io/compress/zlib/ZlibDecompressor.c
     ${BZIP2_SOURCE_FILES}
-    ${D}/io/nativeio/NativeIO.c
-    ${D}/io/nativeio/errno_enum.c
-    ${D}/io/nativeio/file_descriptor.c
-    ${D}/io/nativeio/SharedFileDescriptorFactory.c
-    ${D}/net/unix/DomainSocket.c
-    ${D}/net/unix/DomainSocketWatcher.c
-    ${D}/security/JniBasedUnixGroupsMapping.c
-    ${D}/security/JniBasedUnixGroupsNetgroupMapping.c
-    ${D}/security/hadoop_group_info.c
-    ${D}/security/hadoop_user_info.c
-    ${D}/util/NativeCodeLoader.c
-    ${D}/util/NativeCrc32.c
-    ${D}/util/bulk_crc32.c
+    ${SRC}/io/nativeio/NativeIO.c
+    ${SRC}/io/nativeio/errno_enum.c
+    ${SRC}/io/nativeio/file_descriptor.c
+    ${SRC}/io/nativeio/SharedFileDescriptorFactory.c
+    ${SRC}/net/unix/DomainSocket.c
+    ${SRC}/net/unix/DomainSocketWatcher.c
+    ${SRC}/security/JniBasedUnixGroupsMapping.c
+    ${SRC}/security/JniBasedUnixGroupsNetgroupMapping.c
+    ${SRC}/security/hadoop_group_info.c
+    ${SRC}/security/hadoop_user_info.c
+    ${SRC}/util/NativeCodeLoader.c
+    ${SRC}/util/NativeCrc32.c
+    ${SRC}/util/bulk_crc32.c
     ${BULK_CRC_ARCH_SOURCE_FIlE}
 )
-if (NEED_LINK_DL)
+if(NEED_LINK_DL)
    set(LIB_DL dl)
-endif (NEED_LINK_DL)
+endif()
 
-IF (${CMAKE_SYSTEM_NAME} MATCHES "Linux")
-    #
-    # By embedding '$ORIGIN' into the RPATH of libhadoop.so,
-    # dlopen will look in the directory containing libhadoop.so.
-    # However, $ORIGIN is not supported by all operating systems.
-    #
+hadoop_target_link_dual_libraries(hadoop ${LIB_DL} ${JAVA_JVM_LIBRARY})
+set(LIBHADOOP_VERSION "1.0.0")
+set_target_properties(hadoop PROPERTIES SOVERSION ${LIBHADOOP_VERSION})
+hadoop_dual_output_directory(hadoop target/usr/local/lib)
+
+# By embedding '$ORIGIN' into the RPATH of libhadoop.so, dlopen will look in
+# the directory containing libhadoop.so. However, $ORIGIN is not supported by
+# all operating systems.
+if(${CMAKE_SYSTEM_NAME} MATCHES "Linux|SunOS")
     set(RPATH "\$ORIGIN/")
-    if (EXTRA_LIBHADOOP_RPATH)
+    if(EXTRA_LIBHADOOP_RPATH)
         set(RPATH "${RPATH}:${EXTRA_LIBHADOOP_RPATH}/")
-    endif(EXTRA_LIBHADOOP_RPATH)
-    SET_TARGET_PROPERTIES(hadoop 
-        PROPERTIES INSTALL_RPATH "${RPATH}")
-ENDIF()
+    endif()
+    set_target_properties(hadoop PROPERTIES INSTALL_RPATH "${RPATH}")
+endif()
 
-target_link_dual_libraries(hadoop
-    ${LIB_DL}
-    ${JAVA_JVM_LIBRARY}
+# Build the CRC32 test executable.
+add_executable(test_bulk_crc32
+    ${SRC}/util/bulk_crc32.c
+    ${BULK_CRC_ARCH_SOURCE_FIlE}
+    ${TST}/util/test_bulk_crc32.c
 )
-SET(LIBHADOOP_VERSION "1.0.0")
-SET_TARGET_PROPERTIES(hadoop PROPERTIES
-    SOVERSION ${LIBHADOOP_VERSION})
-dual_output_directory(hadoop target/usr/local/lib)


[21/30] hadoop git commit: HDFS-8665. Fix replication check in DFSTestUtils#waitForReplication.

Posted by aw...@apache.org.
HDFS-8665. Fix replication check in DFSTestUtils#waitForReplication.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ff0e5e57
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ff0e5e57
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ff0e5e57

Branch: refs/heads/HADOOP-12111
Commit: ff0e5e572f5dcf7b49381cbe901360f6e171d423
Parents: aa5b15b
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Jun 25 17:29:24 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Jun 25 17:29:24 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                 | 2 ++
 .../src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java   | 9 ++++++++-
 2 files changed, 10 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff0e5e57/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 01bb92d..dcedb9f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -672,6 +672,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8640. Make reserved RBW space visible through JMX. (kanaka kumar
     avvaru via Arpit Agarwal)
 
+    HDFS-8665. Fix replication check in DFSTestUtils#waitForReplication. (wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ff0e5e57/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 96fb669..89ee674 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
@@ -535,7 +535,14 @@ public class DFSTestUtil {
       public Boolean get() {
         try {
           FileStatus stat = dfs.getFileStatus(file);
-          return replication == stat.getReplication();
+          BlockLocation[] locs = dfs.getFileBlockLocations(stat, 0, stat
+              .getLen());
+          for (BlockLocation loc : locs) {
+            if (replication != loc.getHosts().length) {
+              return false;
+            }
+          }
+          return true;
         } catch (IOException e) {
           LOG.info("getFileStatus on path " + file + " failed!", e);
           return false;


[14/30] hadoop git commit: HDFS-8646. Prune cached replicas from DatanodeDescriptor state on replica invalidation.

Posted by aw...@apache.org.
HDFS-8646. Prune cached replicas from DatanodeDescriptor state on replica invalidation.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/afe9ea3c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/afe9ea3c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/afe9ea3c

Branch: refs/heads/HADOOP-12111
Commit: afe9ea3c12e1f5a71922400eadb642960bc87ca1
Parents: 4c659dd
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Jun 24 14:42:33 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Wed Jun 24 14:42:33 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../server/blockmanagement/BlockManager.java    | 14 +++++++++++
 .../hdfs/server/datanode/BPServiceActor.java    |  6 +++--
 .../hadoop/hdfs/server/datanode/DataNode.java   | 17 +++++++++++--
 .../hdfs/server/namenode/CacheManager.java      | 24 ++++++++++++++++---
 .../hdfs/server/namenode/FSNamesystem.java      |  1 +
 .../hadoop/hdfs/server/namenode/Namesystem.java | 18 +++++++-------
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     | 18 ++++++++++++++
 .../hdfs/server/datanode/DataNodeTestUtils.java | 11 +++++++++
 .../server/namenode/TestCacheDirectives.java    | 25 ++++++++++++++++++++
 10 files changed, 122 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/afe9ea3c/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index d94a213..4268154 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -949,6 +949,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8542. WebHDFS getHomeDirectory behavior does not match specification.
     (Kanaka Kumar Avvaru via jghoman)
 
+    HDFS-8546. Prune cached replicas from DatanodeDescriptor state on replica
+    invalidation. (wang)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afe9ea3c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 7d3a678..368d3b0 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
@@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBloc
 import org.apache.hadoop.hdfs.server.blockmanagement.PendingDataNodeMessages.ReportedBlockInfo;
 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.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
@@ -3108,6 +3109,19 @@ public class BlockManager {
         return;
       }
 
+      CachedBlock cblock = namesystem.getCacheManager().getCachedBlocks()
+          .get(new CachedBlock(block.getBlockId(), (short) 0, false));
+      if (cblock != null) {
+        boolean removed = false;
+        removed |= node.getPendingCached().remove(cblock);
+        removed |= node.getCached().remove(cblock);
+        removed |= node.getPendingUncached().remove(cblock);
+        if (removed) {
+          blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
+              + "related lists on node {}", block, node);
+        }
+      }
+
       //
       // It's possible that the block was removed because of a datanode
       // failure. If the block is still valid, check if replication is

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afe9ea3c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index f84dd99..1817427 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -709,8 +709,10 @@ class BPServiceActor implements Runnable {
         }
         processCommand(cmds == null ? null : cmds.toArray(new DatanodeCommand[cmds.size()]));
 
-        DatanodeCommand cmd = cacheReport();
-        processCommand(new DatanodeCommand[]{ cmd });
+        if (!dn.areCacheReportsDisabledForTests()) {
+          DatanodeCommand cmd = cacheReport();
+          processCommand(new DatanodeCommand[]{ cmd });
+        }
 
         //
         // There is no work to do;  sleep until hearbeat timer elapses, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afe9ea3c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 6c8cf2b..e265dad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -301,6 +301,7 @@ public class DataNode extends ReconfigurableBase
   ThreadGroup threadGroup = null;
   private DNConf dnConf;
   private volatile boolean heartbeatsDisabledForTests = false;
+  private volatile boolean cacheReportsDisabledForTests = false;
   private DataStorage storage = null;
 
   private DatanodeHttpServer httpServer = null;
@@ -1055,15 +1056,27 @@ public class DataNode extends ReconfigurableBase
 
   
   // used only for testing
+  @VisibleForTesting
   void setHeartbeatsDisabledForTests(
       boolean heartbeatsDisabledForTests) {
     this.heartbeatsDisabledForTests = heartbeatsDisabledForTests;
   }
-  
+
+  @VisibleForTesting
   boolean areHeartbeatsDisabledForTests() {
     return this.heartbeatsDisabledForTests;
   }
-  
+
+  @VisibleForTesting
+  void setCacheReportsDisabledForTest(boolean disabled) {
+    this.cacheReportsDisabledForTests = disabled;
+  }
+
+  @VisibleForTesting
+  boolean areCacheReportsDisabledForTests() {
+    return this.cacheReportsDisabledForTests;
+  }
+
   /**
    * This method starts the data node with the specified conf.
    * 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afe9ea3c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
index e5270ad..e09ba32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CacheManager.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
@@ -902,9 +903,26 @@ public final class CacheManager {
     if (cachedBlock == null) {
       return;
     }
-    List<DatanodeDescriptor> datanodes = cachedBlock.getDatanodes(Type.CACHED);
-    for (DatanodeDescriptor datanode : datanodes) {
-      block.addCachedLoc(datanode);
+    List<DatanodeDescriptor> cachedDNs = cachedBlock.getDatanodes(Type.CACHED);
+    for (DatanodeDescriptor datanode : cachedDNs) {
+      // Filter out cached blocks that do not have a backing replica.
+      //
+      // This should not happen since it means the CacheManager thinks
+      // something is cached that does not exist, but it's a safety
+      // measure.
+      boolean found = false;
+      for (DatanodeInfo loc : block.getLocations()) {
+        if (loc.equals(datanode)) {
+          block.addCachedLoc(loc);
+          found = true;
+          break;
+        }
+      }
+      if (!found) {
+        LOG.warn("Datanode {} is not a valid cache location for block {} "
+            + "because that node does not have a backing replica!",
+            datanode, block.getBlock().getBlockName());
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afe9ea3c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d82da93..b073a89 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
@@ -6460,6 +6460,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     this.dir = dir;
   }
   /** @return the cache manager. */
+  @Override
   public CacheManager getCacheManager() {
     return cacheManager;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afe9ea3c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
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 40c4765..1732865 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
@@ -29,21 +29,23 @@ import org.apache.hadoop.security.AccessControlException;
 @InterfaceAudience.Private
 public interface Namesystem extends RwLock, SafeMode {
   /** Is this name system running? */
-  public boolean isRunning();
+  boolean isRunning();
 
   /** Check if the user has superuser privilege. */
-  public void checkSuperuserPrivilege() throws AccessControlException;
+  void checkSuperuserPrivilege() throws AccessControlException;
 
   /** @return the block pool ID */
-  public String getBlockPoolId();
+  String getBlockPoolId();
 
-  public boolean isInStandbyState();
+  boolean isInStandbyState();
 
-  public boolean isGenStampInFuture(Block block);
+  boolean isGenStampInFuture(Block block);
 
-  public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
+  void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
 
-  public void checkOperation(OperationCategory read) throws StandbyException;
+  void checkOperation(OperationCategory read) throws StandbyException;
 
-  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC);
+  boolean isInSnapshot(BlockInfoUnderConstruction blockUC);
+
+  CacheManager getCacheManager();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afe9ea3c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index d06b024..96fb669 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
@@ -79,6 +79,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.FsShell;
@@ -526,6 +527,23 @@ public class DFSTestUtil {
     }
   }
 
+  public static void waitForReplication(final DistributedFileSystem dfs,
+      final Path file, final short replication, int waitForMillis)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        try {
+          FileStatus stat = dfs.getFileStatus(file);
+          return replication == stat.getReplication();
+        } catch (IOException e) {
+          LOG.info("getFileStatus on path " + file + " failed!", e);
+          return false;
+        }
+      }
+    }, 100, waitForMillis);
+  }
+
   /**
    * Keep accessing the given file until the namenode reports that the
    * given block in the file contains the given number of corrupt replicas.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afe9ea3c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
index 9dee724..2f9a3e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
@@ -23,6 +23,7 @@ import java.io.File;
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -53,6 +54,16 @@ public class DataNodeTestUtils {
     dn.setHeartbeatsDisabledForTests(heartbeatsDisabledForTests);
   }
 
+  /**
+   * Set if cache reports are disabled for all DNs in a mini cluster.
+   */
+  public static void setCacheReportsDisabledForTests(MiniDFSCluster cluster,
+      boolean disabled) {
+    for (DataNode dn : cluster.getDataNodes()) {
+      dn.setCacheReportsDisabledForTest(disabled);
+    }
+  }
+
   public static void triggerDeletionReport(DataNode dn) throws IOException {
     for (BPOfferService bpos : dn.getAllBpOs()) {
       bpos.triggerDeletionReportForTests();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/afe9ea3c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
index 6027934..cf00405 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
@@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.io.nativeio.NativeIO.POSIX.CacheManipulator;
@@ -1510,4 +1511,28 @@ public class TestCacheDirectives {
     Thread.sleep(1000);
     checkPendingCachedEmpty(cluster);
   }
+
+  @Test(timeout=60000)
+  public void testNoBackingReplica() throws Exception {
+    // Cache all three replicas for a file.
+    final Path filename = new Path("/noback");
+    final short replication = (short) 3;
+    DFSTestUtil.createFile(dfs, filename, 1, replication, 0x0BAC);
+    dfs.addCachePool(new CachePoolInfo("pool"));
+    dfs.addCacheDirective(
+        new CacheDirectiveInfo.Builder().setPool("pool").setPath(filename)
+            .setReplication(replication).build());
+    waitForCachedBlocks(namenode, 1, replication, "testNoBackingReplica:1");
+    // Pause cache reports while we change the replication factor.
+    // This will orphan some cached replicas.
+    DataNodeTestUtils.setCacheReportsDisabledForTests(cluster, true);
+    try {
+      dfs.setReplication(filename, (short) 1);
+      DFSTestUtil.waitForReplication(dfs, filename, (short) 1, 30000);
+      // The cache locations should drop down to 1 even without cache reports.
+      waitForCachedBlocks(namenode, 1, (short) 1, "testNoBackingReplica:2");
+    } finally {
+      DataNodeTestUtils.setCacheReportsDisabledForTests(cluster, false);
+    }
+  }
 }


[12/30] hadoop git commit: HADOOP-11958. MetricsSystemImpl fails to show backtrace when an error occurs (Jason Lowe via jeagles)

Posted by aw...@apache.org.
HADOOP-11958. MetricsSystemImpl fails to show backtrace when an error occurs (Jason Lowe via jeagles)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2236b577
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2236b577
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2236b577

Branch: refs/heads/HADOOP-12111
Commit: 2236b577a34b069c0d1f91da99f63a199f260ac2
Parents: 2df00d5
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Wed Jun 24 14:51:04 2015 -0500
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Wed Jun 24 14:51:04 2015 -0500

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                    | 2 ++
 .../java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java    | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2236b577/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index e1d9ca9..b2975dc 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -654,6 +654,8 @@ Release 2.8.0 - UNRELEASED
     HADOOP-7139. Allow appending to existing SequenceFiles
     (kanaka kumar avvaru via vinayakumarb)
 
+    HADOOP-11958. MetricsSystemImpl fails to show backtrace when an error
+    occurs (Jason Lowe via jeagles)
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2236b577/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
index a94d814..a1d258d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/impl/MetricsSystemImpl.java
@@ -367,7 +367,7 @@ public class MetricsSystemImpl extends MetricsSystem implements MetricsSource {
             try {
               onTimerEvent();
             } catch (Exception e) {
-              LOG.warn(e);
+              LOG.warn("Error invoking metrics timer", e);
             }
           }
         }, millis, millis);


[22/30] hadoop git commit: HDFS-8546. Use try with resources in DataStorage and Storage.

Posted by aw...@apache.org.
HDFS-8546. Use try with resources in DataStorage and Storage.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1403b84b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1403b84b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1403b84b

Branch: refs/heads/HADOOP-12111
Commit: 1403b84b122fb76ef2b085a728b5402c32499c1f
Parents: ff0e5e5
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Jun 25 17:50:32 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Thu Jun 25 17:50:32 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 +
 .../hadoop/hdfs/server/common/Storage.java      | 24 ++++-------
 .../hdfs/server/datanode/DataStorage.java       | 45 ++++++--------------
 3 files changed, 25 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1403b84b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index dcedb9f..7b97f41 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -674,6 +674,8 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8665. Fix replication check in DFSTestUtils#waitForReplication. (wang)
 
+    HDFS-8546. Use try with resources in DataStorage and Storage. (wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1403b84b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
index e6f0999..c630206 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
@@ -709,6 +709,7 @@ public abstract class Storage extends StorageInfo {
       try {
         res = file.getChannel().tryLock();
         if (null == res) {
+          LOG.error("Unable to acquire file lock on path " + lockF.toString());
           throw new OverlappingFileLockException();
         }
         file.write(jvmName.getBytes(Charsets.UTF_8));
@@ -972,35 +973,28 @@ public abstract class Storage extends StorageInfo {
   public void writeProperties(File to, StorageDirectory sd) throws IOException {
     Properties props = new Properties();
     setPropertiesFromFields(props, sd);
-    writeProperties(to, sd, props);
+    writeProperties(to, props);
   }
 
-  public static void writeProperties(File to, StorageDirectory sd,
-      Properties props) throws IOException {
-    RandomAccessFile file = new RandomAccessFile(to, "rws");
-    FileOutputStream out = null;
-    try {
+  public static void writeProperties(File to, Properties props)
+      throws IOException {
+    try (RandomAccessFile file = new RandomAccessFile(to, "rws");
+        FileOutputStream out = new FileOutputStream(file.getFD())) {
       file.seek(0);
-      out = new FileOutputStream(file.getFD());
       /*
-       * If server is interrupted before this line, 
+       * If server is interrupted before this line,
        * the version file will remain unchanged.
        */
       props.store(out, null);
       /*
-       * Now the new fields are flushed to the head of the file, but file 
-       * length can still be larger then required and therefore the file can 
+       * Now the new fields are flushed to the head of the file, but file
+       * length can still be larger then required and therefore the file can
        * contain whole or corrupted fields from its old contents in the end.
        * If server is interrupted here and restarted later these extra fields
        * either should not effect server behavior or should be handled
        * by the server correctly.
        */
       file.setLength(out.getChannel().position());
-    } finally {
-      if (out != null) {
-        out.close();
-      }
-      file.close();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1403b84b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
index 0bd08dd..76789f9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
@@ -44,17 +44,15 @@ import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DiskChecker;
 
 import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -82,7 +80,6 @@ import java.util.concurrent.Future;
 public class DataStorage extends Storage {
 
   public final static String BLOCK_SUBDIR_PREFIX = "subdir";
-  final static String COPY_FILE_PREFIX = "dncp_";
   final static String STORAGE_DIR_DETACHED = "detach";
   public final static String STORAGE_DIR_RBW = "rbw";
   public final static String STORAGE_DIR_FINALIZED = "finalized";
@@ -614,20 +611,22 @@ public class DataStorage extends Storage {
   @Override
   public boolean isPreUpgradableLayout(StorageDirectory sd) throws IOException {
     File oldF = new File(sd.getRoot(), "storage");
-    if (!oldF.exists())
+    if (!oldF.exists()) {
       return false;
+    }
     // check the layout version inside the storage file
     // Lock and Read old storage file
-    RandomAccessFile oldFile = new RandomAccessFile(oldF, "rws");
-    FileLock oldLock = oldFile.getChannel().tryLock();
-    try {
+    try (RandomAccessFile oldFile = new RandomAccessFile(oldF, "rws");
+      FileLock oldLock = oldFile.getChannel().tryLock()) {
+      if (null == oldLock) {
+        LOG.error("Unable to acquire file lock on path " + oldF.toString());
+        throw new OverlappingFileLockException();
+      }
       oldFile.seek(0);
       int oldVersion = oldFile.readInt();
-      if (oldVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION)
+      if (oldVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION) {
         return false;
-    } finally {
-      oldLock.release();
-      oldFile.close();
+      }
     }
     return true;
   }
@@ -1218,23 +1217,8 @@ public class DataStorage extends Storage {
       return;
     }
     if (!from.isDirectory()) {
-      if (from.getName().startsWith(COPY_FILE_PREFIX)) {
-        FileInputStream in = new FileInputStream(from);
-        try {
-          FileOutputStream out = new FileOutputStream(to);
-          try {
-            IOUtils.copyBytes(in, out, 16*1024);
-            hl.linkStats.countPhysicalFileCopies++;
-          } finally {
-            out.close();
-          }
-        } finally {
-          in.close();
-        }
-      } else {
-        HardLink.createHardLink(from, to);
-        hl.linkStats.countSingleLinks++;
-      }
+      HardLink.createHardLink(from, to);
+      hl.linkStats.countSingleLinks++;
       return;
     }
     // from is a directory
@@ -1285,8 +1269,7 @@ public class DataStorage extends Storage {
     String[] otherNames = from.list(new java.io.FilenameFilter() {
         @Override
         public boolean accept(File dir, String name) {
-          return name.startsWith(BLOCK_SUBDIR_PREFIX) 
-            || name.startsWith(COPY_FILE_PREFIX);
+          return name.startsWith(BLOCK_SUBDIR_PREFIX);
         }
       });
     for(int i = 0; i < otherNames.length; i++)


[26/30] hadoop git commit: HDFS-8623. Refactor NameNode handling of invalid, corrupt, and under-recovery blocks. Contributed by Zhe Zhang.

Posted by aw...@apache.org.
HDFS-8623. Refactor NameNode handling of invalid, corrupt, and under-recovery blocks. Contributed by Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/de480d6c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/de480d6c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/de480d6c

Branch: refs/heads/HADOOP-12111
Commit: de480d6c8945bd8b5b00e8657b7a72ce8dd9b6b5
Parents: 1b764a0
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Jun 26 10:49:01 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Fri Jun 26 10:49:01 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/blockmanagement/BlockInfo.java  |  12 +-
 .../blockmanagement/BlockInfoContiguous.java    |   2 +-
 .../BlockInfoUnderConstructionContiguous.java   |   2 +-
 .../server/blockmanagement/BlockManager.java    | 595 ++++++++++---------
 .../blockmanagement/DatanodeStorageInfo.java    |  15 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  18 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   2 +-
 .../server/blockmanagement/TestBlockInfo.java   |   2 +-
 .../blockmanagement/TestBlockManager.java       |   4 +-
 .../server/blockmanagement/TestNodeCount.java   |   2 +-
 .../TestOverReplicatedBlocks.java               |   4 +-
 .../blockmanagement/TestReplicationPolicy.java  |   2 +-
 13 files changed, 361 insertions(+), 302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 27e2e89..bb1b3ff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -679,6 +679,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8651. Make hadoop-hdfs-project Native code -Wall-clean (Alan Burlison
     via Colin P. McCabe)
 
+    HDFS-8623. Refactor NameNode handling of invalid, corrupt, and under-recovery
+    blocks. (Zhe Zhang via jing9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
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 4cc2791..5ad992b 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
@@ -172,19 +172,23 @@ public abstract class  BlockInfo extends Block
   public abstract int numNodes();
 
   /**
-   * Add a {@link DatanodeStorageInfo} location for a block.
+   * 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);
+  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
+   * DatanodeStorageInfo's linked list.
    */
   abstract void replaceBlock(BlockInfo newBlock);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
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 b9abcd0..de64ad8 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
@@ -45,7 +45,7 @@ public class BlockInfoContiguous extends BlockInfo {
   }
 
   @Override
-  boolean addStorage(DatanodeStorageInfo storage) {
+  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
     return ContiguousBlockStorageOp.addStorage(this, storage);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
index c66675a..d3cb337 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
@@ -69,7 +69,7 @@ public class BlockInfoUnderConstructionContiguous extends
   }
 
   @Override
-  boolean addStorage(DatanodeStorageInfo storage) {
+  boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) {
     return ContiguousBlockStorageOp.addStorage(this, storage);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 368d3b0..5bd4980 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
@@ -198,8 +198,8 @@ public class BlockManager {
    * Maps a StorageID to the set of blocks that are "extra" for this
    * DataNode. We'll eventually remove these extras.
    */
-  public final Map<String, LightWeightLinkedSet<Block>> excessReplicateMap =
-    new TreeMap<String, LightWeightLinkedSet<Block>>();
+  public final Map<String, LightWeightLinkedSet<BlockInfo>> excessReplicateMap =
+    new TreeMap<>();
 
   /**
    * Store set of Blocks that need to be replicated 1 or more times.
@@ -502,8 +502,8 @@ public class BlockManager {
   /** Dump meta data to out. */
   public void metaSave(PrintWriter out) {
     assert namesystem.hasWriteLock();
-    final List<DatanodeDescriptor> live = new ArrayList<DatanodeDescriptor>();
-    final List<DatanodeDescriptor> dead = new ArrayList<DatanodeDescriptor>();
+    final List<DatanodeDescriptor> live = new ArrayList<>();
+    final List<DatanodeDescriptor> dead = new ArrayList<>();
     datanodeManager.fetchDatanodes(live, dead, false);
     out.println("Live Datanodes: " + live.size());
     out.println("Dead Datanodes: " + dead.size());
@@ -542,8 +542,8 @@ public class BlockManager {
     List<DatanodeDescriptor> containingNodes =
                                       new ArrayList<DatanodeDescriptor>();
     List<DatanodeStorageInfo> containingLiveReplicasNodes =
-      new ArrayList<DatanodeStorageInfo>();
-    
+      new ArrayList<>();
+
     NumberReplicas numReplicas = new NumberReplicas();
     // source node returned is not used
     chooseSourceDatanode(block, containingNodes,
@@ -572,7 +572,7 @@ public class BlockManager {
     Collection<DatanodeDescriptor> corruptNodes = 
                                   corruptReplicas.getNodes(block);
     
-    for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+    for (DatanodeStorageInfo storage : getStorages(block)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       String state = "";
       if (corruptNodes != null && corruptNodes.contains(node)) {
@@ -595,11 +595,23 @@ public class BlockManager {
     return maxReplicationStreams;
   }
 
+  public int getDefaultStorageNum(BlockInfo block) {
+    return defaultReplication;
+  }
+
+  public short getMinStorageNum(BlockInfo block) {
+    return minReplication;
+  }
+
   /**
-   * @return true if the block has minimum replicas
+   * @return true if the block has minimum stored copies
    */
-  public boolean checkMinReplication(BlockInfo block) {
-    return (countNodes(block).liveReplicas() >= minReplication);
+  public boolean hasMinStorage(BlockInfo block) {
+    return hasMinStorage(block, countNodes(block).liveReplicas());
+  }
+
+  public boolean hasMinStorage(BlockInfo block, int liveNum) {
+    return liveNum >= getMinStorageNum(block);
   }
 
   /**
@@ -614,8 +626,9 @@ public class BlockManager {
   private static boolean commitBlock(
       final BlockInfoUnderConstruction block, final Block commitBlock)
       throws IOException {
-    if (block.getBlockUCState() == BlockUCState.COMMITTED)
+    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();
@@ -635,18 +648,22 @@ public class BlockManager {
    */
   public boolean commitOrCompleteLastBlock(BlockCollection bc,
       Block commitBlock) throws IOException {
-    if(commitBlock == null)
+    if (commitBlock == null) {
       return false; // not committing, this is a block allocation retry
+    }
     BlockInfo lastBlock = bc.getLastBlock();
-    if(lastBlock == null)
+    if (lastBlock == null) {
       return false; // no blocks in file yet
-    if(lastBlock.isComplete())
+    }
+    if (lastBlock.isComplete()) {
       return false; // already completed (e.g. by syncBlock)
-    
+    }
+
     final boolean b = commitBlock(
         (BlockInfoUnderConstruction) lastBlock, commitBlock);
-    if(countNodes(lastBlock).liveReplicas() >= minReplication)
+    if(hasMinStorage(lastBlock)) {
       completeBlock(bc, bc.numBlocks()-1, false);
+    }
     return b;
   }
 
@@ -659,20 +676,24 @@ public class BlockManager {
    */
   private BlockInfo completeBlock(final BlockCollection bc,
       final int blkIndex, boolean force) throws IOException {
-    if(blkIndex < 0)
+    if (blkIndex < 0) {
       return null;
+    }
     BlockInfo curBlock = bc.getBlocks()[blkIndex];
-    if(curBlock.isComplete())
+    if(curBlock.isComplete()) {
       return curBlock;
+    }
     BlockInfoUnderConstruction ucBlock =
         (BlockInfoUnderConstruction) curBlock;
     int numNodes = ucBlock.numNodes();
-    if (!force && numNodes < minReplication)
+    if (!force && !hasMinStorage(curBlock, numNodes)) {
       throw new IOException("Cannot complete block: " +
           "block does not satisfy minimal replication requirement.");
-    if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
+    }
+    if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED) {
       throw new IOException(
           "Cannot complete block: block has not been COMMITTED by the client");
+    }
     BlockInfo completeBlock = ucBlock.convertToCompleteBlock();
     // replace penultimate block in file
     bc.setBlock(blkIndex, completeBlock);
@@ -757,7 +778,7 @@ public class BlockManager {
     // count in safe-mode.
     namesystem.adjustSafeModeBlockTotals(
         // decrement safe if we had enough
-        targets.length >= minReplication ? -1 : 0,
+        hasMinStorage(oldBlock, targets.length) ? -1 : 0,
         // always decrement total blocks
         -1);
 
@@ -771,8 +792,8 @@ public class BlockManager {
    */
   private List<DatanodeStorageInfo> getValidLocations(Block block) {
     final List<DatanodeStorageInfo> locations
-        = new ArrayList<DatanodeStorageInfo>(blocksMap.numNodes(block));
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+        = new ArrayList<>(blocksMap.numNodes(block));
+    for(DatanodeStorageInfo storage : getStorages(block)) {
       // filter invalidate replicas
       if(!invalidateBlocks.contains(storage.getDatanodeDescriptor(), block)) {
         locations.add(storage);
@@ -785,7 +806,7 @@ public class BlockManager {
       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++) {
@@ -798,10 +819,10 @@ public class BlockManager {
     }
 
     if (nrBlocks > 0 && curBlk == nrBlocks)   // offset >= end of file
-      return Collections.<LocatedBlock>emptyList();
+      return Collections.emptyList();
 
     long endOff = offset + length;
-    List<LocatedBlock> results = new ArrayList<LocatedBlock>(blocks.length);
+    List<LocatedBlock> results = new ArrayList<>(blocks.length);
     do {
       results.add(createLocatedBlock(blocks[curBlk], curPos, mode));
       curPos += blocks[curBlk].getNumBytes();
@@ -814,7 +835,7 @@ public class BlockManager {
 
   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++) {
@@ -838,8 +859,8 @@ public class BlockManager {
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
-      ) throws IOException {
+  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos)
+      throws IOException {
     if (blk instanceof BlockInfoUnderConstruction) {
       if (blk.isComplete()) {
         throw new IOException(
@@ -849,7 +870,8 @@ public class BlockManager {
       final BlockInfoUnderConstruction uc =
           (BlockInfoUnderConstruction) blk;
       final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-      final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
+      final ExtendedBlock eb =
+          new ExtendedBlock(namesystem.getBlockPoolId(), blk);
       return newLocatedBlock(eb, storages, pos, false);
     }
 
@@ -869,11 +891,12 @@ public class BlockManager {
     final DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines];
     int j = 0;
     if (numMachines > 0) {
-      for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
+      for(DatanodeStorageInfo storage : getStorages(blk)) {
         final DatanodeDescriptor d = storage.getDatanodeDescriptor();
         final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
-        if (isCorrupt || (!replicaCorrupt))
+        if (isCorrupt || (!replicaCorrupt)) {
           machines[j++] = storage;
+        }
       }
     }
     assert j == machines.length :
@@ -1047,7 +1070,7 @@ public class BlockManager {
     for(int i=0; i<startBlock; i++) {
       iter.next();
     }
-    List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
+    List<BlockWithLocations> results = new ArrayList<>();
     long totalSize = 0;
     BlockInfo curBlock;
     while(totalSize<size && iter.hasNext()) {
@@ -1071,7 +1094,7 @@ public class BlockManager {
    
   /** Remove the blocks associated to the given datanode. */
   void removeBlocksAssociatedTo(final DatanodeDescriptor node) {
-    final Iterator<? extends Block> it = node.getBlockIterator();
+    final Iterator<BlockInfo> it = node.getBlockIterator();
     while(it.hasNext()) {
       removeStoredBlock(it.next(), node);
     }
@@ -1085,10 +1108,10 @@ public class BlockManager {
   /** 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<BlockInfo> 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);
     }
@@ -1110,18 +1133,20 @@ public class BlockManager {
    * 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);
+      invalidateBlocks.add(storedBlock, node, false);
       datanodes.append(node).append(" ");
     }
     if (datanodes.length() != 0) {
-      blockLog.info("BLOCK* addToInvalidates: {} {}", b, datanodes.toString());
+      blockLog.info("BLOCK* addToInvalidates: {} {}", storedBlock,
+          datanodes.toString());
     }
   }
 
@@ -1148,7 +1173,8 @@ public class BlockManager {
   public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk,
       final DatanodeInfo dn, String storageID, String reason) throws IOException {
     assert namesystem.hasWriteLock();
-    final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock());
+    final Block reportedBlock = blk.getLocalBlock();
+    final BlockInfo storedBlock = getStoredBlock(reportedBlock);
     if (storedBlock == null) {
       // Check if the replica is in the blockMap, if not
       // ignore the request for now. This could happen when BlockScanner
@@ -1164,8 +1190,8 @@ public class BlockManager {
           + " as corrupt because datanode " + dn + " (" + dn.getDatanodeUuid()
           + ") does not exist");
     }
-    
-    markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock,
+
+    markBlockAsCorrupt(new BlockToMarkCorrupt(reportedBlock, storedBlock,
             blk.getGenerationStamp(), reason, Reason.CORRUPTION_REPORTED),
         storageID == null ? null : node.getStorageInfo(storageID),
         node);
@@ -1181,18 +1207,18 @@ public class BlockManager {
       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();
+        getExpectedReplicaNum(b.stored.getBlockCollection(), b.stored);
 
     // Add replica to the data-node if it is not already there
     if (storageInfo != null) {
-      storageInfo.addBlock(b.stored);
+      storageInfo.addBlock(b.stored, b.corrupted);
     }
 
     // Add this replica to corruptReplicas Map
@@ -1202,8 +1228,8 @@ public class BlockManager {
     NumberReplicas numberOfReplicas = countNodes(b.stored);
     boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >=
         expectedReplicas;
-    boolean minReplicationSatisfied =
-        numberOfReplicas.liveReplicas() >= minReplication;
+    boolean minReplicationSatisfied = hasMinStorage(b.stored,
+        numberOfReplicas.liveReplicas());
     boolean hasMoreCorruptReplicas = minReplicationSatisfied &&
         (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) >
         expectedReplicas;
@@ -1346,7 +1372,7 @@ public class BlockManager {
     int additionalReplRequired;
 
     int scheduledWork = 0;
-    List<ReplicationWork> work = new LinkedList<ReplicationWork>();
+    List<ReplicationWork> work = new LinkedList<>();
 
     namesystem.writeLock();
     try {
@@ -1363,11 +1389,11 @@ public class BlockManager {
               continue;
             }
 
-            requiredReplication = bc.getPreferredBlockReplication();
+            requiredReplication = getExpectedReplicaNum(bc, block);
 
             // get a source data-node
-            containingNodes = new ArrayList<DatanodeDescriptor>();
-            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<DatanodeStorageInfo>();
+            containingNodes = new ArrayList<>();
+            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
             NumberReplicas numReplicas = new NumberReplicas();
             srcNode = chooseSourceDatanode(
                 block, containingNodes, liveReplicaNodes, numReplicas,
@@ -1387,7 +1413,7 @@ public class BlockManager {
       
             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);
@@ -1411,7 +1437,7 @@ public class BlockManager {
       namesystem.writeUnlock();
     }
 
-    final Set<Node> excludedNodes = new HashSet<Node>();
+    final Set<Node> excludedNodes = new HashSet<>();
     for(ReplicationWork rw : work){
       // Exclude all of the containing nodes from being targets.
       // This list includes decommissioning or corrupt nodes.
@@ -1447,7 +1473,7 @@ public class BlockManager {
             rw.targets = null;
             continue;
           }
-          requiredReplication = bc.getPreferredBlockReplication();
+          requiredReplication = getExpectedReplicaNum(bc, block);
 
           // do not schedule more if enough replicas is already pending
           NumberReplicas numReplicas = countNodes(block);
@@ -1456,7 +1482,7 @@ public class BlockManager {
 
           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" +
@@ -1466,7 +1492,7 @@ public class BlockManager {
           }
 
           if ( (numReplicas.liveReplicas() >= requiredReplication) &&
-               (!blockHasEnoughRacks(block)) ) {
+               (!blockHasEnoughRacks(block, requiredReplication)) ) {
             if (rw.srcNode.getNetworkLocation().equals(
                 targets[0].getDatanodeDescriptor().getNetworkLocation())) {
               //No use continuing, unless a new rack in this case
@@ -1581,7 +1607,7 @@ public class BlockManager {
   List<DatanodeDescriptor> getDatanodeDescriptors(List<String> nodes) {
     List<DatanodeDescriptor> datanodeDescriptors = null;
     if (nodes != null) {
-      datanodeDescriptors = new ArrayList<DatanodeDescriptor>(nodes.size());
+      datanodeDescriptors = new ArrayList<>(nodes.size());
       for (int i = 0; i < nodes.size(); i++) {
         DatanodeDescriptor node = datanodeManager.getDatanodeDescriptor(nodes.get(i));
         if (node != null) {
@@ -1637,9 +1663,9 @@ public class BlockManager {
     int excess = 0;
     
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+    for(DatanodeStorageInfo storage : getStorages(block)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
-      LightWeightLinkedSet<Block> excessBlocks =
+      LightWeightLinkedSet<BlockInfo> excessBlocks =
         excessReplicateMap.get(node.getDatanodeUuid());
       int countableReplica = storage.getState() == State.NORMAL ? 1 : 0; 
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
@@ -1707,7 +1733,7 @@ public class BlockManager {
            * Use the blockinfo from the blocksmap to be certain we're working
            * with the most up-to-date block information (e.g. genstamp).
            */
-          BlockInfo bi = blocksMap.getStoredBlock(timedOutItems[i]);
+          BlockInfo bi = getStoredBlock(timedOutItems[i]);
           if (bi == null) {
             continue;
           }
@@ -1757,7 +1783,7 @@ public class BlockManager {
     final BlockInfoUnderConstruction storedBlock;
     final Block reportedBlock;
     final ReplicaState reportedState;
-    
+
     StatefulBlockInfo(BlockInfoUnderConstruction storedBlock,
         Block reportedBlock, ReplicaState reportedState) {
       this.storedBlock = storedBlock;
@@ -1765,14 +1791,34 @@ public class BlockManager {
       this.reportedState = reportedState;
     }
   }
-  
+
+  private static class BlockInfoToAdd {
+    private final BlockInfo stored;
+    private final Block reported;
+
+    BlockInfoToAdd(BlockInfo stored, Block reported) {
+      this.stored = stored;
+      this.reported = reported;
+    }
+
+    public BlockInfo getStored() {
+      return stored;
+    }
+
+    public Block getReported() {
+      return reported;
+    }
+  }
+
   /**
    * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a
    * 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 reason to mark corrupt. */
@@ -1780,7 +1826,7 @@ public class BlockManager {
     /** The reason code to be stored */
     final Reason reasonCode;
 
-    BlockToMarkCorrupt(BlockInfo corrupted,
+    BlockToMarkCorrupt(Block corrupted,
         BlockInfo stored, String reason,
         Reason reasonCode) {
       Preconditions.checkNotNull(corrupted, "corrupted is null");
@@ -1792,15 +1838,9 @@ public class BlockManager {
       this.reasonCode = reasonCode;
     }
 
-    BlockToMarkCorrupt(BlockInfo stored, String reason,
-        Reason reasonCode) {
-      this(stored, stored, reason, reasonCode);
-    }
-
-    BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
-        Reason reasonCode) {
-      this(new BlockInfoContiguous(stored), stored,
-          reason, reasonCode);
+    BlockToMarkCorrupt(Block corrupted, BlockInfo stored, long gs,
+        String reason, Reason reasonCode) {
+      this(corrupted, stored, reason, reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
     }
@@ -1987,7 +2027,7 @@ public class BlockManager {
           break;
         }
 
-        BlockInfo bi = blocksMap.getStoredBlock(b);
+        BlockInfo bi = getStoredBlock(b);
         if (bi == null) {
           if (LOG.isDebugEnabled()) {
             LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " +
@@ -2019,7 +2059,7 @@ public class BlockManager {
           endPostponedMisReplicatedBlocksCount) + " blocks are removed.");
     }
   }
-  
+
   private Collection<Block> processReport(
       final DatanodeStorageInfo storageInfo,
       final BlockListAsLongs report) throws IOException {
@@ -2027,25 +2067,26 @@ public class BlockManager {
     // Modify the (block-->datanode) map, according to the difference
     // between the old and new block report.
     //
-    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
-    Collection<Block> toRemove = new TreeSet<Block>();
-    Collection<Block> toInvalidate = new LinkedList<Block>();
-    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
-    Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
+    Collection<BlockInfoToAdd> toAdd = new LinkedList<>();
+    Collection<BlockInfo> toRemove = new TreeSet<>();
+    Collection<Block> toInvalidate = new LinkedList<>();
+    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<>();
+    Collection<StatefulBlockInfo> toUC = new LinkedList<>();
     reportDiff(storageInfo, report,
         toAdd, toRemove, toInvalidate, toCorrupt, toUC);
-   
+
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     // Process the blocks on each queue
-    for (StatefulBlockInfo b : toUC) { 
+    for (StatefulBlockInfo b : toUC) {
       addStoredBlockUnderConstruction(b, storageInfo);
     }
-    for (Block b : toRemove) {
+    for (BlockInfo b : toRemove) {
       removeStoredBlock(b, node);
     }
     int numBlocksLogged = 0;
-    for (BlockInfo b : toAdd) {
-      addStoredBlock(b, storageInfo, null, numBlocksLogged < maxNumBlocksToLog);
+    for (BlockInfoToAdd b : toAdd) {
+      addStoredBlock(b.getStored(), b.getReported(), storageInfo, null,
+          numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
@@ -2066,17 +2107,17 @@ public class BlockManager {
    * Mark block replicas as corrupt except those on the storages in 
    * newStorages list.
    */
-  public void markBlockReplicasAsCorrupt(BlockInfo block,
-      long oldGenerationStamp, long oldNumBytes, 
+  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 {
@@ -2135,7 +2176,7 @@ public class BlockManager {
         continue;
       }
       
-      BlockInfo storedBlock = blocksMap.getStoredBlock(iblk);
+      BlockInfo storedBlock = getStoredBlock(iblk);
       // If block does not belong to any file, we are done.
       if (storedBlock == null) continue;
       
@@ -2173,24 +2214,26 @@ public class BlockManager {
       }      
       //add replica if appropriate
       if (reportedState == ReplicaState.FINALIZED) {
-        addStoredBlockImmediate(storedBlock, storageInfo);
+        addStoredBlockImmediate(storedBlock, iblk, storageInfo);
       }
     }
   }
 
-  private void reportDiff(DatanodeStorageInfo storageInfo, 
-      BlockListAsLongs newReport, 
-      Collection<BlockInfo> toAdd,              // add to DatanodeDescriptor
-      Collection<Block> toRemove,           // remove from DatanodeDescriptor
+  private void reportDiff(DatanodeStorageInfo storageInfo,
+      BlockListAsLongs newReport,
+      Collection<BlockInfoToAdd> toAdd,     // add to DatanodeDescriptor
+      Collection<BlockInfo> toRemove,       // remove from DatanodeDescriptor
       Collection<Block> toInvalidate,       // should be removed from DN
       Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
       Collection<StatefulBlockInfo> toUC) { // add to under-construction list
 
-    // place a delimiter in the list which separates blocks 
+    // place a delimiter in the list which separates blocks
     // that have been reported from those that have not
-    BlockInfo delimiter = new BlockInfoContiguous(new Block(), (short) 1);
-    AddBlockResult result = storageInfo.addBlock(delimiter);
-    assert result == AddBlockResult.ADDED 
+    Block delimiterBlock = new Block();
+    BlockInfo 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
     int curIndex;
@@ -2207,7 +2250,8 @@ public class BlockManager {
       // move block to the head of the list
       if (storedBlock != null &&
           (curIndex = storedBlock.findStorageInfo(storageInfo)) >= 0) {
-        headIndex = storageInfo.moveBlockToHead(storedBlock, curIndex, headIndex);
+        headIndex =
+            storageInfo.moveBlockToHead(storedBlock, curIndex, headIndex);
       }
     }
 
@@ -2215,8 +2259,9 @@ public class BlockManager {
     // all of them are next to the delimiter
     Iterator<BlockInfo> it =
         storageInfo.new BlockIterator(delimiter.getNext(0));
-    while(it.hasNext())
+    while (it.hasNext()) {
       toRemove.add(it.next());
+    }
     storageInfo.removeBlock(delimiter);
   }
 
@@ -2253,12 +2298,12 @@ public class BlockManager {
    */
   private BlockInfo processReportedBlock(
       final DatanodeStorageInfo storageInfo,
-      final Block block, final ReplicaState reportedState, 
-      final Collection<BlockInfo> toAdd,
-      final Collection<Block> toInvalidate, 
+      final Block block, final ReplicaState reportedState,
+      final Collection<BlockInfoToAdd> toAdd,
+      final Collection<Block> toInvalidate,
       final Collection<BlockToMarkCorrupt> toCorrupt,
       final Collection<StatefulBlockInfo> toUC) {
-    
+
     DatanodeDescriptor dn = storageInfo.getDatanodeDescriptor();
 
     if(LOG.isDebugEnabled()) {
@@ -2266,16 +2311,16 @@ public class BlockManager {
           + " on " + dn + " size " + block.getNumBytes()
           + " replicaState = " + reportedState);
     }
-  
+
     if (shouldPostponeBlocksFromFuture &&
         namesystem.isGenStampInFuture(block)) {
       queueReportedBlock(storageInfo, block, reportedState,
           QUEUE_REASON_FUTURE_GENSTAMP);
       return null;
     }
-    
+
     // find block by blockId
-    BlockInfo storedBlock = blocksMap.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.
@@ -2283,7 +2328,7 @@ public class BlockManager {
       return null;
     }
     BlockUCState ucState = storedBlock.getBlockUCState();
-    
+
     // Block is on the NN
     if(LOG.isDebugEnabled()) {
       LOG.debug("In memory blockUCState = " + ucState);
@@ -2328,8 +2373,8 @@ public class BlockManager {
     // but now okay, it might need to be updated.
     if (reportedState == ReplicaState.FINALIZED
         && (storedBlock.findStorageInfo(storageInfo) == -1 ||
-            corruptReplicas.isReplicaCorrupt(storedBlock, dn))) {
-      toAdd.add(storedBlock);
+        corruptReplicas.isReplicaCorrupt(storedBlock, dn))) {
+      toAdd.add(new BlockInfoToAdd(storedBlock, block));
     }
     return storedBlock;
   }
@@ -2375,7 +2420,7 @@ public class BlockManager {
       if (rbi.getReportedState() == null) {
         // This is a DELETE_BLOCK request
         DatanodeStorageInfo storageInfo = rbi.getStorageInfo();
-        removeStoredBlock(rbi.getBlock(),
+        removeStoredBlock(getStoredBlock(rbi.getBlock()),
             storageInfo.getDatanodeDescriptor());
       } else {
         processAndHandleReportedBlock(rbi.getStorageInfo(),
@@ -2423,15 +2468,15 @@ public class BlockManager {
       case COMMITTED:
         if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
           final long reportedGS = reported.getGenerationStamp();
-          return new BlockToMarkCorrupt(storedBlock, reportedGS,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS,
               "block is " + ucState + " and reported genstamp " + reportedGS
-              + " does not match genstamp in block map "
-              + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
+                  + " 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(new Block(reported), storedBlock,
               "block is " + ucState + " and reported length " +
-              reported.getNumBytes() + " does not match " +
-              "length in block map " + storedBlock.getNumBytes(),
+                  reported.getNumBytes() + " does not match " +
+                  "length in block map " + storedBlock.getNumBytes(),
               Reason.SIZE_MISMATCH);
         } else {
           return null; // not corrupt
@@ -2439,11 +2484,12 @@ public class BlockManager {
       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
-              + ", But reported genstamp " + reportedGS
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
+              reportedGS, "block is " + ucState + " and reported state "
+              + reportedState + ", But reported genstamp " + reportedGS
               + " does not match genstamp in block map "
-              + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
+              + storedBlock.getGenerationStamp(),
+              Reason.GENSTAMP_MISMATCH);
         }
         return null;
       default:
@@ -2453,12 +2499,15 @@ public class BlockManager {
     case RWR:
       if (!storedBlock.isComplete()) {
         return null; // not corrupt
-      } else if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
+      } else if (storedBlock.getGenerationStamp() !=
+          reported.getGenerationStamp()) {
         final long reportedGS = reported.getGenerationStamp();
-        return new BlockToMarkCorrupt(storedBlock, reportedGS,
-            "reported " + reportedState + " replica with genstamp " + reportedGS
-            + " does not match COMPLETE block's genstamp in block map "
-            + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
+        return new BlockToMarkCorrupt(
+            new Block(reported), storedBlock, reportedGS,
+            "reported " + reportedState +
+                " replica with genstamp " + reportedGS +
+                " does not match COMPLETE block's genstamp in block map " +
+                storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
       } else { // COMPLETE block, same genstamp
         if (reportedState == ReplicaState.RBW) {
           // If it's a RBW report for a COMPLETE block, it may just be that
@@ -2470,7 +2519,7 @@ public class BlockManager {
               "complete with the same genstamp");
           return null;
         } else {
-          return new BlockToMarkCorrupt(storedBlock,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
               "reported replica has invalid state " + reportedState,
               Reason.INVALID_STATE);
         }
@@ -2483,7 +2532,8 @@ public class BlockManager {
       " 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(new Block(reported), storedBlock, msg,
+          Reason.INVALID_STATE);
     }
   }
 
@@ -2516,7 +2566,7 @@ public class BlockManager {
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
         (block.findStorageInfo(storageInfo) < 0)) {
-      addStoredBlock(block, storageInfo, null, true);
+      addStoredBlock(block, ucBlock.reportedBlock, storageInfo, null, true);
     }
   } 
 
@@ -2531,23 +2581,23 @@ public class BlockManager {
    * 
    * @throws IOException
    */
-  private void addStoredBlockImmediate(BlockInfo storedBlock,
+  private void addStoredBlockImmediate(BlockInfo storedBlock, Block reported,
       DatanodeStorageInfo storageInfo)
-  throws IOException {
+      throws IOException {
     assert (storedBlock != null && namesystem.hasWriteLock());
-    if (!namesystem.isInStartupSafeMode() 
+    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);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
-        && numCurrentReplica >= minReplication) {
+        && hasMinStorage(storedBlock, numCurrentReplica)) {
       completeBlock(storedBlock.getBlockCollection(), storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
@@ -2561,19 +2611,20 @@ public class BlockManager {
   /**
    * 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,
-                               DatanodeStorageInfo storageInfo,
-                               DatanodeDescriptor delNodeHint,
-                               boolean logEveryBlock)
-  throws IOException {
+      final Block reportedBlock,
+      DatanodeStorageInfo storageInfo,
+      DatanodeDescriptor delNodeHint,
+      boolean logEveryBlock)
+      throws IOException {
     assert block != null && namesystem.hasWriteLock();
     BlockInfo storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     if (block instanceof BlockInfoUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
-      storedBlock = blocksMap.getStoredBlock(block);
+      storedBlock = getStoredBlock(block);
     } else {
       storedBlock = block;
     }
@@ -2587,10 +2638,9 @@ public class BlockManager {
       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);
+    AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock);
 
     int curReplicaDelta;
     if (result == AddBlockResult.ADDED) {
@@ -2618,10 +2668,10 @@ public class BlockManager {
     NumberReplicas num = countNodes(storedBlock);
     int numLiveReplicas = num.liveReplicas();
     int numCurrentReplica = numLiveReplicas
-      + pendingReplications.getNumReplicas(storedBlock);
+        + pendingReplications.getNumReplicas(storedBlock);
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
-        numLiveReplicas >= minReplication) {
+        hasMinStorage(storedBlock, numLiveReplicas)) {
       storedBlock = completeBlock(bc, storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
@@ -2631,7 +2681,7 @@ public class BlockManager {
       // handles the safe block count maintenance.
       namesystem.incrementSafeBlockCount(numCurrentReplica);
     }
-    
+
     // if file is under construction, then done for now
     if (bc.isUnderConstruction()) {
       return storedBlock;
@@ -2643,7 +2693,7 @@ public class BlockManager {
     }
 
     // handle underReplication/overReplication
-    short fileReplication = bc.getPreferredBlockReplication();
+    short fileReplication = getExpectedReplicaNum(bc, storedBlock);
     if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
       neededReplications.remove(storedBlock, numCurrentReplica,
           num.decommissionedAndDecommissioning(), fileReplication);
@@ -2659,11 +2709,12 @@ public class BlockManager {
     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))
-      invalidateCorruptReplicas(storedBlock);
+    if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) {
+      invalidateCorruptReplicas(storedBlock, reportedBlock);
+    }
     return storedBlock;
   }
 
@@ -2695,7 +2746,7 @@ public class BlockManager {
    *
    * @param blk Block whose corrupt replicas need to be invalidated
    */
-  private void invalidateCorruptReplicas(BlockInfo blk) {
+  private void invalidateCorruptReplicas(BlockInfo blk, Block reported) {
     Collection<DatanodeDescriptor> nodes = corruptReplicas.getNodes(blk);
     boolean removedFromBlocksMap = true;
     if (nodes == null)
@@ -2705,8 +2756,8 @@ public class BlockManager {
     DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]);
     for (DatanodeDescriptor node : nodesCopy) {
       try {
-        if (!invalidateBlock(new BlockToMarkCorrupt(blk, null,
-              Reason.ANY), node)) {
+        if (!invalidateBlock(new BlockToMarkCorrupt(reported, blk, null,
+            Reason.ANY), node)) {
           removedFromBlocksMap = false;
         }
       } catch (IOException e) {
@@ -2874,7 +2925,7 @@ public class BlockManager {
     }
     // calculate current replication
     short expectedReplication =
-        block.getBlockCollection().getPreferredBlockReplication();
+        getExpectedReplicaNum(block.getBlockCollection(), block);
     NumberReplicas num = countNodes(block);
     int numCurrentReplica = num.liveReplicas();
     // add to under-replicated queue if need to be
@@ -2933,14 +2984,14 @@ public class BlockManager {
    * 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<DatanodeStorageInfo> nonExcess = new ArrayList<DatanodeStorageInfo>();
+    Collection<DatanodeStorageInfo> nonExcess = new ArrayList<>();
     Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
         .getNodes(block);
     for(DatanodeStorageInfo storage : blocksMap.getStorages(block, State.NORMAL)) {
@@ -2954,8 +3005,8 @@ public class BlockManager {
         postponeBlock(block);
         return;
       }
-      LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(cur
-          .getDatanodeUuid());
+      LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+          cur.getDatanodeUuid());
       if (excessBlocks == null || !excessBlocks.contains(block)) {
         if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
           // exclude corrupt replicas
@@ -2965,7 +3016,7 @@ public class BlockManager {
         }
       }
     }
-    chooseExcessReplicates(nonExcess, block, replication, 
+    chooseExcessReplicates(nonExcess, block, replication,
         addedNode, delNodeHint, blockplacement);
   }
 
@@ -2984,29 +3035,29 @@ public class BlockManager {
    * If no such a node is available,
    * then pick a node with least free space
    */
-  private void chooseExcessReplicates(final Collection<DatanodeStorageInfo> nonExcess, 
-                              Block b, short replication,
-                              DatanodeDescriptor addedNode,
-                              DatanodeDescriptor delNodeHint,
-                              BlockPlacementPolicy replicator) {
+  private void chooseExcessReplicates(
+      final Collection<DatanodeStorageInfo> nonExcess,
+      BlockInfo storedBlock, short replication,
+      DatanodeDescriptor addedNode,
+      DatanodeDescriptor delNodeHint,
+      BlockPlacementPolicy replicator) {
     assert namesystem.hasWriteLock();
     // first form a rack to datanodes map and
-    BlockCollection bc = getBlockCollection(b);
-    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(bc.getStoragePolicyID());
+    BlockCollection bc = getBlockCollection(storedBlock);
+    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(
+        bc.getStoragePolicyID());
     final List<StorageType> excessTypes = storagePolicy.chooseExcess(
         replication, DatanodeStorageInfo.toStorageTypes(nonExcess));
 
+    final Map<String, List<DatanodeStorageInfo>> rackMap = new HashMap<>();
+    final List<DatanodeStorageInfo> moreThanOne = new ArrayList<>();
+    final List<DatanodeStorageInfo> exactlyOne = new ArrayList<>();
 
-    final Map<String, List<DatanodeStorageInfo>> rackMap
-        = new HashMap<String, List<DatanodeStorageInfo>>();
-    final List<DatanodeStorageInfo> moreThanOne = new ArrayList<DatanodeStorageInfo>();
-    final List<DatanodeStorageInfo> exactlyOne = new ArrayList<DatanodeStorageInfo>();
-    
     // 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);
-    
+
     // 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
@@ -3021,7 +3072,7 @@ public class BlockManager {
           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;
@@ -3030,24 +3081,29 @@ public class BlockManager {
       replicator.adjustSetsWithChosenReplica(rackMap, moreThanOne,
           exactlyOne, cur);
 
-      nonExcess.remove(cur);
-      addToExcessReplicate(cur.getDatanodeDescriptor(), b);
-
-      //
-      // The 'excessblocks' tracks blocks until we get confirmation
-      // that the datanode has deleted them; the only way we remove them
-      // is when we get a "removeBlock" message.  
-      //
-      // The 'invalidate' list is used to inform the datanode the block 
-      // should be deleted.  Items are removed from the invalidate list
-      // upon giving instructions to the namenode.
-      //
-      addToInvalidates(b, cur.getDatanodeDescriptor());
-      blockLog.info("BLOCK* chooseExcessReplicates: "
-                +"({}, {}) is added to invalidated blocks set", cur, b);
+      processChosenExcessReplica(nonExcess, cur, storedBlock);
     }
   }
 
+  private void processChosenExcessReplica(
+      final Collection<DatanodeStorageInfo> nonExcess,
+      final DatanodeStorageInfo chosen, BlockInfo storedBlock) {
+    nonExcess.remove(chosen);
+    addToExcessReplicate(chosen.getDatanodeDescriptor(), storedBlock);
+    //
+    // The 'excessblocks' tracks blocks until we get confirmation
+    // that the datanode has deleted them; the only way we remove them
+    // is when we get a "removeBlock" message.
+    //
+    // The 'invalidate' list is used to inform the datanode the block
+    // should be deleted.  Items are removed from the invalidate list
+    // upon giving instructions to the datanodes.
+    //
+    addToInvalidates(storedBlock, chosen.getDatanodeDescriptor());
+    blockLog.info("BLOCK* chooseExcessReplicates: "
+        +"({}, {}) is added to invalidated blocks set", chosen, storedBlock);
+  }
+
   /** Check if we can use delHint */
   static boolean useDelHint(boolean isFirst, DatanodeStorageInfo delHint,
       DatanodeStorageInfo added, List<DatanodeStorageInfo> moreThan1Racks,
@@ -3069,17 +3125,18 @@ public class BlockManager {
     }
   }
 
-  private void addToExcessReplicate(DatanodeInfo dn, Block block) {
+  private void addToExcessReplicate(DatanodeInfo dn, BlockInfo storedBlock) {
     assert namesystem.hasWriteLock();
-    LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid());
+    LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+        dn.getDatanodeUuid());
     if (excessBlocks == null) {
-      excessBlocks = new LightWeightLinkedSet<Block>();
+      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);
     }
   }
 
@@ -3091,26 +3148,26 @@ public class BlockManager {
           QUEUE_REASON_FUTURE_GENSTAMP);
       return;
     }
-    removeStoredBlock(block, node);
+    removeStoredBlock(getStoredBlock(block), node);
   }
 
   /**
    * 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;
       }
 
       CachedBlock cblock = namesystem.getCacheManager().getCachedBlocks()
-          .get(new CachedBlock(block.getBlockId(), (short) 0, false));
+          .get(new CachedBlock(storedBlock.getBlockId(), (short) 0, false));
       if (cblock != null) {
         boolean removed = false;
         removed |= node.getPendingCached().remove(cblock);
@@ -3118,7 +3175,7 @@ public class BlockManager {
         removed |= node.getPendingUncached().remove(cblock);
         if (removed) {
           blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching "
-              + "related lists on node {}", block, node);
+              + "related lists on node {}", storedBlock, node);
         }
       }
 
@@ -3128,7 +3185,7 @@ public class BlockManager {
       // 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);
@@ -3138,13 +3195,13 @@ public class BlockManager {
       // We've removed a block from a node, so it's definitely no longer
       // in "excess" there.
       //
-      LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(node
-          .getDatanodeUuid());
+      LightWeightLinkedSet<BlockInfo> 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());
           }
@@ -3152,7 +3209,7 @@ public class BlockManager {
       }
 
       // Remove the replica from corruptReplicas
-      corruptReplicas.removeFromCorruptReplicasMap(block, node);
+      corruptReplicas.removeFromCorruptReplicasMap(storedBlock, node);
     }
   }
 
@@ -3160,7 +3217,7 @@ public class BlockManager {
    * Get all valid locations of the block & add the block to results
    * return the length of the added block; 0 if the block is not added
    */
-  private long addBlock(Block block, List<BlockWithLocations> results) {
+  private long addBlock(BlockInfo block, List<BlockWithLocations> results) {
     final List<DatanodeStorageInfo> locations = getValidLocations(block);
     if(locations.size() == 0) {
       return 0;
@@ -3212,31 +3269,32 @@ public class BlockManager {
     processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
         delHintNode);
   }
-  
+
   private void processAndHandleReportedBlock(
       DatanodeStorageInfo storageInfo, Block block,
       ReplicaState reportedState, DatanodeDescriptor delHintNode)
       throws IOException {
     // blockReceived reports a finalized block
-    Collection<BlockInfo> toAdd = new LinkedList<BlockInfo>();
-    Collection<Block> toInvalidate = new LinkedList<Block>();
-    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
-    Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
+    Collection<BlockInfoToAdd> toAdd = new LinkedList<>();
+    Collection<Block> toInvalidate = new LinkedList<>();
+    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<>();
+    Collection<StatefulBlockInfo> 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
-      : "The block should be only in one of the lists.";
+        : "The block should be only in one of the lists.";
 
-    for (StatefulBlockInfo b : toUC) { 
+    for (StatefulBlockInfo b : toUC) {
       addStoredBlockUnderConstruction(b, storageInfo);
     }
     long numBlocksLogged = 0;
-    for (BlockInfo b : toAdd) {
-      addStoredBlock(b, storageInfo, delHintNode, numBlocksLogged < maxNumBlocksToLog);
+    for (BlockInfoToAdd b : toAdd) {
+      addStoredBlock(b.getStored(), b.getReported(), storageInfo, delHintNode,
+          numBlocksLogged < maxNumBlocksToLog);
       numBlocksLogged++;
     }
     if (numBlocksLogged > maxNumBlocksToLog) {
@@ -3301,7 +3359,7 @@ public class BlockManager {
                                       ReplicaState.RBW, null);
         break;
       default:
-        String msg = 
+        String msg =
           "Unknown block status code reported by " + nodeID +
           ": " + rdbi;
         blockLog.warn(msg);
@@ -3337,8 +3395,8 @@ public class BlockManager {
       } else if (node.isDecommissioned()) {
         decommissioned++;
       } else {
-        LightWeightLinkedSet<Block> blocksExcess = excessReplicateMap.get(node
-            .getDatanodeUuid());
+        LightWeightLinkedSet<BlockInfo> blocksExcess =
+            excessReplicateMap.get(node.getDatanodeUuid());
         if (blocksExcess != null && blocksExcess.contains(b)) {
           excess++;
         } else {
@@ -3391,13 +3449,13 @@ public class BlockManager {
     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++;
       }
     }
@@ -3423,7 +3481,7 @@ public class BlockManager {
     if (pendingReplicationBlocksCount == 0 &&
         underReplicatedBlocksCount == 0) {
       LOG.info("Node {} is dead and there are no under-replicated" +
-          " blocks or blocks pending replication. Safe to decommission.", 
+          " blocks or blocks pending replication. Safe to decommission.",
           node);
       return true;
     }
@@ -3441,6 +3499,12 @@ public class BlockManager {
     return blocksMap.size();
   }
 
+
+  /** @return an iterator of the datanodes. */
+  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
+    return blocksMap.getStorages(block);
+  }
+
   public DatanodeStorageInfo[] getStorages(BlockInfo block) {
     final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[block.numNodes()];
     int i = 0;
@@ -3529,10 +3593,12 @@ public class BlockManager {
         final BlockInfoUnderConstruction uc =
             (BlockInfoUnderConstruction)b;
         final int numNodes = b.numNodes();
-        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
-          + uc.getBlockUCState() + ", replication# = " + numNodes
-          + (numNodes < minReplication ? " < ": " >= ")
-          + " minimum = " + minReplication + ") in file " + src);
+        final int min = getMinStorageNum(b);
+        final BlockUCState state = b.getBlockUCState();
+        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " + state
+            + ", replication# = " + numNodes
+            + (numNodes < min ? " < " : " >= ")
+            + " minimum = " + min + ") in file " + src);
         return false;
       }
     }
@@ -3543,15 +3609,15 @@ public class BlockManager {
    * @return 0 if the block is not found;
    *         otherwise, return the replication factor of the block.
    */
-  private int getReplication(Block block) {
+  private int getReplication(BlockInfo block) {
     final BlockCollection bc = blocksMap.getBlockCollection(block);
-    return bc == null? 0: bc.getPreferredBlockReplication();
+    return bc == null? 0: getExpectedReplicaNum(bc, block);
   }
 
 
   /**
-   * Get blocks to invalidate for <i>nodeId</i>
-   * in {@link #invalidateBlocks}.
+   * Get blocks to invalidate for <i>nodeId</i>.
+   * in {@link #invalidateBlocks}.boolean blockHasEnoughRacks
    *
    * @return number of blocks scheduled for removal during this iteration.
    */
@@ -3589,22 +3655,20 @@ public class BlockManager {
     return toInvalidate.size();
   }
 
-  boolean blockHasEnoughRacks(Block b) {
+  boolean blockHasEnoughRacks(BlockInfo storedBlock, int expectedStorageNum) {
     if (!this.shouldCheckForEnoughRacks) {
       return true;
     }
-    boolean enoughRacks = false;;
-    Collection<DatanodeDescriptor> corruptNodes = 
-                                  corruptReplicas.getNodes(b);
-    int numExpectedReplicas = getReplication(b);
+    boolean enoughRacks = false;
+    Collection<DatanodeDescriptor> corruptNodes =
+        corruptReplicas.getNodes(storedBlock);
     String rackName = null;
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
+    for(DatanodeStorageInfo storage : getStorages(storedBlock)) {
       final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
         if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
-          if (numExpectedReplicas == 1 ||
-              (numExpectedReplicas > 1 &&
-                  !datanodeManager.hasClusterEverBeenMultiRack())) {
+          if (expectedStorageNum == 1 || (expectedStorageNum > 1 &&
+              !datanodeManager.hasClusterEverBeenMultiRack())) {
             enoughRacks = true;
             break;
           }
@@ -3625,8 +3689,13 @@ public class BlockManager {
    * 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 short getExpectedReplicaNum(BlockCollection bc, BlockInfo block) {
+    return bc.getPreferredBlockReplication();
   }
   
   public long getMissingBlocksCount() {
@@ -3648,11 +3717,6 @@ public class BlockManager {
     return blocksMap.getBlockCollection(b);
   }
 
-  /** @return an iterator of the datanodes. */
-  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
-    return blocksMap.getStorages(block);
-  }
-
   public int numCorruptReplicas(Block block) {
     return corruptReplicas.numCorruptReplicas(block);
   }
@@ -3668,9 +3732,10 @@ public class BlockManager {
    * If a block is removed from blocksMap, remove it from excessReplicateMap.
    */
   private void removeFromExcessReplicateMap(Block block) {
-    for (DatanodeStorageInfo info : blocksMap.getStorages(block)) {
+    for (DatanodeStorageInfo info : getStorages(block)) {
       String uuid = info.getDatanodeDescriptor().getDatanodeUuid();
-      LightWeightLinkedSet<Block> excessReplicas = excessReplicateMap.get(uuid);
+      LightWeightLinkedSet<BlockInfo> excessReplicas =
+          excessReplicateMap.get(uuid);
       if (excessReplicas != null) {
         if (excessReplicas.remove(block)) {
           excessBlocksCount.decrementAndGet();
@@ -3685,26 +3750,6 @@ public class BlockManager {
   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.
@@ -3879,7 +3924,7 @@ public class BlockManager {
 
   /**
    * A simple result enum for the result of
-   * {@link BlockManager#processMisReplicatedBlock(BlockInfo)}.
+   * {@link BlockManager#processMisReplicatedBlock}.
    */
   enum MisReplicationResult {
     /** The block should be invalidated since it belongs to a deleted file. */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
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 57e81b4..65b83e1 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 java.util.List;
 import com.google.common.annotations.VisibleForTesting;
 
 import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
@@ -226,7 +227,7 @@ public class DatanodeStorageInfo {
     return blockPoolUsed;
   }
 
-  public AddBlockResult addBlock(BlockInfo 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;
@@ -245,10 +246,18 @@ public class DatanodeStorageInfo {
     }
 
     // add to the head of the data-node list
-    b.addStorage(this);
+    b.addStorage(this, reportedBlock);
+    insertToList(b);
+    return result;
+  }
+
+  AddBlockResult addBlock(BlockInfo b) {
+    return addBlock(b, b);
+  }
+
+  public void insertToList(BlockInfo b) {
     blockList = b.listInsert(blockList, this);
     numBlocks++;
-    return result;
   }
 
   public boolean removeBlock(BlockInfo b) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index b073a89..d0f4e08 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
@@ -143,7 +143,6 @@ import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -3136,7 +3135,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (trackBlockCounts) {
         if (b.isComplete()) {
           numRemovedComplete++;
-          if (blockManager.checkMinReplication(b)) {
+          if (blockManager.hasMinStorage(b)) {
             numRemovedSafe++;
           }
         }
@@ -3368,7 +3367,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
         break;
-      assert blockManager.checkMinReplication(curBlock) :
+      assert blockManager.hasMinStorage(curBlock) :
               "A COMPLETE block is not minimally replicated in " + src;
     }
 
@@ -3404,7 +3403,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // If penultimate block doesn't exist then its minReplication is met
     boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
-        blockManager.checkMinReplication(penultimateBlock);
+        blockManager.hasMinStorage(penultimateBlock);
 
     switch(lastBlockState) {
     case COMPLETE:
@@ -3413,7 +3412,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     case COMMITTED:
       // Close file if committed blocks are minimally replicated
       if(penultimateBlockMinReplication &&
-          blockManager.checkMinReplication(lastBlock)) {
+          blockManager.hasMinStorage(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId());
         NameNode.stateChangeLog.warn("BLOCK*"
@@ -3705,9 +3704,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 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);
               }
             }
           }
@@ -3723,8 +3722,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         } else {
           iFile.setLastBlock(storedBlock, trimmedStorageInfos);
           if (closeFile) {
-            blockManager.markBlockReplicasAsCorrupt(storedBlock,
-                oldGenerationStamp, oldNumBytes, trimmedStorageInfos);
+            blockManager.markBlockReplicasAsCorrupt(oldBlock.getLocalBlock(),
+                storedBlock, oldGenerationStamp, oldNumBytes,
+                trimmedStorageInfos);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 7d4cd7e..ab179b4 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
@@ -647,7 +647,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
                   .getStorageType()));
             }
             if (showReplicaDetails) {
-              LightWeightLinkedSet<Block> blocksExcess =
+              LightWeightLinkedSet<BlockInfo> blocksExcess =
                   bm.excessReplicateMap.get(dnDesc.getDatanodeUuid());
               Collection<DatanodeDescriptor> corruptReplicas =
                   bm.getCorruptReplicas(block.getLocalBlock());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
index 5126aa7..bae4f1d 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 class TestBlockInfo {
 
     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));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 396dff3..9e31670 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 @@ public class TestBlockManager {
     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 @@ public class TestBlockManager {
 
     for (DatanodeDescriptor dn : nodes) {
       for (DatanodeStorageInfo storage : dn.getStorageInfos()) {
-        blockInfo.addStorage(storage);
+        blockInfo.addStorage(storage, blockInfo);
       }
     }
     return blockInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
----------------------------------------------------------------------
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 1c3f075..c33667d 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 class TestNodeCount {
       DatanodeDescriptor nonExcessDN = null;
       for(DatanodeStorageInfo storage : bm.blocksMap.getStorages(block.getLocalBlock())) {
         final DatanodeDescriptor dn = storage.getDatanodeDescriptor();
-        Collection<Block> blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid());
+        Collection<BlockInfo> blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid());
         if (blocks == null || !blocks.contains(block.getLocalBlock()) ) {
           nonExcessDN = dn;
           break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
----------------------------------------------------------------------
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 2d7bb44..83b3aa0 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.DFSTestUtil;
 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.datanode.DataNodeTestUtils;
 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 class TestOverReplicatedBlocks {
       // All replicas for deletion should be scheduled on lastDN.
       // And should not actually be deleted, because lastDN does not heartbeat.
       namesystem.readLock();
-      Collection<Block> dnBlocks = 
+      Collection<BlockInfo> dnBlocks =
         namesystem.getBlockManager().excessReplicateMap.get(lastDNid);
       assertEquals("Replicas on node " + lastDNid + " should have been deleted",
           SMALL_FILE_LENGTH / SMALL_BLOCK_SIZE, dnBlocks.size());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/de480d6c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 2812957..44f0e65 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
@@ -1250,7 +1250,7 @@ public class TestReplicationPolicy {
     when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true);
     when(storage.addBlock(any(BlockInfo.class))).thenReturn
         (DatanodeStorageInfo.AddBlockResult.ADDED);
-    ucBlock.addStorage(storage);
+    ucBlock.addStorage(storage, ucBlock);
 
     when(mbc.setLastBlock((BlockInfo) any(), (DatanodeStorageInfo[]) any()))
     .thenReturn(ucBlock);


[25/30] hadoop git commit: HDFS-8651. Make hadoop-hdfs-project Native code -Wall-clean (Alan Burlison via Colin P. McCabe)

Posted by aw...@apache.org.
HDFS-8651. Make hadoop-hdfs-project Native code -Wall-clean (Alan Burlison via Colin P. McCabe)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1b764a01
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1b764a01
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1b764a01

Branch: refs/heads/HADOOP-12111
Commit: 1b764a01fd8010cf9660eb378977a1b2b81f330a
Parents: 40b2569
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Jun 26 10:21:40 2015 -0700
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Fri Jun 26 10:22:40 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c       | 4 ----
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b764a01/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 7b97f41..27e2e89 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -676,6 +676,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8546. Use try with resources in DataStorage and Storage. (wang)
 
+    HDFS-8651. Make hadoop-hdfs-project Native code -Wall-clean (Alan Burlison
+    via Colin P. McCabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1b764a01/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c
index 9ca2650..ca670ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/fuse-dfs/fuse_impls_open.c
@@ -24,9 +24,6 @@
 #include <stdio.h>
 #include <stdlib.h>
 
-static int get_hdfs_open_flags_from_info(hdfsFS fs, const char *path,
-                  int flags, int *outflags, const hdfsFileInfo *info);
-
 /**
  * Given a set of FUSE flags, determine the libhdfs flags we need.
  *
@@ -47,7 +44,6 @@ static int get_hdfs_open_flags_from_info(hdfsFS fs, const char *path,
  */
 static int64_t get_hdfs_open_flags(hdfsFS fs, const char *path, int flags)
 {
-  int hasContent;
   int64_t ret;
   hdfsFileInfo *info;
 


[30/30] hadoop git commit: Merge branch 'trunk' into HADOOP-12111

Posted by aw...@apache.org.
Merge branch 'trunk' into HADOOP-12111


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/084becdd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/084becdd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/084becdd

Branch: refs/heads/HADOOP-12111
Commit: 084becdd7293618cd774e4e0891acb58bc8f8133
Parents: 1b80406 fe6c1bd
Author: Allen Wittenauer <aw...@apache.org>
Authored: Tue Jun 30 11:51:13 2015 -0700
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Tue Jun 30 11:51:13 2015 -0700

----------------------------------------------------------------------
 .../server/AuthenticationFilter.java            |  28 +-
 hadoop-common-project/hadoop-common/CHANGES.txt |   8 +
 .../hadoop-common/HadoopCommon.cmake            | 207 ++++++
 .../hadoop-common/HadoopJNI.cmake               |  97 +++
 .../hadoop-common/src/CMakeLists.txt            | 366 +++++------
 .../apache/hadoop/ha/ZKFailoverController.java  |  61 +-
 .../hadoop/metrics2/impl/MetricsSystemImpl.java |   2 +-
 .../org/apache/hadoop/ha/MiniZKFCCluster.java   |  93 ++-
 .../hadoop/ha/TestZKFailoverController.java     |  32 +
 .../http/TestAuthenticationSessionCookie.java   | 187 ++++++
 .../apache/hadoop/http/TestHttpCookieFlag.java  |   2 +-
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  |  10 +
 .../org/apache/hadoop/hdfs/DFSUtilClient.java   |   7 +-
 .../hdfs/protocol/BlockStoragePolicy.java       |  12 +-
 .../hadoop/hdfs/protocol/CachePoolInfo.java     |   3 -
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   3 +-
 .../org/apache/hadoop/hdfs/web/TokenAspect.java |  16 +-
 .../hadoop/hdfs/web/URLConnectionFactory.java   |  13 +-
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  27 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  30 +
 .../bkjournal/TestBookKeeperHACheckpoints.java  |   7 +-
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   4 +
 .../java/org/apache/hadoop/hdfs/HAUtil.java     |  36 +-
 .../token/block/BlockTokenSecretManager.java    |  40 +-
 .../hdfs/server/blockmanagement/BlockInfo.java  |  12 +-
 .../blockmanagement/BlockInfoContiguous.java    |   2 +-
 .../BlockInfoUnderConstructionContiguous.java   |   2 +-
 .../server/blockmanagement/BlockManager.java    | 626 ++++++++++---------
 .../blockmanagement/DatanodeStorageInfo.java    |  15 +-
 .../hadoop/hdfs/server/common/Storage.java      |  24 +-
 .../hdfs/server/datanode/BPServiceActor.java    |   6 +-
 .../hadoop/hdfs/server/datanode/DataNode.java   |  17 +-
 .../hdfs/server/datanode/DataStorage.java       |  45 +-
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |   5 +-
 .../hdfs/server/namenode/CacheManager.java      |  24 +-
 .../hdfs/server/namenode/CheckpointConf.java    |  14 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  27 +-
 .../hdfs/server/namenode/ImageServlet.java      |  88 ++-
 .../server/namenode/NameNodeHttpServer.java     |   7 +-
 .../hdfs/server/namenode/NameNodeMXBean.java    |   5 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   2 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |  18 +-
 .../hdfs/server/namenode/TransferFsImage.java   |  47 +-
 .../server/namenode/ha/BootstrapStandby.java    |  94 +--
 .../hdfs/server/namenode/ha/EditLogTailer.java  | 162 ++++-
 .../server/namenode/ha/RemoteNameNodeInfo.java  | 133 ++++
 .../server/namenode/ha/StandbyCheckpointer.java | 182 ++++--
 .../hdfs/tools/DFSZKFailoverController.java     |  13 +
 .../offlineImageViewer/FSImageHandler.java      |  57 +-
 .../tools/offlineImageViewer/FSImageLoader.java |  75 ++-
 .../src/main/native/fuse-dfs/fuse_impls_open.c  |   4 -
 .../src/main/resources/hdfs-default.xml         |  20 +
 .../src/site/markdown/HdfsImageViewer.md        |   2 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  25 +
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  | 464 +++++++++-----
 .../apache/hadoop/hdfs/MiniDFSNNTopology.java   |  18 +-
 .../hadoop/hdfs/TestDFSUpgradeFromImage.java    |   6 +-
 .../apache/hadoop/hdfs/TestRollingUpgrade.java  | 170 +++--
 .../hadoop/hdfs/qjournal/MiniQJMHACluster.java  |  94 +--
 .../security/token/block/TestBlockToken.java    |  10 +-
 .../server/blockmanagement/TestBlockInfo.java   |   2 +-
 .../blockmanagement/TestBlockManager.java       |   4 +-
 .../server/blockmanagement/TestNodeCount.java   |   2 +-
 .../TestOverReplicatedBlocks.java               |   4 +-
 .../blockmanagement/TestReplicationPolicy.java  |   2 +-
 .../hdfs/server/datanode/DataNodeTestUtils.java |  11 +
 .../fsdataset/impl/TestRbwSpaceReservation.java |  28 +
 .../hdfs/server/namenode/TestBackupNode.java    |   2 +-
 .../server/namenode/TestCacheDirectives.java    |  25 +
 .../hdfs/server/namenode/TestCheckpoint.java    |   3 +-
 .../server/namenode/TestNameNodeMXBean.java     |   5 +-
 .../server/namenode/ha/HAStressTestHarness.java |  46 +-
 .../hdfs/server/namenode/ha/HATestUtil.java     |  60 +-
 .../namenode/ha/TestBootstrapStandby.java       | 176 +++---
 .../ha/TestBootstrapStandbyWithQJM.java         |  47 +-
 .../ha/TestDNFencingWithReplication.java        |   1 +
 .../server/namenode/ha/TestEditLogTailer.java   |  12 +-
 .../ha/TestFailoverWithBlockTokensEnabled.java  |  55 +-
 .../server/namenode/ha/TestHAConfiguration.java |  49 +-
 .../namenode/ha/TestPipelinesFailover.java      | 110 +++-
 .../namenode/ha/TestRemoteNameNodeInfo.java     |  61 ++
 .../namenode/ha/TestSeveralNameNodes.java       | 179 ++++++
 .../namenode/ha/TestStandbyCheckpoints.java     | 106 ++--
 .../TestOfflineImageViewerForXAttr.java         | 262 ++++++++
 .../hadoop/test/MiniDFSClusterManager.java      |   4 +
 .../src/test/resources/hadoop-0.23-reserved.tgz | Bin 4558 -> 5590 bytes
 .../src/test/resources/hadoop-1-reserved.tgz    | Bin 2572 -> 3348 bytes
 .../src/test/resources/hadoop-2-reserved.tgz    | Bin 2838 -> 3465 bytes
 .../src/test/resources/hadoop-22-dfs-dir.tgz    | Bin 318180 -> 413239 bytes
 .../src/test/resources/hadoop1-bbw.tgz          | Bin 40234 -> 43294 bytes
 .../src/test/resources/log4j.properties         |   2 +-
 hadoop-mapreduce-project/CHANGES.txt            |  10 +
 .../hadoop/mapreduce/task/reduce/Fetcher.java   |   1 +
 .../task/reduce/IFileWrappedMapOutput.java      |  10 +-
 .../mapreduce/task/reduce/LocalFetcher.java     |  15 +-
 .../mapreduce/v2/hs/server/HSAdminServer.java   |  43 +-
 .../v2/hs/server/TestHSAdminServer.java         |  55 ++
 .../hadoop/mapred/TestLocalJobSubmission.java   |   4 +-
 .../apache/hadoop/tools/DistCpConstants.java    |   2 +-
 .../apache/hadoop/tools/DistCpOptionSwitch.java |   5 +-
 .../org/apache/hadoop/tools/DistCpOptions.java  |   6 +-
 .../org/apache/hadoop/tools/OptionsParser.java  |   2 +-
 .../apache/hadoop/tools/mapred/CopyMapper.java  |  17 +-
 .../tools/mapred/RetriableFileCopyCommand.java  |   2 +-
 .../hadoop/tools/util/ThrottledInputStream.java |   6 +-
 .../apache/hadoop/tools/TestOptionsParser.java  |  16 +-
 hadoop-yarn-project/CHANGES.txt                 |  23 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  10 +
 .../impl/pb/SerializedExceptionPBImpl.java      |  26 +-
 .../src/main/resources/yarn-default.xml         |  12 +
 .../impl/pb/TestSerializedExceptionPBImpl.java  |  11 +
 .../server/timeline/TimelineDataManager.java    | 123 ++++
 .../timeline/TimelineDataManagerMetrics.java    | 174 ++++++
 .../TestApplicationHistoryClientService.java    |   1 +
 ...pplicationHistoryManagerOnTimelineStore.java |   1 +
 .../webapp/TestAHSWebServices.java              |   1 +
 .../timeline/TestTimelineDataManager.java       |   1 +
 .../server/utils/YarnServerBuilderUtils.java    |  11 +-
 .../nodemanager/LocalDirsHandlerService.java    |  24 +
 .../launcher/RecoveredContainerLaunch.java      |   3 +-
 .../localizer/ResourceLocalizationService.java  |   2 +-
 .../logaggregation/AppLogAggregatorImpl.java    |   4 +-
 .../nodemanager/webapp/ContainerLogsUtils.java  |   2 +-
 .../TestLogAggregationService.java              |  54 +-
 .../webapp/TestContainerLogsPage.java           |  22 +-
 .../resourcemanager/ResourceTrackerService.java |  23 +-
 .../amlauncher/ApplicationMasterLauncher.java   |  30 +-
 .../scheduler/fair/FairScheduler.java           |  11 +-
 .../server/resourcemanager/TestRMRestart.java   |   8 +-
 129 files changed, 4335 insertions(+), 1500 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/084becdd/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------


[20/30] hadoop git commit: MAPREDUCE-6413. TestLocalJobSubmission is failing with unknown host. Contributed by zhihai xu

Posted by aw...@apache.org.
MAPREDUCE-6413. TestLocalJobSubmission is failing with unknown host. Contributed by zhihai xu


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/aa5b15b0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/aa5b15b0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/aa5b15b0

Branch: refs/heads/HADOOP-12111
Commit: aa5b15b03be61ebb76a226e0de485d5228c8e3d0
Parents: 67a62da
Author: Jason Lowe <jl...@apache.org>
Authored: Thu Jun 25 19:50:07 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Thu Jun 25 19:50:07 2015 +0000

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                             | 3 +++
 .../java/org/apache/hadoop/mapred/TestLocalJobSubmission.java    | 4 +++-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa5b15b0/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 6c65032..4df3b44 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -509,6 +509,9 @@ Release 2.8.0 - UNRELEASED
     too early (Brahma Reddy Battula, Akira AJISAKA, and Gera Shegalov via
     jlowe)
 
+    MAPREDUCE-6413. TestLocalJobSubmission is failing with unknown host
+    (zhihai xu via jlowe)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/aa5b15b0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
index e06551a..d73ee4b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLocalJobSubmission.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.SleepJob;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
@@ -63,7 +64,8 @@ public class TestLocalJobSubmission {
     Path jarPath = makeJar(new Path(TEST_ROOT_DIR, "test.jar"));
 
     Configuration conf = new Configuration();
-    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://testcluster");
+    conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost:9000");
+    conf.set(MRConfig.FRAMEWORK_NAME, "local");
     final String[] args = {
         "-jt" , "local", "-libjars", jarPath.toString(),
         "-m", "1", "-r", "1", "-mt", "1", "-rt", "1"


[17/30] hadoop git commit: YARN-3745. SerializedException should also try to instantiate internal exception with the default constructor. Contributed by Lavkesh Lahngir.

Posted by aw...@apache.org.
YARN-3745. SerializedException should also try to instantiate internal
exception with the default constructor. Contributed by Lavkesh Lahngir.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b381f88c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b381f88c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b381f88c

Branch: refs/heads/HADOOP-12111
Commit: b381f88c71d18497deb35039372b1e9715d2c038
Parents: 57f1a01
Author: Devaraj K <de...@apache.org>
Authored: Thu Jun 25 17:38:11 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Thu Jun 25 17:38:11 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../impl/pb/SerializedExceptionPBImpl.java      | 26 ++++++++++++++++----
 .../impl/pb/TestSerializedExceptionPBImpl.java  | 11 +++++++++
 3 files changed, 35 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b381f88c/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3d608fd..7dc02cd 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -556,6 +556,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3826. Race condition in ResourceTrackerService leads to
     wrong diagnostics messages. (Chengbing Liu via devaraj)
 
+    YARN-3745. SerializedException should also try to instantiate internal
+    exception with the default constructor. (Lavkesh Lahngir via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b381f88c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java
index e683fd5..fd9e170 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java
@@ -158,15 +158,31 @@ public class SerializedExceptionPBImpl extends SerializedException {
     viaProto = false;
   }
 
+  private static <T extends Throwable> T instantiateExceptionImpl(
+      String message, Class<? extends T> cls, Throwable cause)
+      throws NoSuchMethodException, InstantiationException,
+      IllegalAccessException, InvocationTargetException {
+    Constructor<? extends T> cn;
+    T ex = null;
+    cn =
+        cls.getConstructor(message == null ? new Class[0]
+            : new Class[] {String.class});
+    cn.setAccessible(true);
+    ex = message == null ? cn.newInstance() : cn.newInstance(message);
+    ex.initCause(cause);
+    return ex;
+  }
+
   private static <T extends Throwable> T instantiateException(
       Class<? extends T> cls, String message, Throwable cause) {
-    Constructor<? extends T> cn;
     T ex = null;
     try {
-      cn = cls.getConstructor(String.class);
-      cn.setAccessible(true);
-      ex = cn.newInstance(message);
-      ex.initCause(cause);
+      // Try constructor with String argument, if it fails, try default.
+      try {
+        ex = instantiateExceptionImpl(message, cls, cause);
+      } catch (NoSuchMethodException e) {
+        ex = instantiateExceptionImpl(null, cls, cause);
+      }
     } catch (SecurityException e) {
       throw new YarnRuntimeException(e);
     } catch (NoSuchMethodException e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b381f88c/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestSerializedExceptionPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestSerializedExceptionPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestSerializedExceptionPBImpl.java
index b011e3e..ac7e40e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestSerializedExceptionPBImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/records/impl/pb/TestSerializedExceptionPBImpl.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.api.records.impl.pb;
 
+import java.nio.channels.ClosedChannelException;
+
 import org.junit.Assert;
 import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
@@ -55,6 +57,15 @@ public class TestSerializedExceptionPBImpl {
   }
 
   @Test
+  public void testDeserializeWithDefaultConstructor() {
+    // Init SerializedException with an Exception with default constructor.
+    ClosedChannelException ex = new ClosedChannelException();
+    SerializedExceptionPBImpl pb = new SerializedExceptionPBImpl();
+    pb.init(ex);
+    Assert.assertEquals(ex.getClass(), pb.deSerialize().getClass());
+  }
+
+  @Test
   public void testBeforeInit() throws Exception {
     SerializedExceptionProto defaultProto =
         SerializedExceptionProto.newBuilder().build();


[04/30] hadoop git commit: HDFS-6440. Support more than 2 NameNodes. Contributed by Jesse Yates.

Posted by aw...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index fdbacdc..0a21886 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -62,6 +62,8 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -445,7 +447,7 @@ public class MiniDFSCluster {
     final int numNameNodes = builder.nnTopology.countNameNodes();
     LOG.info("starting cluster: numNameNodes=" + numNameNodes
         + ", numDataNodes=" + builder.numDataNodes);
-    nameNodes = new NameNodeInfo[numNameNodes];
+
     this.storagesPerDatanode = builder.storagesPerDatanode;
 
     // Duplicate the storageType setting for each DN.
@@ -515,7 +517,7 @@ public class MiniDFSCluster {
   }
 
   private Configuration conf;
-  private NameNodeInfo[] nameNodes;
+  private Multimap<String, NameNodeInfo> namenodes = ArrayListMultimap.create();
   protected int numDataNodes;
   protected final List<DataNodeProperties> dataNodes =
                          new ArrayList<DataNodeProperties>();
@@ -539,10 +541,10 @@ public class MiniDFSCluster {
    * Stores the information related to a namenode in the cluster
    */
   public static class NameNodeInfo {
-    final NameNode nameNode;
-    final Configuration conf;
-    final String nameserviceId;
-    final String nnId;
+    public NameNode nameNode;
+    Configuration conf;
+    String nameserviceId;
+    String nnId;
     StartupOption startOpt;
     NameNodeInfo(NameNode nn, String nameserviceId, String nnId,
         StartupOption startOpt, Configuration conf) {
@@ -563,7 +565,6 @@ public class MiniDFSCluster {
    * without a name node (ie when the name node is started elsewhere).
    */
   public MiniDFSCluster() {
-    nameNodes = new NameNodeInfo[0]; // No namenode in the cluster
     storagesPerDatanode = DEFAULT_STORAGES_PER_DATANODE;
     synchronized (MiniDFSCluster.class) {
       instanceId = instanceCount++;
@@ -740,7 +741,6 @@ public class MiniDFSCluster {
                         StartupOption operation,
                         String[] racks, String hosts[],
                         long[] simulatedCapacities) throws IOException {
-    this.nameNodes = new NameNodeInfo[1]; // Single namenode in the cluster
     this.storagesPerDatanode = DEFAULT_STORAGES_PER_DATANODE;
     initMiniDFSCluster(conf, numDataNodes, null, format,
                        manageNameDfsDirs, true, manageDataDfsDirs, manageDataDfsDirs,
@@ -814,7 +814,7 @@ public class MiniDFSCluster {
         createNameNodesAndSetConf(
             nnTopology, manageNameDfsDirs, manageNameDfsSharedDirs,
             enableManagedDfsDirsRedundancy,
-            format, startOpt, clusterId, conf);
+            format, startOpt, clusterId);
       } catch (IOException ioe) {
         LOG.error("IOE creating namenodes. Permissions dump:\n" +
             createPermissionsDiagnosisString(data_dir), ioe);
@@ -871,7 +871,127 @@ public class MiniDFSCluster {
   private void createNameNodesAndSetConf(MiniDFSNNTopology nnTopology,
       boolean manageNameDfsDirs, boolean manageNameDfsSharedDirs,
       boolean enableManagedDfsDirsRedundancy, boolean format,
+      StartupOption operation, String clusterId) throws IOException {
+    // do the basic namenode configuration
+    configureNameNodes(nnTopology, federation, conf);
+
+    int nnCounter = 0;
+    int nsCounter = 0;
+    // configure each NS independently
+    for (MiniDFSNNTopology.NSConf nameservice : nnTopology.getNameservices()) {
+      configureNameService(nameservice, nsCounter++, manageNameDfsSharedDirs,
+          manageNameDfsDirs, enableManagedDfsDirsRedundancy,
+          format, operation, clusterId, nnCounter);
+      nnCounter += nameservice.getNNs().size();
+    }
+  }
+
+  /**
+   * Do the rest of the NN configuration for things like shared edits,
+   * as well as directory formatting, etc. for a single nameservice
+   * @param nnCounter the count of the number of namenodes already configured/started. Also,
+   *                  acts as the <i>index</i> to the next NN to start (since indicies start at 0).
+   * @throws IOException
+   */
+  private void configureNameService(MiniDFSNNTopology.NSConf nameservice, int nsCounter,
+      boolean manageNameDfsSharedDirs, boolean manageNameDfsDirs, boolean
+      enableManagedDfsDirsRedundancy, boolean format,
       StartupOption operation, String clusterId,
+      final int nnCounter) throws IOException{
+    String nsId = nameservice.getId();
+    String lastDefaultFileSystem = null;
+
+    // If HA is enabled on this nameservice, enumerate all the namenodes
+    // in the configuration. Also need to set a shared edits dir
+    int numNNs = nameservice.getNNs().size();
+    if (numNNs > 1 && manageNameDfsSharedDirs) {
+      URI sharedEditsUri = getSharedEditsDir(nnCounter, nnCounter + numNNs - 1);
+      conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY, sharedEditsUri.toString());
+      // Clean out the shared edits dir completely, including all subdirectories.
+      FileUtil.fullyDelete(new File(sharedEditsUri));
+    }
+
+    // Now format first NN and copy the storage directory from that node to the others.
+    int nnIndex = nnCounter;
+    Collection<URI> prevNNDirs = null;
+    for (NNConf nn : nameservice.getNNs()) {
+      initNameNodeConf(conf, nsId, nsCounter, nn.getNnId(), manageNameDfsDirs,
+          manageNameDfsDirs,  nnIndex);
+      Collection<URI> namespaceDirs = FSNamesystem.getNamespaceDirs(conf);
+      if (format) {
+        // delete the existing namespaces
+        for (URI nameDirUri : namespaceDirs) {
+          File nameDir = new File(nameDirUri);
+          if (nameDir.exists() && !FileUtil.fullyDelete(nameDir)) {
+            throw new IOException("Could not fully delete " + nameDir);
+          }
+        }
+
+        // delete the checkpoint directories, if they exist
+        Collection<URI> checkpointDirs = Util.stringCollectionAsURIs(conf
+            .getTrimmedStringCollection(DFS_NAMENODE_CHECKPOINT_DIR_KEY));
+        for (URI checkpointDirUri : checkpointDirs) {
+          File checkpointDir = new File(checkpointDirUri);
+          if (checkpointDir.exists() && !FileUtil.fullyDelete(checkpointDir)) {
+            throw new IOException("Could not fully delete " + checkpointDir);
+          }
+        }
+      }
+
+      boolean formatThisOne = format;
+      // if we are looking at not the first NN
+      if (nnIndex++ > nnCounter && format) {
+        // Don't format the second, third, etc NN in an HA setup - that
+        // would result in it having a different clusterID,
+        // block pool ID, etc. Instead, copy the name dirs
+        // from the previous one.
+        formatThisOne = false;
+        assert (null != prevNNDirs);
+        copyNameDirs(prevNNDirs, namespaceDirs, conf);
+      }
+
+      if (formatThisOne) {
+        // Allow overriding clusterID for specific NNs to test
+        // misconfiguration.
+        if (nn.getClusterId() == null) {
+          StartupOption.FORMAT.setClusterId(clusterId);
+        } else {
+          StartupOption.FORMAT.setClusterId(nn.getClusterId());
+        }
+        DFSTestUtil.formatNameNode(conf);
+      }
+      prevNNDirs = namespaceDirs;
+    }
+
+    // create all the namenodes in the namespace
+    nnIndex = nnCounter;
+    for (NNConf nn : nameservice.getNNs()) {
+      initNameNodeConf(conf, nsId, nsCounter, nn.getNnId(), manageNameDfsDirs,
+          enableManagedDfsDirsRedundancy, nnIndex++);
+      NameNodeInfo info = createNameNode(conf, false, operation,
+          clusterId, nsId, nn.getNnId());
+
+      // Record the last namenode uri
+      if (info != null && info.conf != null) {
+        lastDefaultFileSystem =
+            info.conf.get(FS_DEFAULT_NAME_KEY);
+      }
+    }
+    if (!federation && lastDefaultFileSystem != null) {
+      // Set the default file system to the actual bind address of NN.
+      conf.set(FS_DEFAULT_NAME_KEY, lastDefaultFileSystem);
+    }
+  }
+
+  /**
+   * Do the basic NN configuration for the topology. Does not configure things like the shared
+   * edits directories
+   * @param nnTopology
+   * @param federation
+   * @param conf
+   * @throws IOException
+   */
+  public static void configureNameNodes(MiniDFSNNTopology nnTopology, boolean federation,
       Configuration conf) throws IOException {
     Preconditions.checkArgument(nnTopology.countNameNodes() > 0,
         "empty NN topology: no namenodes specified!");
@@ -884,22 +1004,21 @@ public class MiniDFSCluster {
       // NN is started.
       conf.set(FS_DEFAULT_NAME_KEY, "hdfs://127.0.0.1:" + onlyNN.getIpcPort());
     }
-    
+
     List<String> allNsIds = Lists.newArrayList();
     for (MiniDFSNNTopology.NSConf nameservice : nnTopology.getNameservices()) {
       if (nameservice.getId() != null) {
         allNsIds.add(nameservice.getId());
       }
     }
+
     if (!allNsIds.isEmpty()) {
       conf.set(DFS_NAMESERVICES, Joiner.on(",").join(allNsIds));
     }
-    
-    int nnCounter = 0;
+
     for (MiniDFSNNTopology.NSConf nameservice : nnTopology.getNameservices()) {
       String nsId = nameservice.getId();
-      String lastDefaultFileSystem = null;
-      
+
       Preconditions.checkArgument(
           !federation || nsId != null,
           "if there is more than one NS, they must have names");
@@ -918,85 +1037,10 @@ public class MiniDFSCluster {
       // If HA is enabled on this nameservice, enumerate all the namenodes
       // in the configuration. Also need to set a shared edits dir
       if (nnIds.size() > 1) {
-        conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, nameservice.getId()),
-            Joiner.on(",").join(nnIds));
-        if (manageNameDfsSharedDirs) {
-          URI sharedEditsUri = getSharedEditsDir(nnCounter, nnCounter+nnIds.size()-1); 
-          conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY, sharedEditsUri.toString());
-          // Clean out the shared edits dir completely, including all subdirectories.
-          FileUtil.fullyDelete(new File(sharedEditsUri));
-        }
-      }
-
-      // Now format first NN and copy the storage directory from that node to the others.
-      int i = 0;
-      Collection<URI> prevNNDirs = null;
-      int nnCounterForFormat = nnCounter;
-      for (NNConf nn : nameservice.getNNs()) {
-        initNameNodeConf(conf, nsId, nn.getNnId(), manageNameDfsDirs, manageNameDfsDirs,
-            nnCounterForFormat);
-        Collection<URI> namespaceDirs = FSNamesystem.getNamespaceDirs(conf);
-        if (format) {
-          for (URI nameDirUri : namespaceDirs) {
-            File nameDir = new File(nameDirUri);
-            if (nameDir.exists() && !FileUtil.fullyDelete(nameDir)) {
-              throw new IOException("Could not fully delete " + nameDir);
-            }
-          }
-          Collection<URI> checkpointDirs = Util.stringCollectionAsURIs(conf
-              .getTrimmedStringCollection(DFS_NAMENODE_CHECKPOINT_DIR_KEY));
-          for (URI checkpointDirUri : checkpointDirs) {
-            File checkpointDir = new File(checkpointDirUri);
-            if (checkpointDir.exists() && !FileUtil.fullyDelete(checkpointDir)) {
-              throw new IOException("Could not fully delete " + checkpointDir);
-            }
-          }
-        }
-        
-        boolean formatThisOne = format;
-        if (format && i++ > 0) {
-          // Don't format the second NN in an HA setup - that
-          // would result in it having a different clusterID,
-          // block pool ID, etc. Instead, copy the name dirs
-          // from the first one.
-          formatThisOne = false;
-          assert (null != prevNNDirs);
-          copyNameDirs(prevNNDirs, namespaceDirs, conf);
-        }
-        
-        nnCounterForFormat++;
-        if (formatThisOne) {
-          // Allow overriding clusterID for specific NNs to test
-          // misconfiguration.
-          if (nn.getClusterId() == null) {
-            StartupOption.FORMAT.setClusterId(clusterId);
-          } else {
-            StartupOption.FORMAT.setClusterId(nn.getClusterId());
-          }
-          DFSTestUtil.formatNameNode(conf);
-        }
-        prevNNDirs = namespaceDirs;
-      }
-
-      // Start all Namenodes
-      for (NNConf nn : nameservice.getNNs()) {
-        initNameNodeConf(conf, nsId, nn.getNnId(), manageNameDfsDirs,
-            enableManagedDfsDirsRedundancy, nnCounter);
-        createNameNode(nnCounter, conf, numDataNodes, false, operation,
-            clusterId, nsId, nn.getNnId());
-        // Record the last namenode uri
-        if (nameNodes[nnCounter] != null && nameNodes[nnCounter].conf != null) {
-          lastDefaultFileSystem =
-              nameNodes[nnCounter].conf.get(FS_DEFAULT_NAME_KEY);
-        }
-        nnCounter++;
-      }
-      if (!federation && lastDefaultFileSystem != null) {
-        // Set the default file system to the actual bind address of NN.
-        conf.set(FS_DEFAULT_NAME_KEY, lastDefaultFileSystem);
+        conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, nameservice.getId()), Joiner
+            .on(",").join(nnIds));
       }
     }
-
   }
   
   public URI getSharedEditsDir(int minNN, int maxNN) throws IOException {
@@ -1010,39 +1054,92 @@ public class MiniDFSCluster {
   }
   
   public NameNodeInfo[] getNameNodeInfos() {
-    return this.nameNodes;
+    return this.namenodes.values().toArray(new NameNodeInfo[0]);
   }
 
-  private void initNameNodeConf(Configuration conf,
-      String nameserviceId, String nnId,
-      boolean manageNameDfsDirs, boolean enableManagedDfsDirsRedundancy,
-      int nnIndex) throws IOException {
+  /**
+   * @param nsIndex index of the namespace id to check
+   * @return all the namenodes bound to the given namespace index
+   */
+  public NameNodeInfo[] getNameNodeInfos(int nsIndex) {
+    int i = 0;
+    for (String ns : this.namenodes.keys()) {
+      if (i++ == nsIndex) {
+        return this.namenodes.get(ns).toArray(new NameNodeInfo[0]);
+      }
+    }
+    return null;
+  }
+
+  /**
+   * @param nameservice id of nameservice to read
+   * @return all the namenodes bound to the given namespace index
+   */
+  public NameNodeInfo[] getNameNodeInfos(String nameservice) {
+    for (String ns : this.namenodes.keys()) {
+      if (nameservice.equals(ns)) {
+        return this.namenodes.get(ns).toArray(new NameNodeInfo[0]);
+      }
+    }
+    return null;
+  }
+
+
+  private void initNameNodeConf(Configuration conf, String nameserviceId, int nsIndex, String nnId,
+      boolean manageNameDfsDirs, boolean enableManagedDfsDirsRedundancy, int nnIndex)
+      throws IOException {
     if (nameserviceId != null) {
       conf.set(DFS_NAMESERVICE_ID, nameserviceId);
     }
     if (nnId != null) {
       conf.set(DFS_HA_NAMENODE_ID_KEY, nnId);
     }
-    
     if (manageNameDfsDirs) {
       if (enableManagedDfsDirsRedundancy) {
-        conf.set(DFS_NAMENODE_NAME_DIR_KEY,
-            fileAsURI(new File(base_dir, "name" + (2*nnIndex + 1)))+","+
-            fileAsURI(new File(base_dir, "name" + (2*nnIndex + 2))));
-        conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,
-            fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 1)))+","+
-            fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 2))));
+        File[] files = getNameNodeDirectory(nsIndex, nnIndex);
+        conf.set(DFS_NAMENODE_NAME_DIR_KEY, fileAsURI(files[0]) + "," + fileAsURI(files[1]));
+        files = getCheckpointDirectory(nsIndex, nnIndex);
+        conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY, fileAsURI(files[0]) + "," + fileAsURI(files[1]));
       } else {
-        conf.set(DFS_NAMENODE_NAME_DIR_KEY,
-            fileAsURI(new File(base_dir, "name" + (2*nnIndex + 1))).
-              toString());
-        conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY,
-            fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 1))).
-              toString());
+        File[] files = getNameNodeDirectory(nsIndex, nnIndex);
+        conf.set(DFS_NAMENODE_NAME_DIR_KEY, fileAsURI(files[0]).toString());
+        files = getCheckpointDirectory(nsIndex, nnIndex);
+        conf.set(DFS_NAMENODE_CHECKPOINT_DIR_KEY, fileAsURI(files[0]).toString());
       }
     }
   }
 
+  private File[] getNameNodeDirectory(int nameserviceIndex, int nnIndex) {
+    return getNameNodeDirectory(base_dir, nameserviceIndex, nnIndex);
+  }
+
+  public static File[] getNameNodeDirectory(String base_dir, int nsIndex, int nnIndex) {
+    return getNameNodeDirectory(new File(base_dir), nsIndex, nnIndex);
+  }
+
+  public static File[] getNameNodeDirectory(File base_dir, int nsIndex, int nnIndex) {
+    File[] files = new File[2];
+    files[0] = new File(base_dir, "name-" + nsIndex + "-" + (2 * nnIndex + 1));
+    files[1] = new File(base_dir, "name-" + nsIndex + "-" + (2 * nnIndex + 2));
+    return files;
+  }
+
+  public File[] getCheckpointDirectory(int nsIndex, int nnIndex) {
+    return getCheckpointDirectory(base_dir, nsIndex, nnIndex);
+  }
+
+  public static File[] getCheckpointDirectory(String base_dir, int nsIndex, int nnIndex) {
+    return getCheckpointDirectory(new File(base_dir), nsIndex, nnIndex);
+  }
+
+  public static File[] getCheckpointDirectory(File base_dir, int nsIndex, int nnIndex) {
+    File[] files = new File[2];
+    files[0] = new File(base_dir, "namesecondary-" + nsIndex + "-" + (2 * nnIndex + 1));
+    files[1] = new File(base_dir, "namesecondary-" + nsIndex + "-" + (2 * nnIndex + 2));
+    return files;
+  }
+
+
   public static void copyNameDirs(Collection<URI> srcDirs, Collection<URI> dstDirs,
       Configuration dstConf) throws IOException {
     URI srcDir = Lists.newArrayList(srcDirs).get(0);
@@ -1094,12 +1191,9 @@ public class MiniDFSCluster {
             new String[] {} : new String[] {operation.getName()};
     return args;
   }
-  
-  private void createNameNode(int nnIndex, Configuration conf,
-      int numDataNodes, boolean format, StartupOption operation,
-      String clusterId, String nameserviceId,
-      String nnId)
-      throws IOException {
+
+  private NameNodeInfo createNameNode(Configuration conf, boolean format, StartupOption operation,
+      String clusterId, String nameserviceId, String nnId) throws IOException {
     // Format and clean out DataNode directories
     if (format) {
       DFSTestUtil.formatNameNode(conf);
@@ -1113,7 +1207,7 @@ public class MiniDFSCluster {
     String[] args = createArgs(operation);
     NameNode nn =  NameNode.createNameNode(args, conf);
     if (operation == StartupOption.RECOVER) {
-      return;
+      return null;
     }
     
     // After the NN has started, set back the bound ports into
@@ -1131,14 +1225,17 @@ public class MiniDFSCluster {
 
     DFSUtil.setGenericConf(conf, nameserviceId, nnId,
         DFS_NAMENODE_HTTP_ADDRESS_KEY);
-    nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId,
+    NameNodeInfo info = new NameNodeInfo(nn, nameserviceId, nnId,
         operation, new Configuration(conf));
+    namenodes.put(nameserviceId, info);
+
     // Restore the default fs name
     if (originalDefaultFs == null) {
       conf.set(FS_DEFAULT_NAME_KEY, "");
     } else {
       conf.set(FS_DEFAULT_NAME_KEY, originalDefaultFs);
     }
+    return info;
   }
 
   /**
@@ -1154,7 +1251,7 @@ public class MiniDFSCluster {
    */
   public URI getURI(int nnIndex) {
     String hostPort =
-        nameNodes[nnIndex].nameNode.getNameNodeAddressHostPortString();
+        getNN(nnIndex).nameNode.getNameNodeAddressHostPortString();
     URI uri = null;
     try {
       uri = new URI("hdfs://" + hostPort);
@@ -1172,9 +1269,21 @@ public class MiniDFSCluster {
    * @return Configuration of for the given namenode
    */
   public Configuration getConfiguration(int nnIndex) {
-    return nameNodes[nnIndex].conf;
+    return getNN(nnIndex).conf;
   }
 
+  private NameNodeInfo getNN(int nnIndex) {
+    int count = 0;
+    for (NameNodeInfo nn : namenodes.values()) {
+      if (count == nnIndex) {
+        return nn;
+      }
+      count++;
+    }
+    return null;
+  }
+
+
   /**
    * wait for the given namenode to get out of safemode.
    */
@@ -1593,7 +1702,7 @@ public class MiniDFSCluster {
    * @throws Exception
    */
   public void finalizeCluster(int nnIndex, Configuration conf) throws Exception {
-    finalizeNamenode(nameNodes[nnIndex].nameNode, nameNodes[nnIndex].conf);
+    finalizeNamenode(getNN(nnIndex).nameNode, getNN(nnIndex).conf);
   }
 
   /**
@@ -1604,7 +1713,7 @@ public class MiniDFSCluster {
    * @throws IllegalStateException if the Namenode is not running.
    */
   public void finalizeCluster(Configuration conf) throws Exception {
-    for (NameNodeInfo nnInfo : nameNodes) {
+    for (NameNodeInfo nnInfo : namenodes.values()) {
       if (nnInfo == null) {
         throw new IllegalStateException("Attempting to finalize "
             + "Namenode but it is not running");
@@ -1612,9 +1721,9 @@ public class MiniDFSCluster {
       finalizeNamenode(nnInfo.nameNode, nnInfo.conf);
     }
   }
-  
+
   public int getNumNameNodes() {
-    return nameNodes.length;
+    return namenodes.size();
   }
   
   /**
@@ -1644,7 +1753,7 @@ public class MiniDFSCluster {
    * Gets the NameNode for the index.  May be null.
    */
   public NameNode getNameNode(int nnIndex) {
-    return nameNodes[nnIndex].nameNode;
+    return getNN(nnIndex).nameNode;
   }
   
   /**
@@ -1653,11 +1762,11 @@ public class MiniDFSCluster {
    */
   public FSNamesystem getNamesystem() {
     checkSingleNameNode();
-    return NameNodeAdapter.getNamesystem(nameNodes[0].nameNode);
+    return NameNodeAdapter.getNamesystem(getNN(0).nameNode);
   }
-  
+
   public FSNamesystem getNamesystem(int nnIndex) {
-    return NameNodeAdapter.getNamesystem(nameNodes[nnIndex].nameNode);
+    return NameNodeAdapter.getNamesystem(getNN(nnIndex).nameNode);
   }
 
   /**
@@ -1697,14 +1806,14 @@ public class MiniDFSCluster {
    * caller supplied port is not necessarily the actual port used.
    */     
   public int getNameNodePort(int nnIndex) {
-    return nameNodes[nnIndex].nameNode.getNameNodeAddress().getPort();
+    return getNN(nnIndex).nameNode.getNameNodeAddress().getPort();
   }
 
   /**
    * @return the service rpc port used by the NameNode at the given index.
    */     
   public int getNameNodeServicePort(int nnIndex) {
-    return nameNodes[nnIndex].nameNode.getServiceRpcAddress().getPort();
+    return getNN(nnIndex).nameNode.getServiceRpcAddress().getPort();
   }
     
   /**
@@ -1745,7 +1854,7 @@ public class MiniDFSCluster {
       fileSystems.clear();
     }
     shutdownDataNodes();
-    for (NameNodeInfo nnInfo : nameNodes) {
+    for (NameNodeInfo nnInfo : namenodes.values()) {
       if (nnInfo == null) continue;
       NameNode nameNode = nnInfo.nameNode;
       if (nameNode != null) {
@@ -1781,7 +1890,7 @@ public class MiniDFSCluster {
    * Shutdown all the namenodes.
    */
   public synchronized void shutdownNameNodes() {
-    for (int i = 0; i < nameNodes.length; i++) {
+    for (int i = 0; i < namenodes.size(); i++) {
       shutdownNameNode(i);
     }
   }
@@ -1790,13 +1899,15 @@ public class MiniDFSCluster {
    * Shutdown the namenode at a given index.
    */
   public synchronized void shutdownNameNode(int nnIndex) {
-    NameNode nn = nameNodes[nnIndex].nameNode;
+    NameNodeInfo info = getNN(nnIndex);
+    NameNode nn = info.nameNode;
     if (nn != null) {
       LOG.info("Shutting down the namenode");
       nn.stop();
       nn.join();
-      Configuration conf = nameNodes[nnIndex].conf;
-      nameNodes[nnIndex] = new NameNodeInfo(null, null, null, null, conf);
+      info.nnId = null;
+      info.nameNode = null;
+      info.nameserviceId = null;
     }
   }
   
@@ -1804,7 +1915,7 @@ public class MiniDFSCluster {
    * Restart all namenodes.
    */
   public synchronized void restartNameNodes() throws IOException {
-    for (int i = 0; i < nameNodes.length; i++) {
+    for (int i = 0; i < namenodes.size(); i++) {
       restartNameNode(i, false);
     }
     waitActive();
@@ -1840,19 +1951,19 @@ public class MiniDFSCluster {
    */
   public synchronized void restartNameNode(int nnIndex, boolean waitActive,
       String... args) throws IOException {
-    String nameserviceId = nameNodes[nnIndex].nameserviceId;
-    String nnId = nameNodes[nnIndex].nnId;
-    StartupOption startOpt = nameNodes[nnIndex].startOpt;
-    Configuration conf = nameNodes[nnIndex].conf;
+    NameNodeInfo info = getNN(nnIndex);
+    StartupOption startOpt = info.startOpt;
+
     shutdownNameNode(nnIndex);
     if (args.length != 0) {
       startOpt = null;
     } else {
       args = createArgs(startOpt);
     }
-    NameNode nn = NameNode.createNameNode(args, conf);
-    nameNodes[nnIndex] = new NameNodeInfo(nn, nameserviceId, nnId, startOpt,
-        conf);
+
+    NameNode nn = NameNode.createNameNode(args, info.conf);
+    info.nameNode = nn;
+    info.setStartOpt(startOpt);
     if (waitActive) {
       waitClusterUp();
       LOG.info("Restarted the namenode");
@@ -2124,7 +2235,7 @@ public class MiniDFSCluster {
    * or if waiting for safe mode is disabled.
    */
   public boolean isNameNodeUp(int nnIndex) {
-    NameNode nameNode = nameNodes[nnIndex].nameNode;
+    NameNode nameNode = getNN(nnIndex).nameNode;
     if (nameNode == null) {
       return false;
     }
@@ -2142,7 +2253,7 @@ public class MiniDFSCluster {
    * Returns true if all the NameNodes are running and is out of Safe Mode.
    */
   public boolean isClusterUp() {
-    for (int index = 0; index < nameNodes.length; index++) {
+    for (int index = 0; index < namenodes.size(); index++) {
       if (!isNameNodeUp(index)) {
         return false;
       }
@@ -2172,15 +2283,13 @@ public class MiniDFSCluster {
     checkSingleNameNode();
     return getFileSystem(0);
   }
-  
+
   /**
    * Get a client handle to the DFS cluster for the namenode at given index.
    */
   public DistributedFileSystem getFileSystem(int nnIndex) throws IOException {
-    DistributedFileSystem dfs = (DistributedFileSystem) FileSystem.get(
-        getURI(nnIndex), nameNodes[nnIndex].conf);
-    fileSystems.add(dfs);
-    return dfs;
+    return (DistributedFileSystem) addFileSystem(FileSystem.get(getURI(nnIndex),
+        getNN(nnIndex).conf));
   }
 
   /**
@@ -2188,17 +2297,20 @@ public class MiniDFSCluster {
    * This simulating different threads working on different FileSystem instances.
    */
   public FileSystem getNewFileSystemInstance(int nnIndex) throws IOException {
-    FileSystem dfs = FileSystem.newInstance(getURI(nnIndex), nameNodes[nnIndex].conf);
-    fileSystems.add(dfs);
-    return dfs;
+    return addFileSystem(FileSystem.newInstance(getURI(nnIndex), getNN(nnIndex).conf));
   }
-  
+
+  private <T extends FileSystem> T addFileSystem(T fs) {
+    fileSystems.add(fs);
+    return fs;
+  }
+
   /**
    * @return a http URL
    */
   public String getHttpUri(int nnIndex) {
     return "http://"
-        + nameNodes[nnIndex].conf
+        + getNN(nnIndex).conf
             .get(DFS_NAMENODE_HTTP_ADDRESS_KEY);
   }
 
@@ -2206,14 +2318,14 @@ public class MiniDFSCluster {
    * Get the directories where the namenode stores its image.
    */
   public Collection<URI> getNameDirs(int nnIndex) {
-    return FSNamesystem.getNamespaceDirs(nameNodes[nnIndex].conf);
+    return FSNamesystem.getNamespaceDirs(getNN(nnIndex).conf);
   }
 
   /**
    * Get the directories where the namenode stores its edits.
    */
   public Collection<URI> getNameEditsDirs(int nnIndex) throws IOException {
-    return FSNamesystem.getNamespaceEditsDirs(nameNodes[nnIndex].conf);
+    return FSNamesystem.getNamespaceEditsDirs(getNN(nnIndex).conf);
   }
   
   public void transitionToActive(int nnIndex) throws IOException,
@@ -2254,11 +2366,12 @@ public class MiniDFSCluster {
 
   /** Wait until the given namenode gets registration from all the datanodes */
   public void waitActive(int nnIndex) throws IOException {
-    if (nameNodes.length == 0 || nameNodes[nnIndex] == null
-        || nameNodes[nnIndex].nameNode == null) {
+    if (namenodes.size() == 0 || getNN(nnIndex) == null || getNN(nnIndex).nameNode == null) {
       return;
     }
-    InetSocketAddress addr = nameNodes[nnIndex].nameNode.getServiceRpcAddress();
+
+    NameNodeInfo info = getNN(nnIndex);
+    InetSocketAddress addr = info.nameNode.getServiceRpcAddress();
     assert addr.getPort() != 0;
     DFSClient client = new DFSClient(addr, conf);
 
@@ -2278,7 +2391,7 @@ public class MiniDFSCluster {
    * Wait until the cluster is active and running.
    */
   public void waitActive() throws IOException {
-    for (int index = 0; index < nameNodes.length; index++) {
+    for (int index = 0; index < namenodes.size(); index++) {
       int failedCount = 0;
       while (true) {
         try {
@@ -2298,7 +2411,14 @@ public class MiniDFSCluster {
     }
     LOG.info("Cluster is active");
   }
-  
+
+  public void printNNs() {
+    for (int i = 0; i < namenodes.size(); i++) {
+      LOG.info("Have namenode " + i + ", info:" + getNN(i));
+      LOG.info(" has namenode: " + getNN(i).nameNode);
+    }
+  }
+
   private synchronized boolean shouldWait(DatanodeInfo[] dnInfo,
       InetSocketAddress addr) {
     // If a datanode failed to start, then do not wait
@@ -2696,7 +2816,7 @@ public class MiniDFSCluster {
    * namenode
    */
   private void checkSingleNameNode() {
-    if (nameNodes.length != 1) {
+    if (namenodes.size() != 1) {
       throw new IllegalArgumentException("Namenode index is needed");
     }
   }
@@ -2712,13 +2832,9 @@ public class MiniDFSCluster {
     if(!federation)
       throw new IOException("cannot add namenode to non-federated cluster");
 
-    int nnIndex = nameNodes.length;
-    int numNameNodes = nameNodes.length + 1;
-    NameNodeInfo[] newlist = new NameNodeInfo[numNameNodes];
-    System.arraycopy(nameNodes, 0, newlist, 0, nameNodes.length);
-    nameNodes = newlist;
-    String nameserviceId = NAMESERVICE_ID_PREFIX + (nnIndex + 1);
-    
+    int nameServiceIndex = namenodes.keys().size();
+    String nameserviceId = NAMESERVICE_ID_PREFIX + (namenodes.keys().size() + 1);
+
     String nameserviceIds = conf.get(DFS_NAMESERVICES);
     nameserviceIds += "," + nameserviceId;
     conf.set(DFS_NAMESERVICES, nameserviceIds);
@@ -2726,9 +2842,11 @@ public class MiniDFSCluster {
     String nnId = null;
     initNameNodeAddress(conf, nameserviceId,
         new NNConf(nnId).setIpcPort(namenodePort));
-    initNameNodeConf(conf, nameserviceId, nnId, true, true, nnIndex);
-    createNameNode(nnIndex, conf, numDataNodes, true, null, null,
-        nameserviceId, nnId);
+    // figure out the current number of NNs
+    NameNodeInfo[] infos = this.getNameNodeInfos(nameserviceId);
+    int nnIndex = infos == null ? 0 : infos.length;
+    initNameNodeConf(conf, nameserviceId, nameServiceIndex, nnId, true, true, nnIndex);
+    NameNodeInfo info = createNameNode(conf, true, null, null, nameserviceId, nnId);
 
     // Refresh datanodes with the newly started namenode
     for (DataNodeProperties dn : dataNodes) {
@@ -2738,7 +2856,7 @@ public class MiniDFSCluster {
 
     // Wait for new namenode to get registrations from all the datanodes
     waitActive(nnIndex);
-    return nameNodes[nnIndex].nameNode;
+    return info.nameNode;
   }
   
   protected void setupDatanodeAddress(Configuration conf, boolean setupHostsFile,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
index a99e9c3..b9786a3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSNNTopology.java
@@ -56,10 +56,20 @@ public class MiniDFSNNTopology {
    * Set up an HA topology with a single HA nameservice.
    */
   public static MiniDFSNNTopology simpleHATopology() {
-    return new MiniDFSNNTopology()
-      .addNameservice(new MiniDFSNNTopology.NSConf("minidfs-ns")
-        .addNN(new MiniDFSNNTopology.NNConf("nn1"))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2")));
+    return simpleHATopology(2);
+  }
+
+  /**
+   * Set up an HA topology with a single HA nameservice.
+   * @param nnCount of namenodes to use with the nameservice
+   */
+  public static MiniDFSNNTopology simpleHATopology(int nnCount) {
+    MiniDFSNNTopology.NSConf nameservice = new MiniDFSNNTopology.NSConf("minidfs-ns");
+    for (int i = 1; i <= nnCount; i++) {
+      nameservice.addNN(new MiniDFSNNTopology.NNConf("nn" + i));
+    }
+    MiniDFSNNTopology topology = new MiniDFSNNTopology().addNameservice(nameservice);
+    return topology;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
index ad907f6..fae1024 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
@@ -303,12 +303,12 @@ public class TestDFSUpgradeFromImage {
     unpackStorage(HADOOP22_IMAGE, HADOOP_DFS_DIR_TXT);
     
     // Overwrite the md5 stored in the VERSION files
-    File baseDir = new File(MiniDFSCluster.getBaseDirectory());
+    File[] nnDirs = MiniDFSCluster.getNameNodeDirectory(MiniDFSCluster.getBaseDirectory(), 0, 0);
     FSImageTestUtil.corruptVersionFile(
-        new File(baseDir, "name1/current/VERSION"),
+        new File(nnDirs[0], "current/VERSION"),
         "imageMD5Digest", "22222222222222222222222222222222");
     FSImageTestUtil.corruptVersionFile(
-        new File(baseDir, "name2/current/VERSION"),
+        new File(nnDirs[1], "current/VERSION"),
         "imageMD5Digest", "22222222222222222222222222222222");
     
     // Attach our own log appender so we can verify output

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
index c4c890f..b50b1cb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
@@ -45,6 +45,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
+import static org.junit.Assert.assertTrue;
 
 /**
  * This class tests rolling upgrade.
@@ -66,7 +67,7 @@ public class TestRollingUpgrade {
    */
   @Test
   public void testDFSAdminRollingUpgradeCommands() throws Exception {
-    // start a cluster 
+    // start a cluster
     final Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
     try {
@@ -97,7 +98,7 @@ public class TestRollingUpgrade {
         runCmd(dfsadmin, true, "-rollingUpgrade", "query");
 
         dfs.mkdirs(bar);
-        
+
         //finalize rolling upgrade
         runCmd(dfsadmin, true, "-rollingUpgrade", "finalize");
 
@@ -143,7 +144,7 @@ public class TestRollingUpgrade {
     String nnDirPrefix = MiniDFSCluster.getBaseDirectory() + "/nn/";
     final File nn1Dir = new File(nnDirPrefix + "image1");
     final File nn2Dir = new File(nnDirPrefix + "image2");
-    
+
     LOG.info("nn1Dir=" + nn1Dir);
     LOG.info("nn2Dir=" + nn2Dir);
 
@@ -186,9 +187,9 @@ public class TestRollingUpgrade {
 
       final RollingUpgradeInfo info1;
       {
-        final DistributedFileSystem dfs = cluster.getFileSystem(); 
+        final DistributedFileSystem dfs = cluster.getFileSystem();
         dfs.mkdirs(foo);
-  
+
         //start rolling upgrade
         dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
         info1 = dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
@@ -197,7 +198,7 @@ public class TestRollingUpgrade {
 
         //query rolling upgrade
         Assert.assertEquals(info1, dfs.rollingUpgrade(RollingUpgradeAction.QUERY));
-  
+
         dfs.mkdirs(bar);
         cluster.shutdown();
       }
@@ -209,8 +210,8 @@ public class TestRollingUpgrade {
         .format(false)
         .manageNameDfsDirs(false)
         .build();
-      final DistributedFileSystem dfs2 = cluster2.getFileSystem(); 
-      
+      final DistributedFileSystem dfs2 = cluster2.getFileSystem();
+
       // Check that cluster2 sees the edits made on cluster1
       Assert.assertTrue(dfs2.exists(foo));
       Assert.assertTrue(dfs2.exists(bar));
@@ -260,7 +261,7 @@ public class TestRollingUpgrade {
 
   @Test
   public void testRollback() throws IOException {
-    // start a cluster 
+    // start a cluster
     final Configuration conf = new HdfsConfiguration();
     MiniDFSCluster cluster = null;
     try {
@@ -305,7 +306,7 @@ public class TestRollingUpgrade {
       if(cluster != null) cluster.shutdown();
     }
   }
-  
+
   private static void startRollingUpgrade(Path foo, Path bar,
       Path file, byte[] data,
       MiniDFSCluster cluster) throws IOException {
@@ -327,7 +328,7 @@ public class TestRollingUpgrade {
     TestFileTruncate.checkBlockRecovery(file, dfs);
     AppendTestUtil.checkFullFile(dfs, file, newLength, data);
   }
-  
+
   private static void rollbackRollingUpgrade(Path foo, Path bar,
       Path file, byte[] data,
       MiniDFSCluster cluster) throws IOException {
@@ -372,22 +373,33 @@ public class TestRollingUpgrade {
     }
   }
 
-  @Test (timeout = 300000)
+  @Test(timeout = 300000)
   public void testFinalize() throws Exception {
+    testFinalize(2);
+  }
+
+  @Test(timeout = 300000)
+  public void testFinalizeWithMultipleNN() throws Exception {
+    testFinalize(3);
+  }
+
+  private void testFinalize(int nnCount) throws Exception {
     final Configuration conf = new HdfsConfiguration();
     MiniQJMHACluster cluster = null;
     final Path foo = new Path("/foo");
     final Path bar = new Path("/bar");
 
     try {
-      cluster = new MiniQJMHACluster.Builder(conf).build();
+      cluster = new MiniQJMHACluster.Builder(conf).setNumNameNodes(nnCount).build();
       MiniDFSCluster dfsCluster = cluster.getDfsCluster();
       dfsCluster.waitActive();
 
-      // let NN1 tail editlog every 1s
-      dfsCluster.getConfiguration(1).setInt(
-          DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
-      dfsCluster.restartNameNode(1);
+      // let other NN tail editlog every 1s
+      for(int i=1; i < nnCount; i++) {
+        dfsCluster.getConfiguration(i).setInt(
+            DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
+      }
+      dfsCluster.restartNameNodes();
 
       dfsCluster.transitionToActive(0);
       DistributedFileSystem dfs = dfsCluster.getFileSystem(0);
@@ -425,17 +437,29 @@ public class TestRollingUpgrade {
 
   @Test (timeout = 300000)
   public void testQuery() throws Exception {
+    testQuery(2);
+  }
+
+  @Test (timeout = 300000)
+  public void testQueryWithMultipleNN() throws Exception {
+    testQuery(3);
+  }
+
+  private void testQuery(int nnCount) throws Exception{
     final Configuration conf = new Configuration();
     MiniQJMHACluster cluster = null;
     try {
-      cluster = new MiniQJMHACluster.Builder(conf).build();
+      cluster = new MiniQJMHACluster.Builder(conf).setNumNameNodes(nnCount).build();
       MiniDFSCluster dfsCluster = cluster.getDfsCluster();
       dfsCluster.waitActive();
 
       dfsCluster.transitionToActive(0);
       DistributedFileSystem dfs = dfsCluster.getFileSystem(0);
 
-      dfsCluster.shutdownNameNode(1);
+      // shutdown other NNs
+      for (int i = 1; i < nnCount; i++) {
+        dfsCluster.shutdownNameNode(i);
+      }
 
       // start rolling upgrade
       RollingUpgradeInfo info = dfs
@@ -445,13 +469,16 @@ public class TestRollingUpgrade {
       info = dfs.rollingUpgrade(RollingUpgradeAction.QUERY);
       Assert.assertFalse(info.createdRollbackImages());
 
-      dfsCluster.restartNameNode(1);
-
+      // restart other NNs
+      for (int i = 1; i < nnCount; i++) {
+        dfsCluster.restartNameNode(i);
+      }
+      // check that one of the other NNs has created the rollback image and uploaded it
       queryForPreparation(dfs);
 
       // The NN should have a copy of the fsimage in case of rollbacks.
       Assert.assertTrue(dfsCluster.getNamesystem(0).getFSImage()
-          .hasRollbackFSImage());
+              .hasRollbackFSImage());
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -487,6 +514,15 @@ public class TestRollingUpgrade {
 
   @Test(timeout = 300000)
   public void testCheckpoint() throws IOException, InterruptedException {
+    testCheckpoint(2);
+  }
+
+  @Test(timeout = 300000)
+  public void testCheckpointWithMultipleNN() throws IOException, InterruptedException {
+    testCheckpoint(3);
+  }
+
+  public void testCheckpoint(int nnCount) throws IOException, InterruptedException {
     final Configuration conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_HA_TAILEDITS_PERIOD_KEY, 1);
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY, 1);
@@ -495,7 +531,7 @@ public class TestRollingUpgrade {
     final Path foo = new Path("/foo");
 
     try {
-      cluster = new MiniQJMHACluster.Builder(conf).build();
+      cluster = new MiniQJMHACluster.Builder(conf).setNumNameNodes(nnCount).build();
       MiniDFSCluster dfsCluster = cluster.getDfsCluster();
       dfsCluster.waitActive();
 
@@ -513,16 +549,9 @@ public class TestRollingUpgrade {
       long txid = dfs.rollEdits();
       Assert.assertTrue(txid > 0);
 
-      int retries = 0;
-      while (++retries < 5) {
-        NNStorage storage = dfsCluster.getNamesystem(1).getFSImage()
-            .getStorage();
-        if (storage.getFsImageName(txid - 1) != null) {
-          return;
-        }
-        Thread.sleep(1000);
+      for(int i=1; i< nnCount; i++) {
+        verifyNNCheckpoint(dfsCluster, txid, i);
       }
-      Assert.fail("new checkpoint does not exist");
 
     } finally {
       if (cluster != null) {
@@ -531,6 +560,22 @@ public class TestRollingUpgrade {
     }
   }
 
+  /**
+   * Verify that the namenode at the given index has an FSImage with a TxId up to txid-1
+   */
+  private void verifyNNCheckpoint(MiniDFSCluster dfsCluster, long txid, int nnIndex) throws InterruptedException {
+    int retries = 0;
+    while (++retries < 5) {
+      NNStorage storage = dfsCluster.getNamesystem(nnIndex).getFSImage()
+              .getStorage();
+      if (storage.getFsImageName(txid - 1) != null) {
+        return;
+      }
+      Thread.sleep(1000);
+    }
+    Assert.fail("new checkpoint does not exist");
+  }
+
   static void queryForPreparation(DistributedFileSystem dfs) throws IOException,
       InterruptedException {
     RollingUpgradeInfo info;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
index ef4c559..470a08b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniQJMHACluster.java
@@ -17,43 +17,39 @@
  */
 package org.apache.hadoop.hdfs.qjournal;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
-
-import java.io.IOException;
-import java.net.BindException;
-import java.net.URI;
-import java.util.Random;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
+import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
+
+import java.io.IOException;
+import java.net.BindException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
 
 public class MiniQJMHACluster {
   private MiniDFSCluster cluster;
   private MiniJournalCluster journalCluster;
   private final Configuration conf;
   private static final Log LOG = LogFactory.getLog(MiniQJMHACluster.class);
-  
+
   public static final String NAMESERVICE = "ns1";
-  private static final String NN1 = "nn1";
-  private static final String NN2 = "nn2";
   private static final Random RANDOM = new Random();
   private int basePort = 10000;
 
   public static class Builder {
     private final Configuration conf;
     private StartupOption startOpt = null;
+    private int numNNs = 2;
     private final MiniDFSCluster.Builder dfsBuilder;
-    
+
     public Builder(Configuration conf) {
       this.conf = conf;
       // most QJMHACluster tests don't need DataNodes, so we'll make
@@ -64,7 +60,7 @@ public class MiniQJMHACluster {
     public MiniDFSCluster.Builder getDfsBuilder() {
       return dfsBuilder;
     }
-    
+
     public MiniQJMHACluster build() throws IOException {
       return new MiniQJMHACluster(this);
     }
@@ -72,15 +68,25 @@ public class MiniQJMHACluster {
     public void startupOption(StartupOption startOpt) {
       this.startOpt = startOpt;
     }
+
+    public Builder setNumNameNodes(int nns) {
+      this.numNNs = nns;
+      return this;
+    }
+  }
+
+  public static MiniDFSNNTopology createDefaultTopology(int nns, int startingPort) {
+    MiniDFSNNTopology.NSConf nameservice = new MiniDFSNNTopology.NSConf(NAMESERVICE);
+    for (int i = 0; i < nns; i++) {
+      nameservice.addNN(new MiniDFSNNTopology.NNConf("nn" + i).setIpcPort(startingPort++)
+          .setHttpPort(startingPort++));
+    }
+
+    return new MiniDFSNNTopology().addNameservice(nameservice);
   }
-  
+
   public static MiniDFSNNTopology createDefaultTopology(int basePort) {
-    return new MiniDFSNNTopology()
-      .addNameservice(new MiniDFSNNTopology.NSConf(NAMESERVICE).addNN(
-        new MiniDFSNNTopology.NNConf("nn1").setIpcPort(basePort)
-            .setHttpPort(basePort + 1)).addNN(
-        new MiniDFSNNTopology.NNConf("nn2").setIpcPort(basePort + 2)
-            .setHttpPort(basePort + 3)));
+    return createDefaultTopology(2, basePort);
   }
 
   private MiniQJMHACluster(Builder builder) throws IOException {
@@ -94,10 +100,10 @@ public class MiniQJMHACluster {
             .build();
         URI journalURI = journalCluster.getQuorumJournalURI(NAMESERVICE);
 
-        // start cluster with 2 NameNodes
-        MiniDFSNNTopology topology = createDefaultTopology(basePort);
+        // start cluster with specified NameNodes
+        MiniDFSNNTopology topology = createDefaultTopology(builder.numNNs, basePort);
 
-        initHAConf(journalURI, builder.conf);
+        initHAConf(journalURI, builder.conf, builder.numNNs);
 
         // First start up the NNs just to format the namespace. The MinIDFSCluster
         // has no way to just format the NameNodes without also starting them.
@@ -110,8 +116,9 @@ public class MiniQJMHACluster {
         Configuration confNN0 = cluster.getConfiguration(0);
         NameNode.initializeSharedEdits(confNN0, true);
 
-        cluster.getNameNodeInfos()[0].setStartOpt(builder.startOpt);
-        cluster.getNameNodeInfos()[1].setStartOpt(builder.startOpt);
+        for (MiniDFSCluster.NameNodeInfo nn : cluster.getNameNodeInfos()) {
+          nn.setStartOpt(builder.startOpt);
+        }
 
         // restart the cluster
         cluster.restartNameNodes();
@@ -123,31 +130,28 @@ public class MiniQJMHACluster {
       }
     }
   }
-  
-  private Configuration initHAConf(URI journalURI, Configuration conf) {
+
+  private Configuration initHAConf(URI journalURI, Configuration conf, int numNNs) {
     conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
         journalURI.toString());
-    
-    String address1 = "127.0.0.1:" + basePort;
-    String address2 = "127.0.0.1:" + (basePort + 2);
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
-        NAMESERVICE, NN1), address1);
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
-        NAMESERVICE, NN2), address2);
-    conf.set(DFSConfigKeys.DFS_NAMESERVICES, NAMESERVICE);
-    conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, NAMESERVICE),
-        NN1 + "," + NN2);
-    conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + NAMESERVICE,
-        ConfiguredFailoverProxyProvider.class.getName());
-    conf.set("fs.defaultFS", "hdfs://" + NAMESERVICE);
-    
+
+    List<String> nns = new ArrayList<String>(numNNs);
+    int port = basePort;
+    for (int i = 0; i < numNNs; i++) {
+      nns.add("127.0.0.1:" + port);
+      // increment by 2 each time to account for the http port in the config setting
+      port += 2;
+    }
+
+    // use standard failover configurations
+    HATestUtil.setFailoverConfigurations(conf, NAMESERVICE, nns);
     return conf;
   }
 
   public MiniDFSCluster getDfsCluster() {
     return cluster;
   }
-  
+
   public MiniJournalCluster getJournalCluster() {
     return journalCluster;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
index d5a9426..b203872 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/token/block/TestBlockToken.java
@@ -162,7 +162,7 @@ public class TestBlockToken {
   public void testWritable() throws Exception {
     TestWritable.testWritable(new BlockTokenIdentifier());
     BlockTokenSecretManager sm = new BlockTokenSecretManager(
-        blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
+        blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null);
     TestWritable.testWritable(generateTokenId(sm, block1,
         EnumSet.allOf(BlockTokenIdentifier.AccessMode.class)));
     TestWritable.testWritable(generateTokenId(sm, block2,
@@ -201,7 +201,7 @@ public class TestBlockToken {
   @Test
   public void testBlockTokenSecretManager() throws Exception {
     BlockTokenSecretManager masterHandler = new BlockTokenSecretManager(
-        blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
+        blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null);
     BlockTokenSecretManager slaveHandler = new BlockTokenSecretManager(
         blockKeyUpdateInterval, blockTokenLifetime, "fake-pool", null);
     ExportedBlockKeys keys = masterHandler.exportKeys();
@@ -244,7 +244,7 @@ public class TestBlockToken {
     UserGroupInformation.setConfiguration(conf);
     
     BlockTokenSecretManager sm = new BlockTokenSecretManager(
-        blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
+        blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null);
     Token<BlockTokenIdentifier> token = sm.generateToken(block3,
         EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
 
@@ -283,7 +283,7 @@ public class TestBlockToken {
     
     Assume.assumeTrue(FD_DIR.exists());
     BlockTokenSecretManager sm = new BlockTokenSecretManager(
-        blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
+        blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null);
     Token<BlockTokenIdentifier> token = sm.generateToken(block3,
         EnumSet.allOf(BlockTokenIdentifier.AccessMode.class));
 
@@ -352,7 +352,7 @@ public class TestBlockToken {
     for (int i = 0; i < 10; i++) {
       String bpid = Integer.toString(i);
       BlockTokenSecretManager masterHandler = new BlockTokenSecretManager(
-          blockKeyUpdateInterval, blockTokenLifetime, 0, "fake-pool", null);
+          blockKeyUpdateInterval, blockTokenLifetime, 0, 1, "fake-pool", null);
       BlockTokenSecretManager slaveHandler = new BlockTokenSecretManager(
           blockKeyUpdateInterval, blockTokenLifetime, "fake-pool", null);
       bpMgr.addBlockPool(bpid, slaveHandler);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
index f01be4b..0818571 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java
@@ -319,7 +319,7 @@ public class TestBackupNode {
       if(fileSys != null) fileSys.close();
       if(cluster != null) cluster.shutdown();
     }
-    File nnCurDir = new File(BASE_DIR, "name1/current/");
+    File nnCurDir = new File(MiniDFSCluster.getNameNodeDirectory(BASE_DIR, 0, 0)[0], "current/");
     File bnCurDir = new File(getBackupNodeDir(op, 1), "/current/");
 
     FSImageTestUtil.assertParallelFilesAreIdentical(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
index 5a51cb7..7073726 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
@@ -1428,7 +1428,8 @@ public class TestCheckpoint {
       //
       secondary = startSecondaryNameNode(conf);
 
-      File secondaryDir = new File(MiniDFSCluster.getBaseDirectory(), "namesecondary1");
+      File secondaryDir = MiniDFSCluster.getCheckpointDirectory(MiniDFSCluster.getBaseDirectory(),
+        0, 0)[0];
       File secondaryCurrent = new File(secondaryDir, "current");
 
       long expectedTxIdToDownload = cluster.getNameNode().getFSImage()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HAStressTestHarness.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HAStressTestHarness.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HAStressTestHarness.java
index 5b72901..a736d27 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HAStressTestHarness.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HAStressTestHarness.java
@@ -42,7 +42,8 @@ public class HAStressTestHarness {
   private MiniDFSCluster cluster;
   static final int BLOCK_SIZE = 1024;
   final TestContext testCtx = new TestContext();
-  
+  private int nns = 2;
+
   public HAStressTestHarness() {
     conf = new Configuration();
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
@@ -55,11 +56,19 @@ public class HAStressTestHarness {
   }
 
   /**
+   * Set the number of namenodes that should be run. This must be set before calling
+   * {@link #startCluster()}
+   */
+  public void setNumberOfNameNodes(int nns) {
+    this.nns = nns;
+  }
+
+  /**
    * Start and return the MiniDFSCluster.
    */
   public MiniDFSCluster startCluster() throws IOException {
     cluster = new MiniDFSCluster.Builder(conf)
-      .nnTopology(MiniDFSNNTopology.simpleHATopology())
+      .nnTopology(MiniDFSNNTopology.simpleHATopology(nns))
       .numDataNodes(3)
       .build();
     return cluster;
@@ -99,28 +108,27 @@ public class HAStressTestHarness {
   }
 
   /**
-   * Add a thread which periodically triggers failover back and forth between
-   * the two namenodes.
+   * Add a thread which periodically triggers failover back and forth between the namenodes.
    */
   public void addFailoverThread(final int msBetweenFailovers) {
     testCtx.addThread(new RepeatingTestThread(testCtx) {
-      
       @Override
       public void doAnAction() throws Exception {
-        System.err.println("==============================\n" +
-            "Failing over from 0->1\n" +
-            "==================================");
-        cluster.transitionToStandby(0);
-        cluster.transitionToActive(1);
-        
-        Thread.sleep(msBetweenFailovers);
-        System.err.println("==============================\n" +
-            "Failing over from 1->0\n" +
-            "==================================");
-
-        cluster.transitionToStandby(1);
-        cluster.transitionToActive(0);
-        Thread.sleep(msBetweenFailovers);
+        // fail over from one namenode to the next, all the way back to the original NN
+        for (int i = 0; i < nns; i++) {
+          // next node, mod nns so we wrap to the 0th NN on the last iteration
+          int next = (i + 1) % nns;
+          System.err.println("==============================\n"
+              + "[Starting] Failing over from " + i + "->" + next + "\n"
+              + "==============================");
+          cluster.transitionToStandby(i);
+          cluster.transitionToActive(next);
+          System.err.println("==============================\n"
+              + "[Completed] Failing over from " + i + "->" + next + ". Sleeping for "+
+              (msBetweenFailovers/1000) +"sec \n"
+              + "==============================");
+          Thread.sleep(msBetweenFailovers);
+        }
       }
     });
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
index c7c4a77..5543a2a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/HATestUtil.java
@@ -24,9 +24,14 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.TimeoutException;
 
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -67,12 +72,11 @@ public abstract class HATestUtil {
    */
   public static void waitForStandbyToCatchUp(NameNode active,
       NameNode standby) throws InterruptedException, IOException, CouldNotCatchUpException {
-    
     long activeTxId = active.getNamesystem().getFSImage().getEditLog()
       .getLastWrittenTxId();
-    
+
     active.getRpcServer().rollEditLog();
-    
+
     long start = Time.now();
     while (Time.now() - start < TestEditLogTailer.NN_LAG_TIMEOUT) {
       long nn2HighestTxId = standby.getNamesystem().getFSImage()
@@ -166,34 +170,52 @@ public abstract class HATestUtil {
   /** Sets the required configurations for performing failover.  */
   public static void setFailoverConfigurations(MiniDFSCluster cluster,
       Configuration conf, String logicalName, int nsIndex) {
-    InetSocketAddress nnAddr1 = cluster.getNameNode(2 * nsIndex).getNameNodeAddress();
-    InetSocketAddress nnAddr2 = cluster.getNameNode(2 * nsIndex + 1).getNameNodeAddress();
-    setFailoverConfigurations(conf, logicalName, nnAddr1, nnAddr2);
+    MiniDFSCluster.NameNodeInfo[] nns = cluster.getNameNodeInfos(nsIndex);
+    List<InetSocketAddress> nnAddresses = new ArrayList<InetSocketAddress>(3);
+    for (MiniDFSCluster.NameNodeInfo nn : nns) {
+      nnAddresses.add(nn.nameNode.getNameNodeAddress());
+    }
+    setFailoverConfigurations(conf, logicalName, nnAddresses);
+  }
+
+  public static void setFailoverConfigurations(Configuration conf, String logicalName,
+      InetSocketAddress ... nnAddresses){
+    setFailoverConfigurations(conf, logicalName, Arrays.asList(nnAddresses));
   }
 
   /**
    * Sets the required configurations for performing failover
    */
   public static void setFailoverConfigurations(Configuration conf,
-      String logicalName, InetSocketAddress nnAddr1,
-      InetSocketAddress nnAddr2) {
-    String nameNodeId1 = "nn1";
-    String nameNodeId2 = "nn2";
-    String address1 = "hdfs://" + nnAddr1.getHostName() + ":" + nnAddr1.getPort();
-    String address2 = "hdfs://" + nnAddr2.getHostName() + ":" + nnAddr2.getPort();
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
-        logicalName, nameNodeId1), address1);
-    conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
-        logicalName, nameNodeId2), address2);
-    
+      String logicalName, List<InetSocketAddress> nnAddresses) {
+    setFailoverConfigurations(conf, logicalName,
+        Iterables.transform(nnAddresses, new Function<InetSocketAddress, String>() {
+
+          // transform the inet address to a simple string
+          @Override
+          public String apply(InetSocketAddress addr) {
+            return "hdfs://" + addr.getHostName() + ":" + addr.getPort();
+          }
+        }));
+  }
+
+  public static void setFailoverConfigurations(Configuration conf, String logicalName,
+      Iterable<String> nnAddresses) {
+    List<String> nnids = new ArrayList<String>();
+    int i = 0;
+    for (String address : nnAddresses) {
+      String nnId = "nn" + (i + 1);
+      nnids.add(nnId);
+      conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY, logicalName, nnId), address);
+      i++;
+    }
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, logicalName);
     conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY_PREFIX, logicalName),
-        nameNodeId1 + "," + nameNodeId2);
+        Joiner.on(',').join(nnids));
     conf.set(HdfsClientConfigKeys.Failover.PROXY_PROVIDER_KEY_PREFIX + "." + logicalName,
         ConfiguredFailoverProxyProvider.class.getName());
     conf.set("fs.defaultFS", "hdfs://" + logicalName);
   }
-  
 
   public static String getLogicalHostname(MiniDFSCluster cluster) {
     return String.format(LOGICAL_HOSTNAME, cluster.getInstanceId());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
index 7abc502..16dc766 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandby.java
@@ -46,37 +46,47 @@ import com.google.common.collect.ImmutableList;
 
 public class TestBootstrapStandby {
   private static final Log LOG = LogFactory.getLog(TestBootstrapStandby.class);
-  
+
+  private static final int maxNNCount = 3;
+  private static final int STARTING_PORT = 20000;
+
   private MiniDFSCluster cluster;
   private NameNode nn0;
-  
+
   @Before
   public void setupCluster() throws IOException {
     Configuration conf = new Configuration();
 
-    MiniDFSNNTopology topology = new MiniDFSNNTopology()
-      .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
-        .addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(20001))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(20002)));
-    
+    // duplicate code with MiniQJMHACluster#createDefaultTopology, but don't want to cross
+    // dependencies or munge too much code to support it all correctly
+    MiniDFSNNTopology.NSConf nameservice = new MiniDFSNNTopology.NSConf("ns1");
+    for (int i = 0; i < maxNNCount; i++) {
+      nameservice.addNN(new MiniDFSNNTopology.NNConf("nn" + i).setHttpPort(STARTING_PORT + i + 1));
+    }
+
+    MiniDFSNNTopology topology = new MiniDFSNNTopology().addNameservice(nameservice);
+
     cluster = new MiniDFSCluster.Builder(conf)
-      .nnTopology(topology)
-      .numDataNodes(0)
-      .build();
+        .nnTopology(topology)
+        .numDataNodes(0)
+        .build();
     cluster.waitActive();
-    
+
     nn0 = cluster.getNameNode(0);
     cluster.transitionToActive(0);
-    cluster.shutdownNameNode(1);
+    // shutdown the other NNs
+    for (int i = 1; i < maxNNCount; i++) {
+      cluster.shutdownNameNode(i);
+    }
   }
-  
+
   @After
   public void shutdownCluster() {
     if (cluster != null) {
       cluster.shutdown();
     }
   }
-  
+
   /**
    * Test for the base success case. The primary NN
    * hasn't made any checkpoints, and we copy the fsimage_0
@@ -85,30 +95,29 @@ public class TestBootstrapStandby {
   @Test
   public void testSuccessfulBaseCase() throws Exception {
     removeStandbyNameDirs();
-    
-    try {
-      cluster.restartNameNode(1);
-      fail("Did not throw");
-    } catch (IOException ioe) {
-      GenericTestUtils.assertExceptionContains(
-          "storage directory does not exist or is not accessible",
-          ioe);
+
+    // skip the first NN, its up
+    for (int index = 1; index < maxNNCount; index++) {
+      try {
+        cluster.restartNameNode(index);
+        fail("Did not throw");
+      } catch (IOException ioe) {
+        GenericTestUtils.assertExceptionContains(
+            "storage directory does not exist or is not accessible", ioe);
+      }
+
+      int rc = BootstrapStandby.run(new String[] { "-nonInteractive" },
+          cluster.getConfiguration(index));
+      assertEquals(0, rc);
+
+      // Should have copied over the namespace from the active
+      FSImageTestUtil.assertNNHasCheckpoints(cluster, index, ImmutableList.of(0));
     }
-    
-    int rc = BootstrapStandby.run(
-        new String[]{"-nonInteractive"},
-        cluster.getConfiguration(1));
-    assertEquals(0, rc);
-    
-    // Should have copied over the namespace from the active
-    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
-        ImmutableList.of(0));
-    FSImageTestUtil.assertNNFilesMatch(cluster);
 
-    // We should now be able to start the standby successfully.
-    cluster.restartNameNode(1);
+    // We should now be able to start the standbys successfully.
+    restartNameNodesFromIndex(1);
   }
-  
+
   /**
    * Test for downloading a checkpoint made at a later checkpoint
    * from the active.
@@ -123,21 +132,21 @@ public class TestBootstrapStandby {
     NameNodeAdapter.saveNamespace(nn0);
     NameNodeAdapter.leaveSafeMode(nn0);
     long expectedCheckpointTxId = NameNodeAdapter.getNamesystem(nn0)
-      .getFSImage().getMostRecentCheckpointTxId();
+        .getFSImage().getMostRecentCheckpointTxId();
     assertEquals(6, expectedCheckpointTxId);
 
-    int rc = BootstrapStandby.run(
-        new String[]{"-force"},
-        cluster.getConfiguration(1));
-    assertEquals(0, rc);
-    
-    // Should have copied over the namespace from the active
-    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
-        ImmutableList.of((int)expectedCheckpointTxId));
+    for (int i = 1; i < maxNNCount; i++) {
+      assertEquals(0, forceBootstrap(i));
+
+      // Should have copied over the namespace from the active
+      LOG.info("Checking namenode: " + i);
+      FSImageTestUtil.assertNNHasCheckpoints(cluster, i,
+          ImmutableList.of((int) expectedCheckpointTxId));
+    }
     FSImageTestUtil.assertNNFilesMatch(cluster);
 
     // We should now be able to start the standby successfully.
-    cluster.restartNameNode(1);
+    restartNameNodesFromIndex(1);
   }
 
   /**
@@ -147,36 +156,40 @@ public class TestBootstrapStandby {
   @Test
   public void testSharedEditsMissingLogs() throws Exception {
     removeStandbyNameDirs();
-    
+
     CheckpointSignature sig = nn0.getRpcServer().rollEditLog();
     assertEquals(3, sig.getCurSegmentTxId());
-    
+
     // Should have created edits_1-2 in shared edits dir
-    URI editsUri = cluster.getSharedEditsDir(0, 1);
+    URI editsUri = cluster.getSharedEditsDir(0, maxNNCount - 1);
     File editsDir = new File(editsUri);
-    File editsSegment = new File(new File(editsDir, "current"),
+    File currentDir = new File(editsDir, "current");
+    File editsSegment = new File(currentDir,
         NNStorage.getFinalizedEditsFileName(1, 2));
     GenericTestUtils.assertExists(editsSegment);
+    GenericTestUtils.assertExists(currentDir);
 
     // Delete the segment.
     assertTrue(editsSegment.delete());
-    
+
     // Trying to bootstrap standby should now fail since the edit
     // logs aren't available in the shared dir.
     LogCapturer logs = GenericTestUtils.LogCapturer.captureLogs(
         LogFactory.getLog(BootstrapStandby.class));
     try {
-      int rc = BootstrapStandby.run(
-          new String[]{"-force"},
-          cluster.getConfiguration(1));
-      assertEquals(BootstrapStandby.ERR_CODE_LOGS_UNAVAILABLE, rc);
+      assertEquals(BootstrapStandby.ERR_CODE_LOGS_UNAVAILABLE, forceBootstrap(1));
     } finally {
       logs.stopCapturing();
     }
     GenericTestUtils.assertMatches(logs.getOutput(),
         "FATAL.*Unable to read transaction ids 1-3 from the configured shared");
   }
-  
+
+  /**
+   * Show that bootstrapping will fail on a given NameNode if its directories already exist. Its not
+   * run across all the NN because its testing the state local on each node.
+   * @throws Exception on unexpected failure
+   */
   @Test
   public void testStandbyDirsAlreadyExist() throws Exception {
     // Should not pass since standby dirs exist, force not given
@@ -186,12 +199,9 @@ public class TestBootstrapStandby {
     assertEquals(BootstrapStandby.ERR_CODE_ALREADY_FORMATTED, rc);
 
     // Should pass with -force
-    rc = BootstrapStandby.run(
-        new String[]{"-force"},
-        cluster.getConfiguration(1));
-    assertEquals(0, rc);
+    assertEquals(0, forceBootstrap(1));
   }
-  
+
   /**
    * Test that, even if the other node is not active, we are able
    * to bootstrap standby from it.
@@ -199,18 +209,44 @@ public class TestBootstrapStandby {
   @Test(timeout=30000)
   public void testOtherNodeNotActive() throws Exception {
     cluster.transitionToStandby(0);
-    int rc = BootstrapStandby.run(
-        new String[]{"-force"},
-        cluster.getConfiguration(1));
-    assertEquals(0, rc);
+    assertSuccessfulBootstrapFromIndex(1);
   }
 
   private void removeStandbyNameDirs() {
-    for (URI u : cluster.getNameDirs(1)) {
-      assertTrue(u.getScheme().equals("file"));
-      File dir = new File(u.getPath());
-      LOG.info("Removing standby dir " + dir);
-      assertTrue(FileUtil.fullyDelete(dir));
+    for (int i = 1; i < maxNNCount; i++) {
+      for (URI u : cluster.getNameDirs(i)) {
+        assertTrue(u.getScheme().equals("file"));
+        File dir = new File(u.getPath());
+        LOG.info("Removing standby dir " + dir);
+        assertTrue(FileUtil.fullyDelete(dir));
+      }
+    }
+  }
+
+  private void restartNameNodesFromIndex(int start) throws IOException {
+    for (int i = start; i < maxNNCount; i++) {
+      // We should now be able to start the standby successfully.
+      cluster.restartNameNode(i, false);
+    }
+
+    cluster.waitClusterUp();
+    cluster.waitActive();
+  }
+
+  /**
+   * Force boot strapping on a namenode
+   * @param i index of the namenode to attempt
+   * @return exit code
+   * @throws Exception on unexpected failure
+   */
+  private int forceBootstrap(int i) throws Exception {
+    return BootstrapStandby.run(new String[] { "-force" },
+        cluster.getConfiguration(i));
+  }
+
+  private void assertSuccessfulBootstrapFromIndex(int start) throws Exception {
+    for (int i = start; i < maxNNCount; i++) {
+      assertEquals(0, forceBootstrap(i));
     }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
index ca8f563..db9a2de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestBootstrapStandbyWithQJM.java
@@ -52,7 +52,8 @@ public class TestBootstrapStandbyWithQJM {
 
   private MiniDFSCluster cluster;
   private MiniJournalCluster jCluster;
-  
+  private int nnCount = 3;
+
   @Before
   public void setup() throws Exception {
     Configuration conf = new Configuration();
@@ -62,7 +63,8 @@ public class TestBootstrapStandbyWithQJM {
         CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
         0);
 
-    MiniQJMHACluster miniQjmHaCluster = new MiniQJMHACluster.Builder(conf).build();
+    MiniQJMHACluster miniQjmHaCluster =
+        new MiniQJMHACluster.Builder(conf).setNumNameNodes(nnCount).build();
     cluster = miniQjmHaCluster.getDfsCluster();
     jCluster = miniQjmHaCluster.getJournalCluster();
     
@@ -90,18 +92,7 @@ public class TestBootstrapStandbyWithQJM {
   public void testBootstrapStandbyWithStandbyNN() throws Exception {
     // make the first NN in standby state
     cluster.transitionToStandby(0);
-    Configuration confNN1 = cluster.getConfiguration(1);
-    
-    // shut down nn1
-    cluster.shutdownNameNode(1);
-    
-    int rc = BootstrapStandby.run(new String[] { "-force" }, confNN1);
-    assertEquals(0, rc);
-    
-    // Should have copied over the namespace from the standby
-    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
-        ImmutableList.of(0));
-    FSImageTestUtil.assertNNFilesMatch(cluster);
+    bootstrapStandbys();
   }
   
   /** BootstrapStandby when the existing NN is active */
@@ -109,17 +100,23 @@ public class TestBootstrapStandbyWithQJM {
   public void testBootstrapStandbyWithActiveNN() throws Exception {
     // make the first NN in active state
     cluster.transitionToActive(0);
-    Configuration confNN1 = cluster.getConfiguration(1);
-    
-    // shut down nn1
-    cluster.shutdownNameNode(1);
-    
-    int rc = BootstrapStandby.run(new String[] { "-force" }, confNN1);
-    assertEquals(0, rc);
-    
-    // Should have copied over the namespace from the standby
-    FSImageTestUtil.assertNNHasCheckpoints(cluster, 1,
-        ImmutableList.of(0));
+    bootstrapStandbys();
+  }
+
+  private void bootstrapStandbys() throws Exception {
+    // shutdown and bootstrap all the other nns, except the first (start 1, not 0)
+    for (int i = 1; i < nnCount; i++) {
+      Configuration otherNNConf = cluster.getConfiguration(i);
+
+      // shut down other nn
+      cluster.shutdownNameNode(i);
+
+      int rc = BootstrapStandby.run(new String[] { "-force" }, otherNNConf);
+      assertEquals(0, rc);
+
+      // Should have copied over the namespace from the standby
+      FSImageTestUtil.assertNNHasCheckpoints(cluster, i, ImmutableList.of(0));
+    }
     FSImageTestUtil.assertNNFilesMatch(cluster);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java
index e7cba75..9164582 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencingWithReplication.java
@@ -107,6 +107,7 @@ public class TestDNFencingWithReplication {
   @Test
   public void testFencingStress() throws Exception {
     HAStressTestHarness harness = new HAStressTestHarness();
+    harness.setNumberOfNameNodes(3);
     harness.conf.setInt(
         DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000);
     harness.conf.setInt(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
index 8c61c92..aea4f87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
@@ -113,7 +113,12 @@ public class TestEditLogTailer {
   public void testNN1TriggersLogRolls() throws Exception {
     testStandbyTriggersLogRolls(1);
   }
-  
+
+  @Test
+  public void testNN2TriggersLogRolls() throws Exception {
+    testStandbyTriggersLogRolls(2);
+  }
+
   private static void testStandbyTriggersLogRolls(int activeIndex)
       throws Exception {
     Configuration conf = new Configuration();
@@ -125,7 +130,8 @@ public class TestEditLogTailer {
     MiniDFSNNTopology topology = new MiniDFSNNTopology()
       .addNameservice(new MiniDFSNNTopology.NSConf("ns1")
         .addNN(new MiniDFSNNTopology.NNConf("nn1").setIpcPort(10031))
-        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10032)));
+        .addNN(new MiniDFSNNTopology.NNConf("nn2").setIpcPort(10032))
+        .addNN(new MiniDFSNNTopology.NNConf("nn3").setIpcPort(10033)));
 
     MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
       .nnTopology(topology)
@@ -145,7 +151,7 @@ public class TestEditLogTailer {
   
   private static void waitForLogRollInSharedDir(MiniDFSCluster cluster,
       long startTxId) throws Exception {
-    URI sharedUri = cluster.getSharedEditsDir(0, 1);
+    URI sharedUri = cluster.getSharedEditsDir(0, 2);
     File sharedDir = new File(sharedUri.getPath(), "current");
     final File expectedLog = new File(sharedDir,
         NNStorage.getInProgressEditsFileName(startTxId));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/49dfad94/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java
index 151e7d3..116079a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailoverWithBlockTokensEnabled.java
@@ -56,10 +56,11 @@ public class TestFailoverWithBlockTokensEnabled {
   
   private static final Path TEST_PATH = new Path("/test-path");
   private static final String TEST_DATA = "very important text";
-  
+  private static final int numNNs = 3;
+
   private Configuration conf;
   private MiniDFSCluster cluster;
-  
+
   @Before
   public void startCluster() throws IOException {
     conf = new Configuration();
@@ -67,7 +68,7 @@ public class TestFailoverWithBlockTokensEnabled {
     // Set short retry timeouts so this test runs faster
     conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10);
     cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleHATopology())
+        .nnTopology(MiniDFSNNTopology.simpleHATopology(numNNs))
         .numDataNodes(1)
         .build();
   }
@@ -78,33 +79,41 @@ public class TestFailoverWithBlockTokensEnabled {
       cluster.shutdown();
     }
   }
-  
+
   @Test
   public void ensureSerialNumbersNeverOverlap() {
     BlockTokenSecretManager btsm1 = cluster.getNamesystem(0).getBlockManager()
         .getBlockTokenSecretManager();
     BlockTokenSecretManager btsm2 = cluster.getNamesystem(1).getBlockManager()
         .getBlockTokenSecretManager();
-    
-    btsm1.setSerialNo(0);
-    btsm2.setSerialNo(0);
-    assertFalse(btsm1.getSerialNoForTesting() == btsm2.getSerialNoForTesting());
-    
-    btsm1.setSerialNo(Integer.MAX_VALUE);
-    btsm2.setSerialNo(Integer.MAX_VALUE);
-    assertFalse(btsm1.getSerialNoForTesting() == btsm2.getSerialNoForTesting());
-    
-    btsm1.setSerialNo(Integer.MIN_VALUE);
-    btsm2.setSerialNo(Integer.MIN_VALUE);
-    assertFalse(btsm1.getSerialNoForTesting() == btsm2.getSerialNoForTesting());
-    
-    btsm1.setSerialNo(Integer.MAX_VALUE / 2);
-    btsm2.setSerialNo(Integer.MAX_VALUE / 2);
-    assertFalse(btsm1.getSerialNoForTesting() == btsm2.getSerialNoForTesting());
+    BlockTokenSecretManager btsm3 = cluster.getNamesystem(2).getBlockManager()
+        .getBlockTokenSecretManager();
+
+    setAndCheckSerialNumber(0, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MAX_VALUE, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MIN_VALUE, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MAX_VALUE / 2, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MIN_VALUE / 2, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MAX_VALUE / 3, btsm1, btsm2, btsm3);
+    setAndCheckSerialNumber(Integer.MIN_VALUE / 3, btsm1, btsm2, btsm3);
+  }
+
+  private void setAndCheckSerialNumber(int serialNumber, BlockTokenSecretManager... btsms) {
+    for (BlockTokenSecretManager btsm : btsms) {
+      btsm.setSerialNo(serialNumber);
+    }
 
-    btsm1.setSerialNo(Integer.MIN_VALUE / 2);
-    btsm2.setSerialNo(Integer.MIN_VALUE / 2);
-    assertFalse(btsm1.getSerialNoForTesting() == btsm2.getSerialNoForTesting());
+    for (int i = 0; i < btsms.length; i++) {
+      for (int j = 0; j < btsms.length; j++) {
+        if (j == i) {
+          continue;
+        }
+        int first = btsms[i].getSerialNoForTesting();
+        int second = btsms[j].getSerialNoForTesting();
+        assertFalse("Overlap found for set serial number (" + serialNumber + ") is " + i + ": "
+            + first + " == " + j + ": " + second, first == second);
+      }
+    }
   }
   
   @Test


[11/30] hadoop git commit: Move YARN-3790 from 2.7.1 to 2.8 in CHANGES.txt

Posted by aw...@apache.org.
Move YARN-3790 from 2.7.1 to 2.8 in CHANGES.txt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2df00d53
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2df00d53
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2df00d53

Branch: refs/heads/HADOOP-12111
Commit: 2df00d53d13d16628b6bde5e05133d239f138f52
Parents: dd4b387
Author: rohithsharmaks <ro...@apache.org>
Authored: Wed Jun 24 23:31:03 2015 +0530
Committer: rohithsharmaks <ro...@apache.org>
Committed: Wed Jun 24 23:31:03 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2df00d53/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 9547f0f..f831f0a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -548,6 +548,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3835. hadoop-yarn-server-resourcemanager test package bundles
     core-site.xml, yarn-site.xml (vamsee via rkanter)
 
+    YARN-3790. usedResource from rootQueue metrics may get stale data for FS
+    scheduler after recovering the container (Zhihai Xu via rohithsharmaks)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -705,9 +708,6 @@ Release 2.7.1 - UNRELEASED
     YARN-3832. Resource Localization fails on a cluster due to existing cache
     directories (Brahma Reddy Battula via jlowe)
 
-    YARN-3790. usedResource from rootQueue metrics may get stale data for FS
-    scheduler after recovering the container (Zhihai Xu via rohithsharmaks)
-
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[09/30] hadoop git commit: YARN-3832. Resource Localization fails on a cluster due to existing cache directories. Contributed by Brahma Reddy Battula

Posted by aw...@apache.org.
YARN-3832. Resource Localization fails on a cluster due to existing cache directories. Contributed by Brahma Reddy Battula


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8d58512d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8d58512d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8d58512d

Branch: refs/heads/HADOOP-12111
Commit: 8d58512d6e6d9fe93784a9de2af0056bcc316d96
Parents: 2a20dd9
Author: Jason Lowe <jl...@apache.org>
Authored: Wed Jun 24 16:37:39 2015 +0000
Committer: Jason Lowe <jl...@apache.org>
Committed: Wed Jun 24 16:37:39 2015 +0000

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../containermanager/localizer/ResourceLocalizationService.java   | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d58512d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 7ecdee3..a5fc86b 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -702,6 +702,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3809. Failed to launch new attempts because
     ApplicationMasterLauncher's threads all hang (Jun Gong via jlowe)
 
+    YARN-3832. Resource Localization fails on a cluster due to existing cache
+    directories (Brahma Reddy Battula via jlowe)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8d58512d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index d6e0903..b417c5e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -1340,7 +1340,7 @@ public class ResourceLocalizationService extends CompositeService
   }
 
   private void cleanUpLocalDirs(FileContext lfs, DeletionService del) {
-    for (String localDir : dirsHandler.getLocalDirs()) {
+    for (String localDir : dirsHandler.getLocalDirsForCleanup()) {
       cleanUpLocalDir(lfs, del, localDir);
     }
   }


[19/30] hadoop git commit: HDFS-8640. Make reserved RBW space visible through JMX. (Contributed by kanaka kumar avvaru)

Posted by aw...@apache.org.
HDFS-8640. Make reserved RBW space visible through JMX. (Contributed by kanaka kumar avvaru)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/67a62da5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/67a62da5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/67a62da5

Branch: refs/heads/HADOOP-12111
Commit: 67a62da5c5f592b07d083440ced3666c7709b20d
Parents: bc43390
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jun 25 10:13:22 2015 -0700
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jun 25 10:13:22 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../datanode/fsdataset/impl/FsDatasetImpl.java  |  5 +++-
 .../fsdataset/impl/TestRbwSpaceReservation.java | 28 ++++++++++++++++++++
 3 files changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/67a62da5/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e5c30bd..01bb92d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -669,6 +669,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8462. Implement GETXATTRS and LISTXATTRS operations for WebImageViewer.
     (Jagadesh Kiran N via aajisaka)
 
+    HDFS-8640. Make reserved RBW space visible through JMX. (kanaka kumar
+    avvaru via Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67a62da5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index a1ff918..a2bb2c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -2559,13 +2559,15 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     final String directory;
     final long usedSpace; // size of space used by HDFS
     final long freeSpace; // size of free space excluding reserved space
-    final long reservedSpace; // size of space reserved for non-HDFS and RBW
+    final long reservedSpace; // size of space reserved for non-HDFS
+    final long reservedSpaceForRBW; // size of space reserved RBW
 
     VolumeInfo(FsVolumeImpl v, long usedSpace, long freeSpace) {
       this.directory = v.toString();
       this.usedSpace = usedSpace;
       this.freeSpace = freeSpace;
       this.reservedSpace = v.getReserved();
+      this.reservedSpaceForRBW = v.getReservedForRbw();
     }
   }  
 
@@ -2599,6 +2601,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
       innerInfo.put("usedSpace", v.usedSpace);
       innerInfo.put("freeSpace", v.freeSpace);
       innerInfo.put("reservedSpace", v.reservedSpace);
+      innerInfo.put("reservedSpaceForRBW", v.reservedSpaceForRBW);
       info.put(v.directory, innerInfo);
     }
     return info;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/67a62da5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
index 44bf81b..a647d96 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestRbwSpaceReservation.java
@@ -49,11 +49,15 @@ import org.mockito.Mockito;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.lang.management.ManagementFactory;
 import java.lang.reflect.Field;
 import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.TimeoutException;
 
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
 /**
  * Ensure that the DN reserves disk space equivalent to a full block for
  * replica being written (RBW).
@@ -324,6 +328,30 @@ public class TestRbwSpaceReservation {
         fsVolumeImpl.getReservedForRbw() == 0);
   }
 
+  @Test(timeout = 30000)
+  public void testRBWInJMXBean() throws Exception {
+
+    final short replication = 1;
+    startCluster(BLOCK_SIZE, replication, -1);
+
+    final String methodName = GenericTestUtils.getMethodName();
+    final Path file = new Path("/" + methodName + ".01.dat");
+
+    try (FSDataOutputStream os = fs.create(file, replication)) {
+      // Write 1 byte to the file
+      os.write(new byte[1]);
+      os.hsync();
+
+      final MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      final ObjectName mxbeanName = new ObjectName(
+          "Hadoop:service=DataNode,name=DataNodeInfo");
+      final String volumeInfo = (String) mbs.getAttribute(mxbeanName,
+          "VolumeInfo");
+
+      assertTrue(volumeInfo.contains("reservedSpaceForRBW"));
+    }
+  }
+
   /**
    * Stress test to ensure we are not leaking reserved space.
    * @throws IOException