|
@@ -34,6 +34,7 @@ import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamenodeServiceState;
|
|
|
import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
|
|
@@ -43,6 +44,7 @@ import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeat
|
|
|
import org.apache.hadoop.hdfs.server.federation.store.protocol.NamenodeHeartbeatResponse;
|
|
|
import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateNamenodeRegistrationRequest;
|
|
|
import org.apache.hadoop.hdfs.server.federation.store.records.MembershipState;
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.BeforeClass;
|
|
@@ -57,10 +59,14 @@ public class TestStateStoreMembershipState extends TestStateStoreBase {
|
|
|
|
|
|
@BeforeClass
|
|
|
public static void create() {
|
|
|
- // Reduce expirations to 5 seconds
|
|
|
+ // Reduce expirations to 2 seconds
|
|
|
getConf().setLong(
|
|
|
RBFConfigKeys.FEDERATION_STORE_MEMBERSHIP_EXPIRATION_MS,
|
|
|
- TimeUnit.SECONDS.toMillis(5));
|
|
|
+ TimeUnit.SECONDS.toMillis(2));
|
|
|
+ // Set deletion time to 2 seconds
|
|
|
+ getConf().setLong(
|
|
|
+ RBFConfigKeys.FEDERATION_STORE_MEMBERSHIP_EXPIRATION_DELETION_MS,
|
|
|
+ TimeUnit.SECONDS.toMillis(2));
|
|
|
}
|
|
|
|
|
|
@Before
|
|
@@ -363,8 +369,8 @@ public class TestStateStoreMembershipState extends TestStateStoreBase {
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testRegistrationExpired()
|
|
|
- throws InterruptedException, IOException {
|
|
|
+ public void testRegistrationExpiredAndDeletion()
|
|
|
+ throws InterruptedException, IOException, TimeoutException {
|
|
|
|
|
|
// Populate the state store with a single NN element
|
|
|
// 1) ns0:nn0 - Active
|
|
@@ -385,20 +391,32 @@ public class TestStateStoreMembershipState extends TestStateStoreBase {
|
|
|
assertNotNull(quorumEntry);
|
|
|
assertEquals(ROUTERS[0], quorumEntry.getRouterId());
|
|
|
assertEquals(FederationNamenodeServiceState.ACTIVE, quorumEntry.getState());
|
|
|
-
|
|
|
- // Wait past expiration (set in conf to 5 seconds)
|
|
|
- Thread.sleep(6000);
|
|
|
- // Reload cache
|
|
|
- assertTrue(getStateStore().loadCache(MembershipStore.class, true));
|
|
|
-
|
|
|
- // Verify entry is now expired and is no longer in the cache
|
|
|
- quorumEntry = getNamenodeRegistration(NAMESERVICES[0], NAMENODES[0]);
|
|
|
+ quorumEntry = getExpiredNamenodeRegistration(report.getNameserviceId(),
|
|
|
+ report.getNamenodeId());
|
|
|
assertNull(quorumEntry);
|
|
|
|
|
|
+ // Wait past expiration (set in conf to 2 seconds)
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ try {
|
|
|
+ assertTrue(getStateStore().loadCache(MembershipStore.class, true));
|
|
|
+ // Verify entry is expired and is no longer in the cache
|
|
|
+ return getNamenodeRegistration(NAMESERVICES[0], NAMENODES[0]) == null;
|
|
|
+ } catch (IOException e) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }, 100, 3000);
|
|
|
+
|
|
|
+ // Verify entry is in expired membership records
|
|
|
+ quorumEntry = getExpiredNamenodeRegistration(NAMESERVICES[0], NAMENODES[0]);
|
|
|
+ assertNotNull(quorumEntry);
|
|
|
+
|
|
|
// Verify entry is now expired and can't be used by RPC service
|
|
|
quorumEntry = getNamenodeRegistration(
|
|
|
report.getNameserviceId(), report.getNamenodeId());
|
|
|
assertNull(quorumEntry);
|
|
|
+ quorumEntry = getExpiredNamenodeRegistration(
|
|
|
+ report.getNameserviceId(), report.getNamenodeId());
|
|
|
+ assertNotNull(quorumEntry);
|
|
|
|
|
|
// Heartbeat again, updates dateModified
|
|
|
assertTrue(namenodeHeartbeat(report));
|
|
@@ -411,6 +429,36 @@ public class TestStateStoreMembershipState extends TestStateStoreBase {
|
|
|
assertNotNull(quorumEntry);
|
|
|
assertEquals(ROUTERS[0], quorumEntry.getRouterId());
|
|
|
assertEquals(FederationNamenodeServiceState.ACTIVE, quorumEntry.getState());
|
|
|
+ quorumEntry = getExpiredNamenodeRegistration(
|
|
|
+ report.getNameserviceId(), report.getNamenodeId());
|
|
|
+ assertNull(quorumEntry);
|
|
|
+
|
|
|
+ // Wait past expiration (set in conf to 2 seconds)
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ try {
|
|
|
+ assertTrue(getStateStore().loadCache(MembershipStore.class, true));
|
|
|
+ // Verify entry is expired and is no longer in the cache
|
|
|
+ return getNamenodeRegistration(NAMESERVICES[0], NAMENODES[0]) == null;
|
|
|
+ } catch (IOException e) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }, 100, 3000);
|
|
|
+
|
|
|
+ // Verify entry is in expired membership records
|
|
|
+ quorumEntry = getExpiredNamenodeRegistration(NAMESERVICES[0], NAMENODES[0]);
|
|
|
+ assertNotNull(quorumEntry);
|
|
|
+
|
|
|
+ // Wait past deletion (set in conf to 2 seconds)
|
|
|
+ GenericTestUtils.waitFor(() -> {
|
|
|
+ try {
|
|
|
+ assertTrue(getStateStore().loadCache(MembershipStore.class, true));
|
|
|
+ // Verify entry is deleted from even the expired membership records
|
|
|
+ return getExpiredNamenodeRegistration(NAMESERVICES[0], NAMENODES[0])
|
|
|
+ == null;
|
|
|
+ } catch (IOException e) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }, 100, 3000);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -441,6 +489,34 @@ public class TestStateStoreMembershipState extends TestStateStoreBase {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Get a single expired namenode membership record from the store.
|
|
|
+ *
|
|
|
+ * @param nsId The HDFS nameservice ID to search for
|
|
|
+ * @param nnId The HDFS namenode ID to search for
|
|
|
+ * @return The single expired NamenodeMembershipRecord that matches the query
|
|
|
+ * or null if not found.
|
|
|
+ * @throws IOException if the query could not be executed.
|
|
|
+ */
|
|
|
+ private MembershipState getExpiredNamenodeRegistration(
|
|
|
+ final String nsId, final String nnId) throws IOException {
|
|
|
+
|
|
|
+ MembershipState partial = MembershipState.newInstance();
|
|
|
+ partial.setNameserviceId(nsId);
|
|
|
+ partial.setNamenodeId(nnId);
|
|
|
+ GetNamenodeRegistrationsRequest request =
|
|
|
+ GetNamenodeRegistrationsRequest.newInstance(partial);
|
|
|
+ GetNamenodeRegistrationsResponse response =
|
|
|
+ membershipStore.getExpiredNamenodeRegistrations(request);
|
|
|
+
|
|
|
+ List<MembershipState> results = response.getNamenodeMemberships();
|
|
|
+ if (results != null && results.size() == 1) {
|
|
|
+ MembershipState record = results.get(0);
|
|
|
+ return record;
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Register a namenode heartbeat with the state store.
|
|
|
*
|