You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by cn...@apache.org on 2013/07/11 07:35:30 UTC

svn commit: r1502120 [2/2] - in /hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs: ./ src/main/java/org/apache/hadoop/hdfs/security/token/delegation/ src/main/java/org/apache/hadoop/hdfs/server/namenode/ src/main/java/org/apache/hadoop/hdfs/server/n...

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/Status.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/Status.java?rev=1502120&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/Status.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/Status.java Thu Jul 11 05:35:29 2013
@@ -0,0 +1,40 @@
+/**
+ * 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.startupprogress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Indicates run status of a {@link Phase}.
+ */
+@InterfaceAudience.Private
+public enum Status {
+  /**
+   * The phase has not yet started running.
+   */
+  PENDING,
+
+  /**
+   * The phase is running right now.
+   */
+  RUNNING,
+
+  /**
+   * The phase has already completed.
+   */
+  COMPLETE
+}

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/Step.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/Step.java?rev=1502120&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/Step.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/Step.java Thu Jul 11 05:35:29 2013
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.startupprogress;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.lang.builder.CompareToBuilder;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * A step performed by the namenode during a {@link Phase} of startup.
+ */
+@InterfaceAudience.Private
+public class Step implements Comparable<Step> {
+  private static final AtomicInteger SEQUENCE = new AtomicInteger();
+
+  private final String file;
+  private final int sequenceNumber;
+  private final long size;
+  private final StepType type;
+
+  /**
+   * Creates a new Step.
+   * 
+   * @param type StepType type of step
+   */
+  public Step(StepType type) {
+    this(type, null, Long.MIN_VALUE);
+  }
+
+  /**
+   * Creates a new Step.
+   * 
+   * @param file String file
+   */
+  public Step(String file) {
+    this(null, file, Long.MIN_VALUE);
+  }
+
+  /**
+   * Creates a new Step.
+   * 
+   * @param file String file
+   * @param size long size in bytes
+   */
+  public Step(String file, long size) {
+    this(null, file, size);
+  }
+
+  /**
+   * Creates a new Step.
+   * 
+   * @param type StepType type of step
+   * @param file String file
+   */
+  public Step(StepType type, String file) {
+    this(type, file, Long.MIN_VALUE);
+  }
+
+  /**
+   * Creates a new Step.
+   * 
+   * @param type StepType type of step
+   * @param file String file
+   * @param size long size in bytes
+   */
+  public Step(StepType type, String file, long size) {
+    this.file = file;
+    this.sequenceNumber = SEQUENCE.incrementAndGet();
+    this.size = size;
+    this.type = type;
+  }
+
+  @Override
+  public int compareTo(Step other) {
+    // Sort steps by file and then sequentially within the file to achieve the
+    // desired order.  There is no concurrent map structure in the JDK that
+    // maintains insertion order, so instead we attach a sequence number to each
+    // step and sort on read.
+    return new CompareToBuilder().append(file, other.file)
+      .append(sequenceNumber, other.sequenceNumber).toComparison();
+  }
+
+  @Override
+  public boolean equals(Object otherObj) {
+    if (otherObj == null || otherObj.getClass() != getClass()) {
+      return false;
+    }
+    Step other = (Step)otherObj;
+    return new EqualsBuilder().append(this.file, other.file)
+      .append(this.size, other.size).append(this.type, other.type).isEquals();
+  }
+
+  /**
+   * Returns the optional file name, possibly null.
+   * 
+   * @return String optional file name, possibly null
+   */
+  public String getFile() {
+    return file;
+  }
+
+  /**
+   * Returns the optional size in bytes, possibly Long.MIN_VALUE if undefined.
+   * 
+   * @return long optional size in bytes, possibly Long.MIN_VALUE
+   */
+  public long getSize() {
+    return size;
+  }
+
+  /**
+   * Returns the optional step type, possibly null.
+   * 
+   * @return StepType optional step type, possibly null
+   */
+  public StepType getType() {
+    return type;
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder().append(file).append(size).append(type)
+      .toHashCode();
+  }
+}

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepTracking.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepTracking.java?rev=1502120&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepTracking.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepTracking.java Thu Jul 11 05:35:29 2013
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.startupprogress;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Internal data structure used to track progress of a {@link Step}.
+ */
+@InterfaceAudience.Private
+final class StepTracking extends AbstractTracking {
+  AtomicLong count = new AtomicLong();
+  long total = Long.MIN_VALUE;
+
+  @Override
+  public StepTracking clone() {
+    StepTracking clone = new StepTracking();
+    super.copy(clone);
+    clone.count = new AtomicLong(count.get());
+    clone.total = total;
+    return clone;
+  }
+}

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java?rev=1502120&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StepType.java Thu Jul 11 05:35:29 2013
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.startupprogress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Indicates a particular type of {@link Step}.
+ */
+@InterfaceAudience.Private
+public enum StepType {
+  /**
+   * The namenode has entered safemode and is awaiting block reports from
+   * datanodes.
+   */
+  AWAITING_REPORTED_BLOCKS("AwaitingReportedBlocks", "awaiting reported blocks"),
+
+  /**
+   * The namenode is performing an operation related to delegation keys.
+   */
+  DELEGATION_KEYS("DelegationKeys", "delegation keys"),
+
+  /**
+   * The namenode is performing an operation related to delegation tokens.
+   */
+  DELEGATION_TOKENS("DelegationTokens", "delegation tokens"),
+
+  /**
+   * The namenode is performing an operation related to inodes.
+   */
+  INODES("Inodes", "inodes");
+
+  private final String name, description;
+
+  /**
+   * Private constructor of enum.
+   * 
+   * @param name String step type name
+   * @param description String step type description
+   */
+  private StepType(String name, String description) {
+    this.name = name;
+    this.description = description;
+  }
+
+  /**
+   * Returns step type description.
+   * 
+   * @return String step type description
+   */
+  public String getDescription() {
+    return description;
+  }
+
+  /**
+   * Returns step type name.
+   * 
+   * @return String step type name
+   */
+  public String getName() {
+    return name;
+  }
+}

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/package-info.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/package-info.java?rev=1502120&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/package-info.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/package-info.java Thu Jul 11 05:35:29 2013
@@ -0,0 +1,43 @@
+/**
+ * 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.
+ */
+
+/**
+ * This package provides a mechanism for tracking {@link NameNode} startup
+ * progress.  The package models NameNode startup as a series of {@link Phase}s,
+ * with each phase further sub-divided into multiple {@link Step}s.  All phases
+ * are coarse-grained and typically known in advance, implied by the structure of
+ * the NameNode codebase (example: loading fsimage).  Steps are more granular and
+ * typically only known at runtime after startup begins (example: loading a
+ * specific fsimage file with a known length from a particular location).
+ * 
+ * {@link StartupProgress} provides a thread-safe data structure for
+ * recording status information and counters.  Various parts of the NameNode
+ * codebase use this to describe the NameNode's activities during startup.
+ * 
+ * {@link StartupProgressView} provides an immutable, consistent view of the
+ * current state of NameNode startup progress.  This can be used to present the
+ * data to a user.
+ * 
+ * {@link StartupProgressMetrics} exposes startup progress information via JMX
+ * through the standard metrics system.
+ */
+@InterfaceAudience.Private
+package org.apache.hadoop.hdfs.server.namenode.startupprogress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StartupProgressTestHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StartupProgressTestHelper.java?rev=1502120&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StartupProgressTestHelper.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/StartupProgressTestHelper.java Thu Jul 11 05:35:29 2013
@@ -0,0 +1,103 @@
+/**
+ * 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.startupprogress;
+
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.*;
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType.*;
+
+/**
+ * Utility methods that help with writing tests covering startup progress.
+ */
+public class StartupProgressTestHelper {
+
+  /**
+   * Increments a counter a certain number of times.
+   * 
+   * @param prog StartupProgress to increment
+   * @param phase Phase to increment
+   * @param step Step to increment
+   * @param delta long number of times to increment
+   */
+  public static void incrementCounter(StartupProgress prog, Phase phase,
+      Step step, long delta) {
+    StartupProgress.Counter counter = prog.getCounter(phase, step);
+    for (long i = 0; i < delta; ++i) {
+      counter.increment();
+    }
+  }
+
+  /**
+   * Sets up StartupProgress to a state part-way through the startup sequence.
+   * 
+   * @param prog StartupProgress to set
+   */
+  public static void setStartupProgressForRunningState(StartupProgress prog) {
+    prog.beginPhase(LOADING_FSIMAGE);
+    Step loadingFsImageInodes = new Step(INODES);
+    prog.beginStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    prog.setTotal(LOADING_FSIMAGE, loadingFsImageInodes, 100L);
+    incrementCounter(prog, LOADING_FSIMAGE, loadingFsImageInodes, 100L);
+    prog.endStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    prog.endPhase(LOADING_FSIMAGE);
+
+    prog.beginPhase(LOADING_EDITS);
+    Step loadingEditsFile = new Step("file", 1000L);
+    prog.beginStep(LOADING_EDITS, loadingEditsFile);
+    prog.setTotal(LOADING_EDITS, loadingEditsFile, 200L);
+    incrementCounter(prog, LOADING_EDITS, loadingEditsFile, 100L);
+  }
+
+  /**
+   * Sets up StartupProgress to final state after startup sequence has completed.
+   * 
+   * @param prog StartupProgress to set
+   */
+  public static void setStartupProgressForFinalState(StartupProgress prog) {
+    prog.beginPhase(LOADING_FSIMAGE);
+    Step loadingFsImageInodes = new Step(INODES);
+    prog.beginStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    prog.setTotal(LOADING_FSIMAGE, loadingFsImageInodes, 100L);
+    incrementCounter(prog, LOADING_FSIMAGE, loadingFsImageInodes, 100L);
+    prog.endStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    prog.endPhase(LOADING_FSIMAGE);
+
+    prog.beginPhase(LOADING_EDITS);
+    Step loadingEditsFile = new Step("file", 1000L);
+    prog.beginStep(LOADING_EDITS, loadingEditsFile);
+    prog.setTotal(LOADING_EDITS, loadingEditsFile, 200L);
+    incrementCounter(prog, LOADING_EDITS, loadingEditsFile, 200L);
+    prog.endStep(LOADING_EDITS, loadingEditsFile);
+    prog.endPhase(LOADING_EDITS);
+
+    prog.beginPhase(SAVING_CHECKPOINT);
+    Step savingCheckpointInodes = new Step(INODES);
+    prog.beginStep(SAVING_CHECKPOINT, savingCheckpointInodes);
+    prog.setTotal(SAVING_CHECKPOINT, savingCheckpointInodes, 300L);
+    incrementCounter(prog, SAVING_CHECKPOINT, savingCheckpointInodes, 300L);
+    prog.endStep(SAVING_CHECKPOINT, savingCheckpointInodes);
+    prog.endPhase(SAVING_CHECKPOINT);
+
+    prog.beginPhase(SAFEMODE);
+    Step awaitingBlocks = new Step(AWAITING_REPORTED_BLOCKS);
+    prog.beginStep(SAFEMODE, awaitingBlocks);
+    prog.setTotal(SAFEMODE, awaitingBlocks, 400L);
+    incrementCounter(prog, SAFEMODE, awaitingBlocks, 400L);
+    prog.endStep(SAFEMODE, awaitingBlocks);
+    prog.endPhase(SAFEMODE);
+  }
+}

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgress.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgress.java?rev=1502120&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgress.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgress.java Thu Jul 11 05:35:29 2013
@@ -0,0 +1,427 @@
+/**
+ * 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.startupprogress;
+
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.*;
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgressTestHelper.*;
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Status.*;
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType.*;
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestStartupProgress {
+
+  private StartupProgress startupProgress;
+
+  @Before
+  public void setUp() {
+    startupProgress = new StartupProgress();
+  }
+
+  @Test(timeout=10000)
+  public void testCounter() {
+    startupProgress.beginPhase(LOADING_FSIMAGE);
+    Step loadingFsImageInodes = new Step(INODES);
+    startupProgress.beginStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    incrementCounter(startupProgress, LOADING_FSIMAGE, loadingFsImageInodes,
+      100L);
+    startupProgress.endStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    Step loadingFsImageDelegationKeys = new Step(DELEGATION_KEYS);
+    startupProgress.beginStep(LOADING_FSIMAGE, loadingFsImageDelegationKeys);
+    incrementCounter(startupProgress, LOADING_FSIMAGE,
+      loadingFsImageDelegationKeys, 200L);
+    startupProgress.endStep(LOADING_FSIMAGE, loadingFsImageDelegationKeys);
+    startupProgress.endPhase(LOADING_FSIMAGE);
+
+    startupProgress.beginPhase(LOADING_EDITS);
+    Step loadingEditsFile = new Step("file", 1000L);
+    startupProgress.beginStep(LOADING_EDITS, loadingEditsFile);
+    incrementCounter(startupProgress, LOADING_EDITS, loadingEditsFile, 5000L);
+
+    StartupProgressView view = startupProgress.createView();
+    assertNotNull(view);
+    assertEquals(100L, view.getCount(LOADING_FSIMAGE, loadingFsImageInodes));
+    assertEquals(200L, view.getCount(LOADING_FSIMAGE,
+      loadingFsImageDelegationKeys));
+    assertEquals(5000L, view.getCount(LOADING_EDITS, loadingEditsFile));
+    assertEquals(0L, view.getCount(SAVING_CHECKPOINT,
+      new Step(INODES)));
+
+    // Increment a counter again and check that the existing view was not
+    // modified, but a new view shows the updated value.
+    incrementCounter(startupProgress, LOADING_EDITS, loadingEditsFile, 1000L);
+    startupProgress.endStep(LOADING_EDITS, loadingEditsFile);
+    startupProgress.endPhase(LOADING_EDITS);
+
+    assertEquals(5000L, view.getCount(LOADING_EDITS, loadingEditsFile));
+    view = startupProgress.createView();
+    assertNotNull(view);
+    assertEquals(6000L, view.getCount(LOADING_EDITS, loadingEditsFile));
+  }
+
+  @Test(timeout=10000)
+  public void testElapsedTime() throws Exception {
+    startupProgress.beginPhase(LOADING_FSIMAGE);
+    Step loadingFsImageInodes = new Step(INODES);
+    startupProgress.beginStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    Thread.sleep(50L); // brief sleep to fake elapsed time
+    startupProgress.endStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    Step loadingFsImageDelegationKeys = new Step(DELEGATION_KEYS);
+    startupProgress.beginStep(LOADING_FSIMAGE, loadingFsImageDelegationKeys);
+    Thread.sleep(50L); // brief sleep to fake elapsed time
+    startupProgress.endStep(LOADING_FSIMAGE, loadingFsImageDelegationKeys);
+    startupProgress.endPhase(LOADING_FSIMAGE);
+
+    startupProgress.beginPhase(LOADING_EDITS);
+    Step loadingEditsFile = new Step("file", 1000L);
+    startupProgress.beginStep(LOADING_EDITS, loadingEditsFile);
+    startupProgress.setTotal(LOADING_EDITS, loadingEditsFile, 10000L);
+    incrementCounter(startupProgress, LOADING_EDITS, loadingEditsFile, 5000L);
+    Thread.sleep(50L); // brief sleep to fake elapsed time
+
+    StartupProgressView view = startupProgress.createView();
+    assertNotNull(view);
+    assertTrue(view.getElapsedTime() > 0);
+
+    assertTrue(view.getElapsedTime(LOADING_FSIMAGE) > 0);
+    assertTrue(view.getElapsedTime(LOADING_FSIMAGE,
+      loadingFsImageInodes) > 0);
+    assertTrue(view.getElapsedTime(LOADING_FSIMAGE,
+      loadingFsImageDelegationKeys) > 0);
+
+    assertTrue(view.getElapsedTime(LOADING_EDITS) > 0);
+    assertTrue(view.getElapsedTime(LOADING_EDITS, loadingEditsFile) > 0);
+
+    assertTrue(view.getElapsedTime(SAVING_CHECKPOINT) == 0);
+    assertTrue(view.getElapsedTime(SAVING_CHECKPOINT,
+      new Step(INODES)) == 0);
+
+    // Brief sleep, then check that completed phases/steps have the same elapsed
+    // time, but running phases/steps have updated elapsed time.
+    long totalTime = view.getElapsedTime();
+    long loadingFsImageTime = view.getElapsedTime(LOADING_FSIMAGE);
+    long loadingFsImageInodesTime = view.getElapsedTime(LOADING_FSIMAGE,
+      loadingFsImageInodes);
+    long loadingFsImageDelegationKeysTime = view.getElapsedTime(LOADING_FSIMAGE,
+      loadingFsImageInodes);
+    long loadingEditsTime = view.getElapsedTime(LOADING_EDITS);
+    long loadingEditsFileTime = view.getElapsedTime(LOADING_EDITS,
+      loadingEditsFile);
+
+    Thread.sleep(50L);
+
+    assertTrue(totalTime < view.getElapsedTime());
+    assertEquals(loadingFsImageTime, view.getElapsedTime(LOADING_FSIMAGE));
+    assertEquals(loadingFsImageInodesTime, view.getElapsedTime(LOADING_FSIMAGE,
+      loadingFsImageInodes));
+    assertTrue(loadingEditsTime < view.getElapsedTime(LOADING_EDITS));
+    assertTrue(loadingEditsFileTime < view.getElapsedTime(LOADING_EDITS,
+      loadingEditsFile));
+  }
+
+  @Test(timeout=10000)
+  public void testFrozenAfterStartupCompletes() {
+    // Do some updates and counter increments.
+    startupProgress.beginPhase(LOADING_FSIMAGE);
+    startupProgress.setFile(LOADING_FSIMAGE, "file1");
+    startupProgress.setSize(LOADING_FSIMAGE, 1000L);
+    Step step = new Step(INODES);
+    startupProgress.beginStep(LOADING_FSIMAGE, step);
+    startupProgress.setTotal(LOADING_FSIMAGE, step, 10000L);
+    incrementCounter(startupProgress, LOADING_FSIMAGE, step, 100L);
+    startupProgress.endStep(LOADING_FSIMAGE, step);
+    startupProgress.endPhase(LOADING_FSIMAGE);
+
+    // Force completion of phases, so that entire startup process is completed.
+    for (Phase phase: EnumSet.allOf(Phase.class)) {
+      if (startupProgress.getStatus(phase) != Status.COMPLETE) {
+        startupProgress.beginPhase(phase);
+        startupProgress.endPhase(phase);
+      }
+    }
+
+    StartupProgressView before = startupProgress.createView();
+
+    // Attempt more updates and counter increments.
+    startupProgress.beginPhase(LOADING_FSIMAGE);
+    startupProgress.setFile(LOADING_FSIMAGE, "file2");
+    startupProgress.setSize(LOADING_FSIMAGE, 2000L);
+    startupProgress.beginStep(LOADING_FSIMAGE, step);
+    startupProgress.setTotal(LOADING_FSIMAGE, step, 20000L);
+    incrementCounter(startupProgress, LOADING_FSIMAGE, step, 100L);
+    startupProgress.endStep(LOADING_FSIMAGE, step);
+    startupProgress.endPhase(LOADING_FSIMAGE);
+
+    StartupProgressView after = startupProgress.createView();
+
+    // Expect that data was frozen after completion of entire startup process, so
+    // second set of updates and counter increments should have had no effect.
+    assertEquals(before.getCount(LOADING_FSIMAGE),
+      after.getCount(LOADING_FSIMAGE));
+    assertEquals(before.getCount(LOADING_FSIMAGE, step),
+      after.getCount(LOADING_FSIMAGE, step));
+    assertEquals(before.getElapsedTime(), after.getElapsedTime());
+    assertEquals(before.getElapsedTime(LOADING_FSIMAGE),
+      after.getElapsedTime(LOADING_FSIMAGE));
+    assertEquals(before.getElapsedTime(LOADING_FSIMAGE, step),
+      after.getElapsedTime(LOADING_FSIMAGE, step));
+    assertEquals(before.getFile(LOADING_FSIMAGE),
+      after.getFile(LOADING_FSIMAGE));
+    assertEquals(before.getSize(LOADING_FSIMAGE),
+      after.getSize(LOADING_FSIMAGE));
+    assertEquals(before.getTotal(LOADING_FSIMAGE),
+      after.getTotal(LOADING_FSIMAGE));
+    assertEquals(before.getTotal(LOADING_FSIMAGE, step),
+      after.getTotal(LOADING_FSIMAGE, step));
+  }
+
+  @Test(timeout=10000)
+  public void testInitialState() {
+    StartupProgressView view = startupProgress.createView();
+    assertNotNull(view);
+    assertEquals(0L, view.getElapsedTime());
+    assertEquals(0.0f, view.getPercentComplete(), 0.001f);
+    List<Phase> phases = new ArrayList<Phase>();
+
+    for (Phase phase: view.getPhases()) {
+      phases.add(phase);
+      assertEquals(0L, view.getElapsedTime(phase));
+      assertNull(view.getFile(phase));
+      assertEquals(0.0f, view.getPercentComplete(phase), 0.001f);
+      assertEquals(Long.MIN_VALUE, view.getSize(phase));
+      assertEquals(PENDING, view.getStatus(phase));
+      assertEquals(0L, view.getTotal(phase));
+
+      for (Step step: view.getSteps(phase)) {
+        fail(String.format("unexpected step %s in phase %s at initial state",
+          step, phase));
+      }
+    }
+
+    assertArrayEquals(EnumSet.allOf(Phase.class).toArray(), phases.toArray());
+  }
+
+  @Test(timeout=10000)
+  public void testPercentComplete() {
+    startupProgress.beginPhase(LOADING_FSIMAGE);
+    Step loadingFsImageInodes = new Step(INODES);
+    startupProgress.beginStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    startupProgress.setTotal(LOADING_FSIMAGE, loadingFsImageInodes, 1000L);
+    incrementCounter(startupProgress, LOADING_FSIMAGE, loadingFsImageInodes,
+      100L);
+    Step loadingFsImageDelegationKeys = new Step(DELEGATION_KEYS);
+    startupProgress.beginStep(LOADING_FSIMAGE, loadingFsImageDelegationKeys);
+    startupProgress.setTotal(LOADING_FSIMAGE, loadingFsImageDelegationKeys,
+      800L);
+    incrementCounter(startupProgress, LOADING_FSIMAGE,
+      loadingFsImageDelegationKeys, 200L);
+
+    startupProgress.beginPhase(LOADING_EDITS);
+    Step loadingEditsFile = new Step("file", 1000L);
+    startupProgress.beginStep(LOADING_EDITS, loadingEditsFile);
+    startupProgress.setTotal(LOADING_EDITS, loadingEditsFile, 10000L);
+    incrementCounter(startupProgress, LOADING_EDITS, loadingEditsFile, 5000L);
+
+    StartupProgressView view = startupProgress.createView();
+    assertNotNull(view);
+    assertEquals(0.167f, view.getPercentComplete(), 0.001f);
+    assertEquals(0.167f, view.getPercentComplete(LOADING_FSIMAGE), 0.001f);
+    assertEquals(0.10f, view.getPercentComplete(LOADING_FSIMAGE,
+      loadingFsImageInodes), 0.001f);
+    assertEquals(0.25f, view.getPercentComplete(LOADING_FSIMAGE,
+      loadingFsImageDelegationKeys), 0.001f);
+    assertEquals(0.5f, view.getPercentComplete(LOADING_EDITS), 0.001f);
+    assertEquals(0.5f, view.getPercentComplete(LOADING_EDITS, loadingEditsFile),
+      0.001f);
+    assertEquals(0.0f, view.getPercentComplete(SAVING_CHECKPOINT), 0.001f);
+    assertEquals(0.0f, view.getPercentComplete(SAVING_CHECKPOINT,
+      new Step(INODES)), 0.001f);
+
+    // End steps/phases, and confirm that they jump to 100% completion.
+    startupProgress.endStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    startupProgress.endStep(LOADING_FSIMAGE, loadingFsImageDelegationKeys);
+    startupProgress.endPhase(LOADING_FSIMAGE);
+    startupProgress.endStep(LOADING_EDITS, loadingEditsFile);
+    startupProgress.endPhase(LOADING_EDITS);
+
+    view = startupProgress.createView();
+    assertNotNull(view);
+    assertEquals(0.5f, view.getPercentComplete(), 0.001f);
+    assertEquals(1.0f, view.getPercentComplete(LOADING_FSIMAGE), 0.001f);
+    assertEquals(1.0f, view.getPercentComplete(LOADING_FSIMAGE,
+      loadingFsImageInodes), 0.001f);
+    assertEquals(1.0f, view.getPercentComplete(LOADING_FSIMAGE,
+      loadingFsImageDelegationKeys), 0.001f);
+    assertEquals(1.0f, view.getPercentComplete(LOADING_EDITS), 0.001f);
+    assertEquals(1.0f, view.getPercentComplete(LOADING_EDITS, loadingEditsFile),
+      0.001f);
+    assertEquals(0.0f, view.getPercentComplete(SAVING_CHECKPOINT), 0.001f);
+    assertEquals(0.0f, view.getPercentComplete(SAVING_CHECKPOINT,
+      new Step(INODES)), 0.001f);
+  }
+
+  @Test(timeout=10000)
+  public void testStatus() {
+    startupProgress.beginPhase(LOADING_FSIMAGE);
+    startupProgress.endPhase(LOADING_FSIMAGE);
+    startupProgress.beginPhase(LOADING_EDITS);
+    StartupProgressView view = startupProgress.createView();
+    assertNotNull(view);
+    assertEquals(COMPLETE, view.getStatus(LOADING_FSIMAGE));
+    assertEquals(RUNNING, view.getStatus(LOADING_EDITS));
+    assertEquals(PENDING, view.getStatus(SAVING_CHECKPOINT));
+  }
+
+  @Test(timeout=10000)
+  public void testStepSequence() {
+    // Test that steps are returned in the correct sort order (by file and then
+    // sequence number) by starting a few steps in a randomly shuffled order and
+    // then asserting that they are returned in the expected order.
+    Step[] expectedSteps = new Step[] {
+      new Step(INODES, "file1"),
+      new Step(DELEGATION_KEYS, "file1"),
+      new Step(INODES, "file2"),
+      new Step(DELEGATION_KEYS, "file2"),
+      new Step(INODES, "file3"),
+      new Step(DELEGATION_KEYS, "file3")
+    };
+
+    List<Step> shuffledSteps = new ArrayList<Step>(Arrays.asList(expectedSteps));
+    Collections.shuffle(shuffledSteps);
+
+    startupProgress.beginPhase(SAVING_CHECKPOINT);
+    for (Step step: shuffledSteps) {
+      startupProgress.beginStep(SAVING_CHECKPOINT, step);
+    }
+
+    List<Step> actualSteps = new ArrayList<Step>(expectedSteps.length);
+    StartupProgressView view = startupProgress.createView();
+    assertNotNull(view);
+    for (Step step: view.getSteps(SAVING_CHECKPOINT)) {
+      actualSteps.add(step);
+    }
+
+    assertArrayEquals(expectedSteps, actualSteps.toArray());
+  }
+
+  @Test(timeout=10000)
+  public void testThreadSafety() throws Exception {
+    // Test for thread safety by starting multiple threads that mutate the same
+    // StartupProgress instance in various ways.  We expect no internal
+    // corruption of data structures and no lost updates on counter increments.
+    int numThreads = 100;
+
+    // Data tables used by each thread to determine values to pass to APIs.
+    Phase[] phases = { LOADING_FSIMAGE, LOADING_FSIMAGE, LOADING_EDITS,
+      LOADING_EDITS };
+    Step[] steps = new Step[] { new Step(INODES), new Step(DELEGATION_KEYS),
+      new Step(INODES), new Step(DELEGATION_KEYS) };
+    String[] files = { "file1", "file1", "file2", "file2" };
+    long[] sizes = { 1000L, 1000L, 2000L, 2000L };
+    long[] totals = { 10000L, 20000L, 30000L, 40000L };
+
+    ExecutorService exec = Executors.newFixedThreadPool(numThreads);
+
+    try {
+      for (int i = 0; i < numThreads; ++i) {
+        final Phase phase = phases[i % phases.length];
+        final Step step = steps[i % steps.length];
+        final String file = files[i % files.length];
+        final long size = sizes[i % sizes.length];
+        final long total = totals[i % totals.length];
+
+        exec.submit(new Callable<Void>() {
+          @Override
+          public Void call() {
+            startupProgress.beginPhase(phase);
+            startupProgress.setFile(phase, file);
+            startupProgress.setSize(phase, size);
+            startupProgress.setTotal(phase, step, total);
+            incrementCounter(startupProgress, phase, step, 100L);
+            startupProgress.endStep(phase, step);
+            startupProgress.endPhase(phase);
+            return null;
+          }
+        });
+      }
+    } finally {
+      exec.shutdown();
+      assertTrue(exec.awaitTermination(10000L, TimeUnit.MILLISECONDS));
+    }
+
+    StartupProgressView view = startupProgress.createView();
+    assertNotNull(view);
+    assertEquals("file1", view.getFile(LOADING_FSIMAGE));
+    assertEquals(1000L, view.getSize(LOADING_FSIMAGE));
+    assertEquals(10000L, view.getTotal(LOADING_FSIMAGE, new Step(INODES)));
+    assertEquals(2500L, view.getCount(LOADING_FSIMAGE, new Step(INODES)));
+    assertEquals(20000L, view.getTotal(LOADING_FSIMAGE,
+      new Step(DELEGATION_KEYS)));
+    assertEquals(2500L, view.getCount(LOADING_FSIMAGE,
+      new Step(DELEGATION_KEYS)));
+
+    assertEquals("file2", view.getFile(LOADING_EDITS));
+    assertEquals(2000L, view.getSize(LOADING_EDITS));
+    assertEquals(30000L, view.getTotal(LOADING_EDITS, new Step(INODES)));
+    assertEquals(2500L, view.getCount(LOADING_EDITS, new Step(INODES)));
+    assertEquals(40000L, view.getTotal(LOADING_EDITS,
+      new Step(DELEGATION_KEYS)));
+    assertEquals(2500L, view.getCount(LOADING_EDITS, new Step(DELEGATION_KEYS)));
+  }
+
+  @Test(timeout=10000)
+  public void testTotal() {
+    startupProgress.beginPhase(LOADING_FSIMAGE);
+    Step loadingFsImageInodes = new Step(INODES);
+    startupProgress.beginStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    startupProgress.setTotal(LOADING_FSIMAGE, loadingFsImageInodes, 1000L);
+    startupProgress.endStep(LOADING_FSIMAGE, loadingFsImageInodes);
+    Step loadingFsImageDelegationKeys = new Step(DELEGATION_KEYS);
+    startupProgress.beginStep(LOADING_FSIMAGE, loadingFsImageDelegationKeys);
+    startupProgress.setTotal(LOADING_FSIMAGE, loadingFsImageDelegationKeys,
+      800L);
+    startupProgress.endStep(LOADING_FSIMAGE, loadingFsImageDelegationKeys);
+    startupProgress.endPhase(LOADING_FSIMAGE);
+
+    startupProgress.beginPhase(LOADING_EDITS);
+    Step loadingEditsFile = new Step("file", 1000L);
+    startupProgress.beginStep(LOADING_EDITS, loadingEditsFile);
+    startupProgress.setTotal(LOADING_EDITS, loadingEditsFile, 10000L);
+    startupProgress.endStep(LOADING_EDITS, loadingEditsFile);
+    startupProgress.endPhase(LOADING_EDITS);
+
+    StartupProgressView view = startupProgress.createView();
+    assertNotNull(view);
+    assertEquals(1000L, view.getTotal(LOADING_FSIMAGE, loadingFsImageInodes));
+    assertEquals(800L, view.getTotal(LOADING_FSIMAGE,
+      loadingFsImageDelegationKeys));
+    assertEquals(10000L, view.getTotal(LOADING_EDITS, loadingEditsFile));
+  }
+}

Added: hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgressMetrics.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgressMetrics.java?rev=1502120&view=auto
==============================================================================
--- hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgressMetrics.java (added)
+++ hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgressMetrics.java Thu Jul 11 05:35:29 2013
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.startupprogress;
+
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.*;
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgressTestHelper.*;
+import static org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType.*;
+import static org.apache.hadoop.test.MetricsAsserts.*;
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestStartupProgressMetrics {
+
+  private StartupProgress startupProgress;
+  private StartupProgressMetrics metrics;
+
+  @Before
+  public void setUp() {
+    mockMetricsSystem();
+    startupProgress = new StartupProgress();
+    metrics = new StartupProgressMetrics(startupProgress);
+  }
+
+  @Test
+  public void testInitialState() {
+    MetricsRecordBuilder builder = getMetrics(metrics, true);
+    assertCounter("ElapsedTime", 0L, builder);
+    assertGauge("PercentComplete", 0.0f, builder);
+    assertCounter("LoadingFsImageCount", 0L, builder);
+    assertCounter("LoadingFsImageElapsedTime", 0L, builder);
+    assertCounter("LoadingFsImageTotal", 0L, builder);
+    assertGauge("LoadingFsImagePercentComplete", 0.0f, builder);
+    assertCounter("LoadingEditsCount", 0L, builder);
+    assertCounter("LoadingEditsElapsedTime", 0L, builder);
+    assertCounter("LoadingEditsTotal", 0L, builder);
+    assertGauge("LoadingEditsPercentComplete", 0.0f, builder);
+    assertCounter("SavingCheckpointCount", 0L, builder);
+    assertCounter("SavingCheckpointElapsedTime", 0L, builder);
+    assertCounter("SavingCheckpointTotal", 0L, builder);
+    assertGauge("SavingCheckpointPercentComplete", 0.0f, builder);
+    assertCounter("SafeModeCount", 0L, builder);
+    assertCounter("SafeModeElapsedTime", 0L, builder);
+    assertCounter("SafeModeTotal", 0L, builder);
+    assertGauge("SafeModePercentComplete", 0.0f, builder);
+  }
+
+  @Test
+  public void testRunningState() {
+    setStartupProgressForRunningState(startupProgress);
+
+    MetricsRecordBuilder builder = getMetrics(metrics, true);
+    assertTrue(getLongCounter("ElapsedTime", builder) >= 0L);
+    assertGauge("PercentComplete", 0.375f, builder);
+    assertCounter("LoadingFsImageCount", 100L, builder);
+    assertTrue(getLongCounter("LoadingFsImageElapsedTime", builder) >= 0L);
+    assertCounter("LoadingFsImageTotal", 100L, builder);
+    assertGauge("LoadingFsImagePercentComplete", 1.0f, builder);
+    assertCounter("LoadingEditsCount", 100L, builder);
+    assertTrue(getLongCounter("LoadingEditsElapsedTime", builder) >= 0L);
+    assertCounter("LoadingEditsTotal", 200L, builder);
+    assertGauge("LoadingEditsPercentComplete", 0.5f, builder);
+    assertCounter("SavingCheckpointCount", 0L, builder);
+    assertCounter("SavingCheckpointElapsedTime", 0L, builder);
+    assertCounter("SavingCheckpointTotal", 0L, builder);
+    assertGauge("SavingCheckpointPercentComplete", 0.0f, builder);
+    assertCounter("SafeModeCount", 0L, builder);
+    assertCounter("SafeModeElapsedTime", 0L, builder);
+    assertCounter("SafeModeTotal", 0L, builder);
+    assertGauge("SafeModePercentComplete", 0.0f, builder);
+  }
+
+  @Test
+  public void testFinalState() {
+    setStartupProgressForFinalState(startupProgress);
+
+    MetricsRecordBuilder builder = getMetrics(metrics, true);
+    assertTrue(getLongCounter("ElapsedTime", builder) >= 0L);
+    assertGauge("PercentComplete", 1.0f, builder);
+    assertCounter("LoadingFsImageCount", 100L, builder);
+    assertTrue(getLongCounter("LoadingFsImageElapsedTime", builder) >= 0L);
+    assertCounter("LoadingFsImageTotal", 100L, builder);
+    assertGauge("LoadingFsImagePercentComplete", 1.0f, builder);
+    assertCounter("LoadingEditsCount", 200L, builder);
+    assertTrue(getLongCounter("LoadingEditsElapsedTime", builder) >= 0L);
+    assertCounter("LoadingEditsTotal", 200L, builder);
+    assertGauge("LoadingEditsPercentComplete", 1.0f, builder);
+    assertCounter("SavingCheckpointCount", 300L, builder);
+    assertTrue(getLongCounter("SavingCheckpointElapsedTime", builder) >= 0L);
+    assertCounter("SavingCheckpointTotal", 300L, builder);
+    assertGauge("SavingCheckpointPercentComplete", 1.0f, builder);
+    assertCounter("SafeModeCount", 400L, builder);
+    assertTrue(getLongCounter("SafeModeElapsedTime", builder) >= 0L);
+    assertCounter("SafeModeTotal", 400L, builder);
+    assertGauge("SafeModePercentComplete", 1.0f, builder);
+  }
+}