HDFS-14500. NameNode StartupProgress should not allow new steps in an already-completed phase. Contributed by Erik Krogen.
(cherry-picked from55e0c134f0
) (cherry-picked fromc26cf22d6b
) (cherry-picked from9295e95174
)
This commit is contained in:
parent
2afc770443
commit
f024d27cd9
|
@ -84,13 +84,14 @@ public class StartupProgress {
|
|||
}
|
||||
|
||||
/**
|
||||
* Begins execution of the specified step within the specified phase.
|
||||
* Begins execution of the specified step within the specified phase. This is
|
||||
* a no-op if the phase is already completed.
|
||||
*
|
||||
* @param phase Phase to begin
|
||||
* @param phase Phase within which the step should be started
|
||||
* @param step Step to begin
|
||||
*/
|
||||
public void beginStep(Phase phase, Step step) {
|
||||
if (!isComplete()) {
|
||||
if (!isComplete(phase)) {
|
||||
lazyInitStep(phase, step).beginTime = monotonicNow();
|
||||
}
|
||||
}
|
||||
|
@ -107,13 +108,14 @@ public class StartupProgress {
|
|||
}
|
||||
|
||||
/**
|
||||
* Ends execution of the specified step within the specified phase.
|
||||
*
|
||||
* @param phase Phase to end
|
||||
* Ends execution of the specified step within the specified phase. This is
|
||||
* a no-op if the phase is already completed.
|
||||
*
|
||||
* @param phase Phase within which the step should be ended
|
||||
* @param step Step to end
|
||||
*/
|
||||
public void endStep(Phase phase, Step step) {
|
||||
if (!isComplete()) {
|
||||
if (!isComplete(phase)) {
|
||||
lazyInitStep(phase, step).endTime = monotonicNow();
|
||||
}
|
||||
}
|
||||
|
@ -149,7 +151,7 @@ public class StartupProgress {
|
|||
* @return Counter associated with phase and step
|
||||
*/
|
||||
public Counter getCounter(Phase phase, Step step) {
|
||||
if (!isComplete()) {
|
||||
if (!isComplete(phase)) {
|
||||
final StepTracking tracking = lazyInitStep(phase, step);
|
||||
return new Counter() {
|
||||
@Override
|
||||
|
@ -243,13 +245,23 @@ public class StartupProgress {
|
|||
*/
|
||||
private boolean isComplete() {
|
||||
for (Phase phase: EnumSet.allOf(Phase.class)) {
|
||||
if (getStatus(phase) != Status.COMPLETE) {
|
||||
if (!isComplete(phase)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the given startup phase has been completed.
|
||||
*
|
||||
* @param phase Which phase to check for completion
|
||||
* @return boolean true if the given startup phase has completed.
|
||||
*/
|
||||
private boolean isComplete(Phase phase) {
|
||||
return getStatus(phase) == Status.COMPLETE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Lazily initializes the internal data structure for tracking the specified
|
||||
* phase and step. Returns either the newly initialized data structure or the
|
||||
|
|
|
@ -159,6 +159,14 @@ public class TestStartupProgress {
|
|||
startupProgress.endStep(LOADING_FSIMAGE, step);
|
||||
startupProgress.endPhase(LOADING_FSIMAGE);
|
||||
|
||||
StartupProgressView beforePhaseUpdate = startupProgress.createView();
|
||||
|
||||
// LOADING_FSIMAGE phase has been completed, but attempt more updates to it
|
||||
Step fsimageStep2 = new Step(INODES);
|
||||
startupProgress.beginStep(LOADING_FSIMAGE, fsimageStep2);
|
||||
incrementCounter(startupProgress, LOADING_FSIMAGE, fsimageStep2, 1000000L);
|
||||
startupProgress.endStep(LOADING_FSIMAGE, fsimageStep2);
|
||||
|
||||
// Force completion of phases, so that entire startup process is completed.
|
||||
for (Phase phase: EnumSet.allOf(Phase.class)) {
|
||||
if (startupProgress.getStatus(phase) != Status.COMPLETE) {
|
||||
|
@ -191,26 +199,39 @@ public class TestStartupProgress {
|
|||
|
||||
// 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));
|
||||
assertViewEquals(before, after, LOADING_FSIMAGE, step, fsimageStep2);
|
||||
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));
|
||||
|
||||
// After the phase was completed but before startup was completed,
|
||||
// everything should be equal, except for the total elapsed time
|
||||
assertViewEquals(beforePhaseUpdate, after, LOADING_FSIMAGE,
|
||||
step, fsimageStep2);
|
||||
|
||||
assertFalse(after.getSteps(LOADING_EDITS).iterator().hasNext());
|
||||
}
|
||||
|
||||
private void assertViewEquals(StartupProgressView view1,
|
||||
StartupProgressView view2, Phase phaseToVerify, Step... stepsToVerify) {
|
||||
assertEquals(view1.getCount(phaseToVerify),
|
||||
view2.getCount(phaseToVerify));
|
||||
assertEquals(view1.getElapsedTime(phaseToVerify),
|
||||
view2.getElapsedTime(phaseToVerify));
|
||||
assertEquals(view1.getFile(phaseToVerify),
|
||||
view2.getFile(phaseToVerify));
|
||||
assertEquals(view1.getSize(phaseToVerify),
|
||||
view2.getSize(phaseToVerify));
|
||||
assertEquals(view1.getTotal(phaseToVerify),
|
||||
view2.getTotal(phaseToVerify));
|
||||
for (Step step : stepsToVerify) {
|
||||
assertEquals(view1.getCount(phaseToVerify, step),
|
||||
view2.getCount(phaseToVerify, step));
|
||||
assertEquals(view1.getElapsedTime(phaseToVerify, step),
|
||||
view2.getElapsedTime(phaseToVerify, step));
|
||||
assertEquals(view1.getTotal(phaseToVerify, step),
|
||||
view2.getTotal(phaseToVerify, step));
|
||||
}
|
||||
}
|
||||
|
||||
@Test(timeout=10000)
|
||||
public void testInitialState() {
|
||||
StartupProgressView view = startupProgress.createView();
|
||||
|
@ -375,7 +396,6 @@ public class TestStartupProgress {
|
|||
startupProgress.setTotal(phase, step, total);
|
||||
incrementCounter(startupProgress, phase, step, 100L);
|
||||
startupProgress.endStep(phase, step);
|
||||
startupProgress.endPhase(phase);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -384,6 +404,11 @@ public class TestStartupProgress {
|
|||
exec.shutdown();
|
||||
assertTrue(exec.awaitTermination(10000L, TimeUnit.MILLISECONDS));
|
||||
}
|
||||
// Once a phase ends, future modifications to the steps in that phase are
|
||||
// ignored. Thus do not end the phases until after the other ops are done.
|
||||
for (Phase phase : phases) {
|
||||
startupProgress.endPhase(phase);
|
||||
}
|
||||
|
||||
StartupProgressView view = startupProgress.createView();
|
||||
assertNotNull(view);
|
||||
|
|
Loading…
Reference in New Issue