@@ -458,21 +458,6 @@ public void testCheckpointCancellationDuringUpload() throws Exception {
458458 cluster .transitionToStandby (0 );
459459 cluster .transitionToActive (1 );
460460
461- GenericTestUtils .waitFor (new Supplier <Boolean >() {
462- @ Override
463- public Boolean get () {
464- int transferThreadCount = 0 ;
465- ThreadMXBean threadBean = ManagementFactory .getThreadMXBean ();
466- ThreadInfo [] threads = threadBean .getThreadInfo (
467- threadBean .getAllThreadIds (), 1 );
468- for (ThreadInfo thread : threads ) {
469- if (thread .getThreadName ().startsWith ("TransferFsImageUpload" )) {
470- transferThreadCount ++;
471- }
472- }
473- return transferThreadCount == NUM_NNS - 1 ;
474- }
475- }, 1000 , 30000 );
476461
477462 // Wait to make sure background TransferFsImageUpload thread was cancelled.
478463 // This needs to be done before the next test in the suite starts, so that a
@@ -498,7 +483,66 @@ public Boolean get() {
498483 // Assert that former active did not accept the canceled checkpoint file.
499484 assertEquals (0 , nns [0 ].getFSImage ().getMostRecentCheckpointTxId ());
500485 }
501-
486+
487+ /**
488+ * Test standby namenode upload fsiamge to multiple other namenodes in parallel, in the
489+ * cluster with observer namenodes.
490+ */
491+ @ Test (timeout =60000 )
492+ public void testCheckpointParallelUpload () throws Exception {
493+ // Set dfs.namenode.checkpoint.txns differently on the first NN to avoid it
494+ // doing checkpoint when it becomes a standby
495+ cluster .getConfiguration (0 ).setInt (
496+ DFSConfigKeys .DFS_NAMENODE_CHECKPOINT_TXNS_KEY , 1000 );
497+
498+ // don't compress, we want a big image
499+ for (int i = 0 ; i < NUM_NNS ; i ++) {
500+ cluster .getConfiguration (i ).setBoolean (
501+ DFSConfigKeys .DFS_IMAGE_COMPRESS_KEY , false );
502+ }
503+
504+ // Throttle SBN upload to make it hang during upload to ANN, and enable parallel upload fsimage.
505+ for (int i = 1 ; i < NUM_NNS ; i ++) {
506+ cluster .getConfiguration (i ).setLong (
507+ DFSConfigKeys .DFS_IMAGE_TRANSFER_RATE_KEY , 100 );
508+ cluster .getConfiguration (i ).setBoolean (
509+ DFSConfigKeys .DFS_NAMENODE_CHECKPOINT_PARALLEL_UPLOAD_ENABLED_KEY , true );
510+ }
511+ for (int i = 0 ; i < NUM_NNS ; i ++) {
512+ cluster .restartNameNode (i );
513+ }
514+
515+ // update references to each of the nns
516+ setNNs ();
517+
518+ cluster .transitionToActive (0 );
519+
520+ doEdits (0 , 100 );
521+
522+ for (int i = 1 ; i < NUM_NNS ; i ++) {
523+ HATestUtil .waitForStandbyToCatchUp (nns [0 ], nns [i ]);
524+ HATestUtil .waitForCheckpoint (cluster , i , ImmutableList .of (104 ));
525+ }
526+ cluster .transitionToStandby (0 );
527+ cluster .transitionToActive (1 );
528+
529+ GenericTestUtils .waitFor (new Supplier <Boolean >() {
530+ @ Override
531+ public Boolean get () {
532+ int transferThreadCount = 0 ;
533+ ThreadMXBean threadBean = ManagementFactory .getThreadMXBean ();
534+ ThreadInfo [] threads = threadBean .getThreadInfo (
535+ threadBean .getAllThreadIds (), 1 );
536+ for (ThreadInfo thread : threads ) {
537+ if (thread .getThreadName ().startsWith ("TransferFsImageUpload" )) {
538+ transferThreadCount ++;
539+ }
540+ }
541+ return transferThreadCount == NUM_NNS - 1 ;
542+ }
543+ }, 1000 , 30000 );
544+ }
545+
502546 /**
503547 * Make sure that clients will receive StandbyExceptions even when a
504548 * checkpoint is in progress on the SBN, and therefore the StandbyCheckpointer
0 commit comments