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