|
31 | 31 | import org.apache.fluss.server.kv.snapshot.ZooKeeperCompletedSnapshotHandleStore; |
32 | 32 | import org.apache.fluss.server.tablet.TabletServer; |
33 | 33 | import org.apache.fluss.server.testutils.FlussClusterExtension; |
| 34 | +import org.apache.fluss.server.zk.data.LeaderAndIsr; |
34 | 35 | import org.apache.fluss.utils.FlussPaths; |
35 | 36 | import org.apache.fluss.utils.types.Tuple2; |
36 | 37 |
|
|
41 | 42 | import java.io.File; |
42 | 43 | import java.time.Duration; |
43 | 44 | import java.util.ArrayList; |
| 45 | +import java.util.Collections; |
44 | 46 | import java.util.HashMap; |
45 | 47 | import java.util.HashSet; |
46 | 48 | import java.util.List; |
@@ -435,6 +437,187 @@ void testStandbyIncrementalSnapshotDownload() throws Exception { |
435 | 437 | assertThat(standbySnapshotManager.getStandbySnapshotSize()).isGreaterThan(0); |
436 | 438 | } |
437 | 439 |
|
| 440 | + @Test |
| 441 | + void testStandbyDemotionAndReStandby() throws Exception { |
| 442 | + TablePath tablePath = TablePath.of("test_db", "test_table_standby_demotion"); |
| 443 | + long tableId = createTable(FLUSS_CLUSTER_EXTENSION, tablePath, DATA1_TABLE_DESCRIPTOR_PK); |
| 444 | + TableBucket tb0 = new TableBucket(tableId, 0); |
| 445 | + |
| 446 | + FLUSS_CLUSTER_EXTENSION.waitAndGetLeaderReplica(tb0); |
| 447 | + int leaderServer = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb0); |
| 448 | + |
| 449 | + // put some data and wait for snapshot |
| 450 | + KvRecordBatch kvRecordBatch = |
| 451 | + genKvRecordBatch( |
| 452 | + Tuple2.of("k1", new Object[] {1, "k1"}), |
| 453 | + Tuple2.of("k2", new Object[] {2, "k2"})); |
| 454 | + PutKvRequest putKvRequest = newPutKvRequest(tableId, 0, -1, kvRecordBatch); |
| 455 | + TabletServerGateway leaderGateway = |
| 456 | + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leaderServer); |
| 457 | + leaderGateway.putKv(putKvRequest).get(); |
| 458 | + |
| 459 | + // wait for snapshot |
| 460 | + waitValue( |
| 461 | + () -> completedSnapshotHandleStore.get(tb0, 0), |
| 462 | + Duration.ofMinutes(2), |
| 463 | + "Fail to wait for snapshot 0"); |
| 464 | + |
| 465 | + // verify standby replica has downloaded snapshot |
| 466 | + int standbyServer = FLUSS_CLUSTER_EXTENSION.waitAndGetStandby(tb0); |
| 467 | + TabletServer standbyTs = FLUSS_CLUSTER_EXTENSION.getTabletServerById(standbyServer); |
| 468 | + Replica standbyReplica = standbyTs.getReplicaManager().getReplicaOrException(tb0); |
| 469 | + assertThat(standbyReplica.isStandby()).isTrue(); |
| 470 | + |
| 471 | + KvSnapshotManager standbySnapshotManager = standbyReplica.getKvSnapshotManager(); |
| 472 | + retry( |
| 473 | + Duration.ofMinutes(1), |
| 474 | + () -> assertThat(standbySnapshotManager.getDownloadedSstFiles()).isNotEmpty()); |
| 475 | + |
| 476 | + // get current leader and isr |
| 477 | + LeaderAndIsr leaderAndIsr = FLUSS_CLUSTER_EXTENSION.waitLeaderAndIsrReady(tb0); |
| 478 | + List<Integer> replicas = new ArrayList<>(leaderAndIsr.isr()); |
| 479 | + |
| 480 | + // demote standby to regular follower by sending notifyLeaderAndIsr without standby |
| 481 | + LeaderAndIsr newLeaderAndIsr = |
| 482 | + new LeaderAndIsr( |
| 483 | + leaderAndIsr.leader(), |
| 484 | + leaderAndIsr.leaderEpoch() + 1, |
| 485 | + leaderAndIsr.isr(), |
| 486 | + Collections.emptyList(), // no standby replicas |
| 487 | + leaderAndIsr.coordinatorEpoch(), |
| 488 | + leaderAndIsr.bucketEpoch()); |
| 489 | + |
| 490 | + FLUSS_CLUSTER_EXTENSION.notifyLeaderAndIsr( |
| 491 | + standbyServer, tablePath, tb0, newLeaderAndIsr, replicas); |
| 492 | + |
| 493 | + // verify the replica is no longer standby |
| 494 | + retry( |
| 495 | + Duration.ofMinutes(1), |
| 496 | + () -> { |
| 497 | + Replica replica = standbyTs.getReplicaManager().getReplicaOrException(tb0); |
| 498 | + assertThat(replica.isStandby()).isFalse(); |
| 499 | + // verify kv tablet is dropped when demoted from standby |
| 500 | + assertThat(replica.getKvTablet()).isNull(); |
| 501 | + // verify standby download cache is cleared |
| 502 | + KvSnapshotManager snapshotManager = replica.getKvSnapshotManager(); |
| 503 | + assertThat(snapshotManager.getDownloadedSstFiles()).isNull(); |
| 504 | + assertThat(snapshotManager.getDownloadedMiscFiles()).isNull(); |
| 505 | + assertThat(snapshotManager.getStandbySnapshotSize()).isEqualTo(0); |
| 506 | + }); |
| 507 | + |
| 508 | + // put more data and create another snapshot |
| 509 | + kvRecordBatch = |
| 510 | + genKvRecordBatch( |
| 511 | + Tuple2.of("k3", new Object[] {3, "k3"}), |
| 512 | + Tuple2.of("k4", new Object[] {4, "k4"})); |
| 513 | + putKvRequest = newPutKvRequest(tableId, 0, 1, kvRecordBatch); |
| 514 | + leaderGateway.putKv(putKvRequest).get(); |
| 515 | + |
| 516 | + waitValue( |
| 517 | + () -> completedSnapshotHandleStore.get(tb0, 1), |
| 518 | + Duration.ofMinutes(2), |
| 519 | + "Fail to wait for snapshot 1"); |
| 520 | + |
| 521 | + // re-promote the replica back to standby by sending notifyLeaderAndIsr with standby |
| 522 | + LeaderAndIsr reStandbyLeaderAndIsr = |
| 523 | + new LeaderAndIsr( |
| 524 | + newLeaderAndIsr.leader(), |
| 525 | + newLeaderAndIsr.leaderEpoch() + 1, |
| 526 | + newLeaderAndIsr.isr(), |
| 527 | + Collections.singletonList(standbyServer), |
| 528 | + newLeaderAndIsr.coordinatorEpoch(), |
| 529 | + newLeaderAndIsr.bucketEpoch()); // re-add as standby |
| 530 | + |
| 531 | + FLUSS_CLUSTER_EXTENSION.notifyLeaderAndIsr( |
| 532 | + standbyServer, tablePath, tb0, reStandbyLeaderAndIsr, replicas); |
| 533 | + |
| 534 | + // verify the replica is standby again and downloads the latest snapshot |
| 535 | + retry( |
| 536 | + Duration.ofMinutes(1), |
| 537 | + () -> { |
| 538 | + Replica replica = standbyTs.getReplicaManager().getReplicaOrException(tb0); |
| 539 | + assertThat(replica.isStandby()).isTrue(); |
| 540 | + KvSnapshotManager snapshotManager = replica.getKvSnapshotManager(); |
| 541 | + assertThat(snapshotManager).isNotNull(); |
| 542 | + assertThat(snapshotManager.getDownloadedSstFiles()).isNotEmpty(); |
| 543 | + // verify it has the latest snapshot data |
| 544 | + assertThat(snapshotManager.getStandbySnapshotSize()).isGreaterThan(0); |
| 545 | + }); |
| 546 | + } |
| 547 | + |
| 548 | + @Test |
| 549 | + void testStandbySnapshotDownloadFailureAndRecovery() throws Exception { |
| 550 | + TablePath tablePath = TablePath.of("test_db", "test_table_download_failure"); |
| 551 | + long tableId = createTable(FLUSS_CLUSTER_EXTENSION, tablePath, DATA1_TABLE_DESCRIPTOR_PK); |
| 552 | + TableBucket tb0 = new TableBucket(tableId, 0); |
| 553 | + |
| 554 | + FLUSS_CLUSTER_EXTENSION.waitAndGetLeaderReplica(tb0); |
| 555 | + int leaderServer = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb0); |
| 556 | + |
| 557 | + // put data and create snapshot |
| 558 | + KvRecordBatch kvRecordBatch = |
| 559 | + genKvRecordBatch( |
| 560 | + Tuple2.of("k1", new Object[] {1, "k1"}), |
| 561 | + Tuple2.of("k2", new Object[] {2, "k2"})); |
| 562 | + PutKvRequest putKvRequest = newPutKvRequest(tableId, 0, -1, kvRecordBatch); |
| 563 | + TabletServerGateway leaderGateway = |
| 564 | + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leaderServer); |
| 565 | + leaderGateway.putKv(putKvRequest).get(); |
| 566 | + |
| 567 | + // wait for first snapshot |
| 568 | + CompletedSnapshot snapshot0 = |
| 569 | + waitValue( |
| 570 | + () -> completedSnapshotHandleStore.get(tb0, 0), |
| 571 | + Duration.ofMinutes(2), |
| 572 | + "Fail to wait for snapshot 0") |
| 573 | + .retrieveCompleteSnapshot(); |
| 574 | + |
| 575 | + // get standby server and verify it's marked as standby even if download fails initially |
| 576 | + int standbyServer = FLUSS_CLUSTER_EXTENSION.waitAndGetStandby(tb0); |
| 577 | + TabletServer standbyTs = FLUSS_CLUSTER_EXTENSION.getTabletServerById(standbyServer); |
| 578 | + Replica standbyReplica = standbyTs.getReplicaManager().getReplicaOrException(tb0); |
| 579 | + |
| 580 | + // verify replica is marked as standby (even if download might fail) |
| 581 | + retry(Duration.ofMinutes(1), () -> assertThat(standbyReplica.isStandby()).isTrue()); |
| 582 | + |
| 583 | + // wait for standby to eventually download snapshot (may retry on failures) |
| 584 | + KvSnapshotManager standbySnapshotManager = standbyReplica.getKvSnapshotManager(); |
| 585 | + assertThat(standbySnapshotManager).isNotNull(); |
| 586 | + |
| 587 | + // verify snapshot is eventually downloaded successfully |
| 588 | + retry( |
| 589 | + Duration.ofMinutes(1), |
| 590 | + () -> { |
| 591 | + assertThat(standbySnapshotManager.getDownloadedSstFiles()).isNotEmpty(); |
| 592 | + assertThat(standbySnapshotManager.getStandbySnapshotSize()) |
| 593 | + .isEqualTo(snapshot0.getSnapshotSize()); |
| 594 | + }); |
| 595 | + |
| 596 | + // create another snapshot to verify recovery continues to work |
| 597 | + kvRecordBatch = |
| 598 | + genKvRecordBatch( |
| 599 | + Tuple2.of("k3", new Object[] {3, "k3"}), |
| 600 | + Tuple2.of("k4", new Object[] {4, "k4"})); |
| 601 | + putKvRequest = newPutKvRequest(tableId, 0, 1, kvRecordBatch); |
| 602 | + leaderGateway.putKv(putKvRequest).get(); |
| 603 | + |
| 604 | + CompletedSnapshot snapshot1 = |
| 605 | + waitValue( |
| 606 | + () -> completedSnapshotHandleStore.get(tb0, 1), |
| 607 | + Duration.ofMinutes(2), |
| 608 | + "Fail to wait for snapshot 1") |
| 609 | + .retrieveCompleteSnapshot(); |
| 610 | + |
| 611 | + // verify standby continues to download new snapshots after recovery |
| 612 | + retry( |
| 613 | + Duration.ofMinutes(1), |
| 614 | + () -> { |
| 615 | + assertThat(standbySnapshotManager.getDownloadedSstFiles()).isNotEmpty(); |
| 616 | + assertThat(standbySnapshotManager.getStandbySnapshotSize()) |
| 617 | + .isEqualTo(snapshot1.getSnapshotSize()); |
| 618 | + }); |
| 619 | + } |
| 620 | + |
438 | 621 | private void checkDirsDeleted(Set<File> bucketDirs, Map<Long, TablePath> tablePathMap) { |
439 | 622 | for (File bucketDir : bucketDirs) { |
440 | 623 | retry(Duration.ofMinutes(1), () -> assertThat(bucketDir.exists()).isFalse()); |
|
0 commit comments