You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2019/10/25 15:59:31 UTC
[geode] 01/01: GEODE-7358: Membership code should use
InternalDistributedMember as the membership identifier
This is an automated email from the ASF dual-hosted git repository.
bschuchardt pushed a commit to branch feature/GEODE-7358
in repository https://gitbox.apache.org/repos/asf/geode.git
commit c84e9a81735036b1466d00a0defdbe96757ae496
Author: Bruce Schuchardt <bs...@pivotal.io>
AuthorDate: Fri Oct 25 08:57:29 2019 -0700
GEODE-7358: Membership code should use InternalDistributedMember as the membership identifier
Initial refactor: removal of NetMember, MemberAttributes and other classes that subordinated GMS
identifiers. Repurposing GMSMember as a data packet to hold information required in a membership
identifer.
---
.../cache30/DistributedAckRegionCCEDUnitTest.java | 5 +-
.../apache/geode/cache30/ReconnectDUnitTest.java | 7 +-
.../distributed/DistributedMemberDUnitTest.java | 10 +-
.../distributed/DistributedSystemDUnitTest.java | 4 +-
.../TcpServerBackwardCompatDUnitTest.java | 4 +-
.../cache/client/ClientCacheFactoryJUnitTest.java | 19 +-
.../internal/membership/MembershipJUnitTest.java | 18 +-
.../adapter/GMSMembershipManagerJUnitTest.java | 35 +-
.../gms/fd/GMSHealthMonitorJUnitTest.java | 126 +++---
.../gms/locator/GMSLocatorIntegrationTest.java | 4 +-
.../locator/GMSLocatorRecoveryIntegrationTest.java | 9 +-
.../gms/membership/GMSJoinLeaveJUnitTest.java | 176 ++++----
.../gms/messenger/GMSEncryptJUnitTest.java | 11 +-
.../gms/messenger/GMSQuorumCheckerJUnitTest.java | 35 +-
.../gms/messenger/JGroupsMessengerJUnitTest.java | 93 ++---
.../cache/FilterProfileConcurrencyTest.java | 3 +-
.../internal/ClusterDistributionManager.java | 32 +-
.../internal/LonerDistributionManager.java | 4 +-
.../membership/InternalDistributedMember.java | 447 +++++++++++----------
.../internal/membership/MemberAttributes.java | 126 ------
.../internal/membership/MembershipView.java | 2 +-
.../distributed/internal/membership/NetMember.java | 115 ------
.../internal/membership/NetMemberFactory.java | 111 -----
.../membership/adapter/GMSMemberAdapter.java | 230 -----------
.../membership/adapter/GMSMembershipManager.java | 95 ++---
.../membership/adapter/GMSMessageAdapter.java | 33 +-
.../membership/adapter/auth/GMSAuthenticator.java | 18 +-
.../gms/{GMSMember.java => GMSMemberData.java} | 242 ++++-------
.../internal/membership/gms/GMSMembershipView.java | 162 ++++----
.../internal/membership/gms/GMSUtil.java | 35 +-
.../membership/gms/MemberDataBuilderImpl.java | 160 ++++++++
...cator.java => MemberIdentifierFactoryImpl.java} | 36 +-
.../membership/gms/MemberIdentifierImpl.java | 117 ++++++
.../membership/gms/MembershipBuilderImpl.java | 13 +-
.../internal/membership/gms/Services.java | 22 +-
.../internal/membership/gms/SuspectMember.java | 10 +-
.../internal/membership/gms/api/Authenticator.java | 6 +-
.../internal/membership/gms/api/MemberData.java | 135 +++++++
.../membership/gms/api/MemberDataBuilder.java | 95 +++++
.../membership/gms/api/MemberIdentifier.java | 98 +++++
...enticator.java => MemberIdentifierFactory.java} | 28 +-
.../membership/gms/api/MembershipBuilder.java | 4 +
.../membership/gms/fd/GMSHealthMonitor.java | 128 +++---
.../membership/gms/interfaces/GMSMessage.java | 12 +-
.../membership/gms/interfaces/HealthMonitor.java | 10 +-
.../membership/gms/interfaces/JoinLeave.java | 12 +-
.../membership/gms/interfaces/Manager.java | 4 +-
.../membership/gms/interfaces/Messenger.java | 20 +-
.../membership/gms/interfaces/Service.java | 6 +-
.../gms/locator/FindCoordinatorRequest.java | 20 +-
.../gms/locator/FindCoordinatorResponse.java | 24 +-
.../membership/gms/locator/GMSLocator.java | 40 +-
.../membership/gms/membership/GMSJoinLeave.java | 303 +++++++-------
.../gms/messages/AbstractGMSMessage.java | 20 +-
.../gms/messages/FinalCheckPassedMessage.java | 10 +-
.../membership/gms/messages/HasMemberID.java | 4 +-
.../gms/messages/HeartbeatRequestMessage.java | 8 +-
.../gms/messages/JoinRequestMessage.java | 8 +-
.../gms/messages/JoinResponseMessage.java | 10 +-
.../gms/messages/LeaveRequestMessage.java | 12 +-
.../gms/messages/NetworkPartitionMessage.java | 4 +-
.../gms/messages/RemoveMemberMessage.java | 12 +-
.../gms/messages/SuspectMembersMessage.java | 4 +-
.../membership/gms/messages/SuspectRequest.java | 8 +-
.../membership/gms/messages/ViewAckMessage.java | 6 +-
.../membership/gms/messenger/GMSEncrypt.java | 26 +-
.../GMSMemberWrapper.java} | 46 +--
.../membership/gms/messenger/GMSQuorumChecker.java | 26 +-
.../membership/gms/messenger/JGAddress.java | 12 +-
.../membership/gms/messenger/JGroupsMessenger.java | 122 +++---
.../gms/messenger/MembershipInformation.java | 8 +-
.../org/apache/geode/internal/DSFIDFactory.java | 2 -
.../geode/internal/cache/CacheServerImpl.java | 4 +-
.../geode/internal/cache/EventStateHelper.java | 30 +-
.../geode/internal/cache/PartitionedRegion.java | 4 +-
.../cache/tier/sockets/CacheClientUpdater.java | 10 +-
.../tier/sockets/ClientProxyMembershipID.java | 2 +-
.../membership/InternalDistributedMemberTest.java | 25 +-
.../membership/MembershipViewJUnitTest.java | 143 +++----
.../auth/AbstractGMSAuthenticatorTestCase.java | 4 +-
...rJUnitTest.java => GMSMemberDataJUnitTest.java} | 78 ++--
.../gms/api/MembershipAPIArchUnitTest.java | 5 +-
.../cli/commands/ExportLogsCommandTest.java | 10 +-
.../serialization/DataSerializableFixedID.java | 2 +-
84 files changed, 2073 insertions(+), 2106 deletions(-)
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedAckRegionCCEDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedAckRegionCCEDUnitTest.java
index bd9cb41..7e6a4ac 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedAckRegionCCEDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/DistributedAckRegionCCEDUnitTest.java
@@ -41,7 +41,7 @@ import org.apache.geode.cache.Scope;
import org.apache.geode.distributed.internal.ClusterDistributionManager;
import org.apache.geode.distributed.internal.DistributionAdvisor;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.NetMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberData;
import org.apache.geode.internal.cache.DistributedCacheOperation;
import org.apache.geode.internal.cache.DistributedRegion;
import org.apache.geode.internal.cache.EntryEventImpl;
@@ -240,7 +240,8 @@ public class DistributedAckRegionCCEDUnitTest extends DistributedAckRegionDUnitT
VersionTag<InternalDistributedMember> tag =
(VersionTag<InternalDistributedMember>) versionStamp.asVersionTag();
// create a fake member ID that will be < mine and lose a concurrency check
- NetMember nm = CCRegion.getDistributionManager().getDistributionManagerId().getNetMember();
+ MemberData nm =
+ CCRegion.getDistributionManager().getDistributionManagerId().getMemberData();
InternalDistributedMember mbr = null;
try {
mbr = new InternalDistributedMember(nm.getInetAddress().getCanonicalHostName(),
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java
index b96b8c9..e5f2238 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/cache30/ReconnectDUnitTest.java
@@ -91,7 +91,6 @@ import org.apache.geode.distributed.internal.InternalDistributedSystem.Reconnect
import org.apache.geode.distributed.internal.InternalLocator;
import org.apache.geode.distributed.internal.ServerLocator;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
import org.apache.geode.distributed.internal.membership.gms.MembershipManagerHelper;
import org.apache.geode.examples.SimpleSecurityManager;
@@ -382,7 +381,7 @@ public class ReconnectDUnitTest extends JUnit4CacheTestCase {
};
vm0.invoke(create1);
- final DistributedMember dm = (DistributedMember) vm1.invoke(create2);
+ final InternalDistributedMember dm = (InternalDistributedMember) vm1.invoke(create2);
IgnoredException.addIgnoredException("ForcedDisconnectException");
forceDisconnect(vm1);
@@ -422,9 +421,7 @@ public class ReconnectDUnitTest extends JUnit4CacheTestCase {
assertTrue(
((GMSMembershipManager) getMembership(system))
.getServices().getMessenger()
- .isOldMembershipIdentifier(
- ((GMSMemberAdapter) ((InternalDistributedMember) dm).getNetMember())
- .getGmsMember()));
+ .isOldMembershipIdentifier(dm));
return ds.getReconnectedSystem().getDistributedMember();
} catch (InterruptedException e) {
System.err.println("interrupted while waiting for reconnect");
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedMemberDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedMemberDUnitTest.java
index 806e79e4..e8c80a4 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedMemberDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedMemberDUnitTest.java
@@ -48,9 +48,7 @@ import org.apache.geode.distributed.internal.DistributionManager;
import org.apache.geode.distributed.internal.HighPriorityAckedMessage;
import org.apache.geode.distributed.internal.InternalDistributedSystem;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.MembershipManagerHelper;
import org.apache.geode.internal.HeapDataOutputStream;
import org.apache.geode.test.dunit.Host;
@@ -264,13 +262,9 @@ public class DistributedMemberDUnitTest extends JUnit4DistributedTestCase {
assertTrue(system == basicGetSystem()); // senders will use basicGetSystem()
InternalDistributedMember internalDistributedMember = system.getDistributedMember();
- GMSMember gmsMember =
- new GMSMember(((GMSMemberAdapter) internalDistributedMember.getNetMember()).getGmsMember());
- assertEquals(gmsMember,
- ((GMSMemberAdapter) internalDistributedMember.getNetMember()).getGmsMember());
- gmsMember.setName(null);
+ internalDistributedMember.getMemberData().setName(null);
HeapDataOutputStream outputStream = new HeapDataOutputStream(100);
- new InternalDistributedMember(new GMSMemberAdapter(gmsMember)).writeEssentialData(outputStream);
+ internalDistributedMember.writeEssentialData(outputStream);
DataInputStream dataInputStream =
new DataInputStream(new ByteArrayInputStream(outputStream.toByteArray()));
InternalDistributedMember partialID =
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedSystemDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedSystemDUnitTest.java
index f31298e..b28d8aa 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedSystemDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/distributed/DistributedSystemDUnitTest.java
@@ -28,7 +28,6 @@ import static org.apache.geode.distributed.ConfigurationProperties.MEMBERSHIP_PO
import static org.apache.geode.distributed.ConfigurationProperties.REDUNDANCY_ZONE;
import static org.apache.geode.distributed.ConfigurationProperties.START_LOCATOR;
import static org.apache.geode.distributed.ConfigurationProperties.TCP_PORT;
-import static org.apache.geode.distributed.internal.ClusterDistributionManager.NORMAL_DM_TYPE;
import static org.apache.geode.distributed.internal.DistributionConfig.DEFAULT_ACK_WAIT_THRESHOLD;
import static org.apache.geode.distributed.internal.OperationExecutors.SERIAL_EXECUTOR;
import static org.apache.geode.internal.AvailablePort.MULTICAST;
@@ -129,8 +128,7 @@ public class DistributedSystemDUnitTest extends JUnit4DistributedTestCase {
// construct a member ID that will represent a departed member
InternalDistributedMember member =
- new InternalDistributedMember("localhost", 12345, "", "", NORMAL_DM_TYPE, null,
- null);
+ new InternalDistributedMember("localhost", 12345);
// schedule a message in order to create a queue for the fake member
ClusterDistributionManager distributionManager =
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java
index 7314d27..bd19f1e 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/distributed/internal/tcpserver/TcpServerBackwardCompatDUnitTest.java
@@ -29,7 +29,7 @@ import org.junit.experimental.categories.Category;
import org.apache.geode.distributed.Locator;
import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorRequest;
import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorResponse;
import org.apache.geode.internal.AvailablePortHelper;
@@ -139,7 +139,7 @@ public class TcpServerBackwardCompatDUnitTest extends JUnit4DistributedTestCase
// Start a gossip client to connect to first locator "locator0".
FindCoordinatorRequest req = new FindCoordinatorRequest(
- new GMSMember("localhost", 1234));
+ new InternalDistributedMember("localhost", 1234));
FindCoordinatorResponse response;
response = (FindCoordinatorResponse) new TcpClient()
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/ClientCacheFactoryJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/ClientCacheFactoryJUnitTest.java
index da17d61..5f2dccb 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/cache/client/ClientCacheFactoryJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/cache/client/ClientCacheFactoryJUnitTest.java
@@ -56,8 +56,7 @@ import org.apache.geode.cache.server.CacheServer;
import org.apache.geode.distributed.DistributedSystem;
import org.apache.geode.distributed.internal.InternalDistributedSystem;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.HeapDataOutputStream;
import org.apache.geode.internal.cache.GemFireCacheImpl;
import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
@@ -346,7 +345,7 @@ public class ClientCacheFactoryJUnitTest {
clientCache = new ClientCacheFactory().create();
InternalDistributedMember memberID =
(InternalDistributedMember) clientCache.getDistributedSystem().getDistributedMember();
- GMSMember gmsID = ((GMSMemberAdapter) memberID.getNetMember()).getGmsMember();
+ MemberIdentifier gmsID = memberID;
memberID.setVersionObjectForTest(Version.GFE_82);
assertThat(memberID.getVersionObject()).isEqualTo(Version.GFE_82);
@@ -363,11 +362,10 @@ public class ClientCacheFactoryJUnitTest {
assertThat(newMemberID.getVersionObject()).isEqualTo(Version.GFE_82);
assertThat(newID.getClientVersion()).isEqualTo(Version.GFE_82);
- GMSMember newGmsID = ((GMSMemberAdapter) newMemberID.getNetMember()).getGmsMember();
- assertThat(newGmsID.getUuidLSBs()).isEqualTo(0);
- assertThat(newGmsID.getUuidMSBs()).isEqualTo(0);
+ assertThat(newMemberID.getMemberData().getUuidLSBs()).isEqualTo(0);
+ assertThat(newMemberID.getMemberData().getUuidMSBs()).isEqualTo(0);
- gmsID.setUUID(new UUID(1234L, 5678L));
+ gmsID.getMemberData().setUUID(new UUID(1234L, 5678L));
memberID.setVersionObjectForTest(Version.CURRENT);
clientID = ClientProxyMembershipID.getClientId(memberID);
out = new HeapDataOutputStream(Version.CURRENT);
@@ -380,9 +378,10 @@ public class ClientCacheFactoryJUnitTest {
assertThat(newMemberID.getVersionObject()).isEqualTo(Version.CURRENT);
assertThat(newID.getClientVersion()).isEqualTo(Version.CURRENT);
- newGmsID = ((GMSMemberAdapter) newMemberID.getNetMember()).getGmsMember();
- assertThat(newGmsID.getUuidLSBs()).isEqualTo(gmsID.getUuidLSBs());
- assertThat(newGmsID.getUuidMSBs()).isEqualTo(gmsID.getUuidMSBs());
+ assertThat(newMemberID.getMemberData().getUuidLSBs())
+ .isEqualTo(gmsID.getMemberData().getUuidLSBs());
+ assertThat(newMemberID.getMemberData().getUuidMSBs())
+ .isEqualTo(gmsID.getMemberData().getUuidMSBs());
}
@Test
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java
index e085ce1..03ae057 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/MembershipJUnitTest.java
@@ -31,7 +31,6 @@ import static org.mockito.Mockito.when;
import java.io.File;
import java.net.InetAddress;
-import java.util.Arrays;
import java.util.List;
import java.util.Properties;
@@ -40,6 +39,8 @@ import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
import org.apache.geode.GemFireConfigException;
import org.apache.geode.distributed.ConfigurationProperties;
@@ -54,9 +55,11 @@ import org.apache.geode.distributed.internal.SerialAckedMessage;
import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
import org.apache.geode.distributed.internal.membership.adapter.ServiceConfig;
import org.apache.geode.distributed.internal.membership.adapter.auth.GMSAuthenticator;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMemberData;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifierFactory;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipBuilder;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipListener;
@@ -185,13 +188,13 @@ public class MembershipJUnitTest {
}
GMSMembershipView view = jl1.getView();
- GMSMember notCreator;
+ MemberIdentifier notCreator;
if (view.getCreator().equals(jl1.getMemberID())) {
notCreator = view.getMembers().get(1);
} else {
notCreator = view.getMembers().get(0);
}
- List<String> result = Arrays.asList(notCreator.getGroups());
+ List<String> result = notCreator.getGroups();
System.out.println("sending SerialAckedMessage from m1 to m2");
SerialAckedMessage msg = new SerialAckedMessage();
@@ -251,6 +254,13 @@ public class MembershipJUnitTest {
final InternalDistributedSystem mockSystem = mock(InternalDistributedSystem.class);
final SecurityService securityService = SecurityServiceFactory.create();
DSFIDSerializer serializer = InternalDataSerializer.getDSFIDSerializer();
+ final MemberIdentifierFactory memberFactory = mock(MemberIdentifierFactory.class);
+ when(memberFactory.create(isA(GMSMemberData.class))).thenAnswer(new Answer<MemberIdentifier>() {
+ @Override
+ public MemberIdentifier answer(InvocationOnMock invocation) throws Throwable {
+ return new InternalDistributedMember((GMSMemberData) invocation.getArgument(0));
+ }
+ });
final MembershipManager m1 =
MembershipBuilder.newMembershipBuilder(null)
.setAuthenticator(new GMSAuthenticator(config.getSecurityProps(), securityService,
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManagerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManagerJUnitTest.java
index f7bc630..2356042 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManagerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManagerJUnitTest.java
@@ -70,12 +70,13 @@ import org.apache.geode.distributed.internal.direct.DirectChannel;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
import org.apache.geode.distributed.internal.membership.MembershipView;
import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager.StartupEvent;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMemberData;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
import org.apache.geode.distributed.internal.membership.gms.Services.Stopper;
import org.apache.geode.distributed.internal.membership.gms.SuspectMember;
import org.apache.geode.distributed.internal.membership.gms.api.Authenticator;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipListener;
import org.apache.geode.distributed.internal.membership.gms.api.MessageListener;
@@ -132,13 +133,12 @@ public class GMSMembershipManagerJUnitTest {
authenticator = mock(Authenticator.class);
myMemberId = new InternalDistributedMember("localhost", 8887);
- GMSMember m = ((GMSMemberAdapter) myMemberId.getNetMember()).getGmsMember();
+ GMSMemberData m = (GMSMemberData) myMemberId.getMemberData();
UUID uuid = new UUID(12345, 12345);
m.setUUID(uuid);
messenger = mock(Messenger.class);
- when(messenger.getMemberID())
- .thenReturn(((GMSMemberAdapter) myMemberId.getNetMember()).getGmsMember());
+ when(messenger.getMemberID()).thenReturn(myMemberId);
stopper = mock(Stopper.class);
when(stopper.isCancelInProgress()).thenReturn(false);
@@ -163,7 +163,7 @@ public class GMSMembershipManagerJUnitTest {
mockMembers = new InternalDistributedMember[5];
for (int i = 0; i < mockMembers.length; i++) {
mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
- m = ((GMSMemberAdapter) mockMembers[i].getNetMember()).getGmsMember();
+ m = (GMSMemberData) mockMembers[i].getMemberData();
uuid = new UUID(r.nextLong(), r.nextLong());
m.setUUID(uuid);
}
@@ -190,10 +190,9 @@ public class GMSMembershipManagerJUnitTest {
m.setRecipient(mockMembers[0]);
manager.getGMSManager().start();
manager.getGMSManager().started();
- GMSMember myGMSMemberId = ((GMSMemberAdapter) myMemberId.getNetMember()).getGmsMember();
- List<GMSMember> gmsMembers =
- members.stream().map(x -> ((GMSMemberAdapter) x.getNetMember()).getGmsMember()).collect(
- Collectors.toList());
+ MemberIdentifier myGMSMemberId = myMemberId;
+ List<MemberIdentifier> gmsMembers =
+ members.stream().map(x -> ((MemberIdentifier) x)).collect(Collectors.toList());
manager.getGMSManager().installView(new GMSMembershipView(myGMSMemberId, 1, gmsMembers));
Set<InternalDistributedMember> failures =
manager.send(m.getRecipients(), m);
@@ -207,12 +206,8 @@ public class GMSMembershipManagerJUnitTest {
private GMSMembershipView createView(InternalDistributedMember creator, int viewId,
List<InternalDistributedMember> members) {
- List<GMSMember> gmsMembers = new ArrayList<>(members.size());
- for (InternalDistributedMember member : members) {
- gmsMembers.add(((GMSMemberAdapter) member.getNetMember()).getGmsMember());
- }
- return new GMSMembershipView(((GMSMemberAdapter) creator.getNetMember()).getGmsMember(), viewId,
- gmsMembers);
+ List<MemberIdentifier> gmsMembers = new ArrayList<>(members);
+ return new GMSMembershipView(creator, viewId, gmsMembers);
}
@Test
@@ -271,8 +266,7 @@ public class GMSMembershipManagerJUnitTest {
// suspect a member
InternalDistributedMember suspectMember = mockMembers[1];
manager.handleOrDeferSuspect(
- new SuspectMember(((GMSMemberAdapter) mockMembers[0].getNetMember()).getGmsMember(),
- ((GMSMemberAdapter) suspectMember.getNetMember()).getGmsMember(), "testing"));
+ new SuspectMember(mockMembers[0], suspectMember, "testing"));
// suspect messages aren't queued - they're ignored before joining the system
assertEquals(2, manager.getStartupEvents().size());
verify(listener, never()).memberSuspect(suspectMember, mockMembers[0], "testing");
@@ -333,8 +327,7 @@ public class GMSMembershipManagerJUnitTest {
reset(listener);
suspectMember = mockMembers[1];
manager.handleOrDeferSuspect(
- new SuspectMember(((GMSMemberAdapter) mockMembers[0].getNetMember()).getGmsMember(),
- ((GMSMemberAdapter) suspectMember.getNetMember()).getGmsMember(), "testing"));
+ new SuspectMember(mockMembers[0], suspectMember, "testing"));
verify(listener).memberSuspect(suspectMember, mockMembers[0], "testing");
}
@@ -443,7 +436,7 @@ public class GMSMembershipManagerJUnitTest {
// each destination w/o a UUID should have been replaced with the corresponding
// ID from the membership view
for (int i = 0; i < destinations.length; i++) {
- assertTrue(((GMSMemberAdapter) destinations[i].getNetMember()).getGmsMember().hasUUID());
+ assertTrue(((GMSMemberData) destinations[i].getMemberData()).hasUUID());
}
}
@@ -483,7 +476,7 @@ public class GMSMembershipManagerJUnitTest {
rp.enableSevereAlertProcessing();
boolean result = rp.waitForReplies(WAIT_FOR_REPLIES_MILLIS);
assertFalse(result); // the wait should have timed out
- verify(healthMonitor, atLeastOnce()).checkIfAvailable(isA(GMSMember.class),
+ verify(healthMonitor, atLeastOnce()).checkIfAvailable(isA(MemberIdentifier.class),
isA(String.class), isA(Boolean.class));
}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
index 057f39e..630d7b2 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
@@ -75,11 +75,12 @@ import org.apache.geode.distributed.internal.DistributionConfigImpl;
import org.apache.geode.distributed.internal.DistributionManager;
import org.apache.geode.distributed.internal.DistributionStats;
import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
import org.apache.geode.distributed.internal.membership.adapter.ServiceConfig;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
import org.apache.geode.distributed.internal.membership.gms.Services.Stopper;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.fd.GMSHealthMonitor.ClientSocketHandler;
import org.apache.geode.distributed.internal.membership.gms.interfaces.JoinLeave;
@@ -102,7 +103,7 @@ public class GMSHealthMonitorJUnitTest {
private Services services;
private MembershipConfig mockConfig;
private DistributionConfig mockDistConfig;
- private List<GMSMember> mockMembers;
+ private List<MemberIdentifier> mockMembers;
private Messenger messenger;
private JoinLeave joinLeave;
private GMSHealthMonitor gmsHealthMonitor;
@@ -160,11 +161,11 @@ public class GMSHealthMonitorJUnitTest {
if (mockMembers == null) {
mockMembers = new ArrayList<>();
for (int i = 0; i < 7; i++) {
- GMSMember mbr = new GMSMember("localhost", 8888 + i);
+ MemberIdentifier mbr = new InternalDistributedMember("localhost", 8888 + i);
if (i == 0 || i == 1) {
- mbr.setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- mbr.setPreferredForCoordinator(true);
+ mbr.getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mbr.getMemberData().setPreferredForCoordinator(true);
}
mockMembers.add(mbr);
}
@@ -186,9 +187,9 @@ public class GMSHealthMonitorJUnitTest {
@Test
public void testHMServiceStarted() throws IOException {
- GMSMember mbr =
- new GMSMember("localhost", 12345);
- mbr.setVmViewId(1);
+ MemberIdentifier mbr =
+ new InternalDistributedMember("localhost", 12345);
+ mbr.getMemberData().setVmViewId(1);
when(messenger.getMemberID()).thenReturn(mbr);
gmsHealthMonitor.started();
@@ -224,11 +225,11 @@ public class GMSHealthMonitorJUnitTest {
System.out.println("testHMNextNeighborAfterTimeout starting");
installAView();
- GMSMember initialNeighbor = mockMembers.get(myAddressIndex + 1);
+ MemberIdentifier initialNeighbor = mockMembers.get(myAddressIndex + 1);
await("wait for new neighbor")
.until(() -> gmsHealthMonitor.getNextNeighbor() != initialNeighbor);
- GMSMember neighbor = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier neighbor = gmsHealthMonitor.getNextNeighbor();
// neighbor should change. In order to not be a flaky test we don't demand
// that it be myAddressIndex+2 but just require that the neighbor being
@@ -248,7 +249,7 @@ public class GMSHealthMonitorJUnitTest {
public void testHMNextNeighborBeforeTimeout() throws IOException {
long startTime = System.currentTimeMillis();
installAView();
- final GMSMember neighbor = gmsHealthMonitor.getNextNeighbor();
+ final MemberIdentifier neighbor = gmsHealthMonitor.getNextNeighbor();
System.out.println("next neighbor is " + neighbor + "\nmy address is "
+ mockMembers.get(myAddressIndex) + "\nview is " + joinLeave.getView());
assertEquals(mockMembers.get(myAddressIndex + 1), neighbor);
@@ -291,7 +292,7 @@ public class GMSHealthMonitorJUnitTest {
}
private void setFailureDetectionPorts(GMSMembershipView v) {
- java.util.Iterator<GMSMember> itr = mockMembers.iterator();
+ java.util.Iterator<MemberIdentifier> itr = mockMembers.iterator();
int port = 7899;
while (itr.hasNext()) {
@@ -306,7 +307,7 @@ public class GMSHealthMonitorJUnitTest {
public void testSuspectMembersNotCalledThroughPingThreadBeforeTimeout() {
long startTime = System.currentTimeMillis();
installAView();
- GMSMember neighbor = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier neighbor = gmsHealthMonitor.getNextNeighbor();
await().until(() -> gmsHealthMonitor.isSuspectMember(neighbor));
long endTime = System.currentTimeMillis();
@@ -358,7 +359,7 @@ public class GMSHealthMonitorJUnitTest {
gmsHealthMonitor.installView(v);
- ArrayList<GMSMember> recipient = new ArrayList<GMSMember>();
+ ArrayList<MemberIdentifier> recipient = new ArrayList<MemberIdentifier>();
recipient.add(mockMembers.get(0));
ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
SuspectRequest sr = new SuspectRequest(mockMembers.get(1), "Not Responding");// removing member
@@ -370,7 +371,7 @@ public class GMSHealthMonitorJUnitTest {
gmsHealthMonitor.processMessage(sm);
await("waiting for remove(member) to be invoked").untilAsserted(() -> {
- verify(joinLeave, atLeastOnce()).remove(any(GMSMember.class),
+ verify(joinLeave, atLeastOnce()).remove(any(MemberIdentifier.class),
any(String.class));
});
Assert.assertTrue(gmsHealthMonitor.getStats().getSuspectsReceived() > 0);
@@ -391,7 +392,7 @@ public class GMSHealthMonitorJUnitTest {
gmsHealthMonitor.installView(v);
- ArrayList<GMSMember> recipient = new ArrayList<GMSMember>();
+ ArrayList<MemberIdentifier> recipient = new ArrayList<MemberIdentifier>();
recipient.add(mockMembers.get(0));
ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
SuspectRequest sr = new SuspectRequest(mockMembers.get(1), "Not Responding");// removing member
@@ -405,7 +406,7 @@ public class GMSHealthMonitorJUnitTest {
await("waiting for remove(member) to be invoked")
.untilAsserted(
- () -> verify(joinLeave, atLeastOnce()).remove(any(GMSMember.class),
+ () -> verify(joinLeave, atLeastOnce()).remove(any(MemberIdentifier.class),
any(String.class)));
long postRemove = System.currentTimeMillis();
@@ -427,7 +428,7 @@ public class GMSHealthMonitorJUnitTest {
gmsHealthMonitor.installView(v);
- ArrayList<GMSMember> recipient = new ArrayList<GMSMember>();
+ ArrayList<MemberIdentifier> recipient = new ArrayList<MemberIdentifier>();
recipient.add(mockMembers.get(0));
recipient.add(mockMembers.get(1));
ArrayList<SuspectRequest> as = new ArrayList<SuspectRequest>();
@@ -440,7 +441,7 @@ public class GMSHealthMonitorJUnitTest {
gmsHealthMonitor.processMessage(sm);
await("waiting for remove(member) to be invoked").untilAsserted(
- () -> verify(joinLeave, atLeastOnce()).remove(any(GMSMember.class),
+ () -> verify(joinLeave, atLeastOnce()).remove(any(MemberIdentifier.class),
any(String.class)));
Assert.assertTrue(gmsHealthMonitor.getStats().getSuspectsReceived() > 0);
@@ -469,7 +470,7 @@ public class GMSHealthMonitorJUnitTest {
public void testCheckIfAvailableWithSimulatedHeartBeat() {
GMSMembershipView v = installAView();
- GMSMember memberToCheck = mockMembers.get(1);
+ MemberIdentifier memberToCheck = mockMembers.get(1);
HeartbeatMessage fakeHeartbeat = new HeartbeatMessage();
fakeHeartbeat.setSender(memberToCheck);
when(messenger.send(any(HeartbeatRequestMessage.class))).then(new Answer() {
@@ -494,7 +495,7 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = mockMembers.get(1);
+ MemberIdentifier memberToCheck = mockMembers.get(1);
boolean retVal = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
assertTrue("CheckIfAvailable should have return true", retVal);
@@ -514,7 +515,7 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = mockMembers.get(1);
+ MemberIdentifier memberToCheck = mockMembers.get(1);
boolean retVal = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
assertTrue("CheckIfAvailable should have return true", retVal);
@@ -538,7 +539,7 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier memberToCheck = gmsHealthMonitor.getNextNeighbor();
gmsHealthMonitor.setNextNeighbor(v, memberToCheck);
assertNotEquals(memberToCheck, gmsHealthMonitor.getNextNeighbor());
@@ -565,7 +566,7 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier memberToCheck = gmsHealthMonitor.getNextNeighbor();
boolean retVal = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
@@ -592,7 +593,7 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier memberToCheck = gmsHealthMonitor.getNextNeighbor();
gmsHealthMonitor.setNextNeighbor(v, memberToCheck);
assertNotEquals(memberToCheck, gmsHealthMonitor.getNextNeighbor());
@@ -618,12 +619,12 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier memberToCheck = gmsHealthMonitor.getNextNeighbor();
gmsHealthMonitor.setNextNeighbor(v, memberToCheck);
assertNotEquals(memberToCheck, gmsHealthMonitor.getNextNeighbor());
- mockMembers.get(0).setVersion(Version.GEODE_1_3_0);
+ ((InternalDistributedMember) mockMembers.get(0)).setVersionObjectForTest(Version.GEODE_1_3_0);
boolean retVal = gmsHealthMonitor.inlineCheckIfAvailable(mockMembers.get(0), v, true,
memberToCheck, "Not responding");
@@ -657,7 +658,7 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier memberToCheck = gmsHealthMonitor.getNextNeighbor();
gmsHealthMonitor.memberSuspected(mockMembers.get(0), memberToCheck, "Not responding");
assertTrue(gmsHealthMonitor.isSuspectMember(memberToCheck));
gmsHealthMonitor.processMessage(new FinalCheckPassedMessage(mockMembers.get(0), memberToCheck));
@@ -674,10 +675,10 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier memberToCheck = gmsHealthMonitor.getNextNeighbor();
boolean available = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
assertFalse(available);
- verify(joinLeave).remove(isA(GMSMember.class), isA(String.class));
+ verify(joinLeave).remove(isA(MemberIdentifier.class), isA(String.class));
assertTrue(gmsHealthMonitor.isSuspectMember(memberToCheck));
}
@@ -691,11 +692,11 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier memberToCheck = gmsHealthMonitor.getNextNeighbor();
gmsHealthMonitor.stopServer();
boolean available = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", false);
assertTrue(available);
- verify(joinLeave, never()).remove(isA(GMSMember.class), isA(String.class));
+ verify(joinLeave, never()).remove(isA(MemberIdentifier.class), isA(String.class));
assertTrue(((GMSHealthMonitorTest) gmsHealthMonitor).availabilityCheckedMembers
.contains(memberToCheck));
assertTrue(((GMSHealthMonitorTest) gmsHealthMonitor).availabilityCheckedMembers
@@ -714,10 +715,10 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier memberToCheck = gmsHealthMonitor.getNextNeighbor();
boolean available = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", false);
assertFalse(available);
- verify(joinLeave, never()).remove(isA(GMSMember.class), isA(String.class));
+ verify(joinLeave, never()).remove(isA(MemberIdentifier.class), isA(String.class));
assertTrue(gmsHealthMonitor.isSuspectMember(memberToCheck));
}
@@ -734,10 +735,10 @@ public class GMSHealthMonitorJUnitTest {
setFailureDetectionPorts(v);
- GMSMember memberToCheck = gmsHealthMonitor.getNextNeighbor();
+ MemberIdentifier memberToCheck = gmsHealthMonitor.getNextNeighbor();
boolean available = gmsHealthMonitor.checkIfAvailable(memberToCheck, "Not responding", true);
assertFalse(available);
- verify(joinLeave).remove(isA(GMSMember.class), isA(String.class));
+ verify(joinLeave).remove(isA(MemberIdentifier.class), isA(String.class));
}
@@ -776,8 +777,8 @@ public class GMSHealthMonitorJUnitTest {
int viewId = 2;
long msb = 3;
long lsb = 4;
- GMSMember otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
- GMSMember gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
+ MemberIdentifier otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
+ MemberIdentifier gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
executeTestClientSocketHandler(gmsMember, otherMember, GMSHealthMonitor.OK);
}
@@ -786,8 +787,8 @@ public class GMSHealthMonitorJUnitTest {
int viewId = 2;
long msb = 3;
long lsb = 4;
- GMSMember otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb + 1, lsb);
- GMSMember gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
+ MemberIdentifier otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb + 1, lsb);
+ MemberIdentifier gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
executeTestClientSocketHandler(gmsMember, otherMember, GMSHealthMonitor.ERROR);
}
@@ -796,8 +797,8 @@ public class GMSHealthMonitorJUnitTest {
int viewId = 2;
long msb = 3;
long lsb = 4;
- GMSMember otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb + 1);
- GMSMember gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
+ MemberIdentifier otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb + 1);
+ MemberIdentifier gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
executeTestClientSocketHandler(gmsMember, otherMember, GMSHealthMonitor.ERROR);
}
@@ -806,23 +807,25 @@ public class GMSHealthMonitorJUnitTest {
int viewId = 2;
long msb = 3;
long lsb = 4;
- GMSMember otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId + 1, msb, lsb);
- GMSMember gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
+ MemberIdentifier otherMember = createGMSMember(Version.CURRENT_ORDINAL, viewId + 1, msb, lsb);
+ MemberIdentifier gmsMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, msb, lsb);
executeTestClientSocketHandler(gmsMember, otherMember, GMSHealthMonitor.ERROR);
}
- public void executeTestClientSocketHandler(GMSMember gmsMember, GMSMember otherMember,
+ public void executeTestClientSocketHandler(MemberIdentifier gmsMember,
+ MemberIdentifier otherMember,
int expectedResult) throws Exception {
// We have already set the view id in the member but when creating the IDM it resets it to -1
// for some reason
int viewId = gmsMember.getVmViewId();
- GMSMember testMember = createGMSMember(Version.CURRENT_ORDINAL, viewId, gmsMember.getUuidMSBs(),
- gmsMember.getUuidLSBs());
- testMember.setUdpPort(9000);
+ MemberIdentifier testMember =
+ createGMSMember(Version.CURRENT_ORDINAL, viewId, gmsMember.getMemberData().getUuidMSBs(),
+ gmsMember.getMemberData().getUuidLSBs());
+ testMember.getMemberData().setUdpPort(9000);
// We set to our expected test viewId in the IDM as well as resetting the gms member
- gmsMember.setBirthViewId(viewId);
+ gmsMember.getMemberData().setVmViewId(viewId);
// Set up the incoming/received bytes. We just wrap output streams and write out the gms member
@@ -906,7 +909,7 @@ public class GMSHealthMonitorJUnitTest {
};
serverThread.setDaemon(true);
serverThread.start();
- GMSMember otherMember =
+ MemberIdentifier otherMember =
createGMSMember(Version.CURRENT_ORDINAL, 0, 1, 1);
long startTime = System.currentTimeMillis();
gmsHealthMonitor.doTCPCheckMember(otherMember, mySocket.getLocalPort(), true);
@@ -932,9 +935,9 @@ public class GMSHealthMonitorJUnitTest {
}
private void executeTestDoTCPCheck(int receivedStatus, boolean expectedResult) throws Exception {
- GMSMember otherMember =
+ MemberIdentifier otherMember =
createGMSMember(Version.CURRENT_ORDINAL, 0, 1, 1);
- GMSMember gmsMember =
+ MemberIdentifier gmsMember =
createGMSMember(Version.CURRENT_ORDINAL, 0, 1, 1);
// Set up the incoming/received bytes. We just wrap output streams and write out the gms member
@@ -963,17 +966,18 @@ public class GMSHealthMonitorJUnitTest {
bytesWritten);
}
- private GMSMember createGMSMember(short version, int viewId, long msb, long lsb)
+ private MemberIdentifier createGMSMember(short version, int viewId, long msb, long lsb)
throws UnknownHostException {
- GMSMember gmsMember = new GMSMember();
- gmsMember.setVersionOrdinal(version);
- gmsMember.setBirthViewId(viewId);
- gmsMember.setUUID(new UUID(msb, lsb));
- gmsMember.setInetAddr(InetAddress.getLocalHost());
+ MemberIdentifier gmsMember = new InternalDistributedMember();
+ ((InternalDistributedMember) gmsMember)
+ .setVersionObjectForTest(Version.fromOrdinalNoThrow(version, false));
+ gmsMember.getMemberData().setVmViewId(viewId);
+ gmsMember.getMemberData().setUUID(new UUID(msb, lsb));
+ gmsMember.getMemberData().setInetAddr(InetAddress.getLocalHost());
return gmsMember;
}
- private byte[] writeMemberToBytes(GMSMember gmsMember) throws IOException {
+ private byte[] writeMemberToBytes(MemberIdentifier gmsMember) throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream();
DataOutputStream dataReceive = new DataOutputStream(baos);
gmsHealthMonitor.writeMemberToStream(gmsMember, dataReceive);
@@ -982,10 +986,10 @@ public class GMSHealthMonitorJUnitTest {
public class GMSHealthMonitorTest extends GMSHealthMonitor {
public boolean useBlockingSocket = false;
- public Set<GMSMember> availabilityCheckedMembers = new HashSet<>();
+ public Set<MemberIdentifier> availabilityCheckedMembers = new HashSet<>();
@Override
- boolean doTCPCheckMember(GMSMember suspectMember, int port,
+ boolean doTCPCheckMember(MemberIdentifier suspectMember, int port,
boolean retryIfConnectFails) {
availabilityCheckedMembers.add(suspectMember);
if (useGMSHealthMonitorTestClass) {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorIntegrationTest.java
index f80f616..b93ab1b 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorIntegrationTest.java
@@ -26,7 +26,7 @@ import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.apache.geode.distributed.internal.LocatorStats;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
import org.apache.geode.distributed.internal.membership.gms.interfaces.JoinLeave;
@@ -63,7 +63,7 @@ public class GMSLocatorIntegrationTest {
when(services.getJoinLeave()).thenReturn(joinLeave);
messenger = mock(Messenger.class);
when(services.getMessenger()).thenReturn(messenger);
- when(messenger.getMemberID()).thenReturn(new GMSMember("localhost", 8080));
+ when(messenger.getMemberID()).thenReturn(new InternalDistributedMember("localhost", 8080));
gmsLocator =
new GMSLocator(null, null, false, false, new LocatorStats(), "",
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryIntegrationTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryIntegrationTest.java
index 3d19123..a77e33d 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryIntegrationTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocatorRecoveryIntegrationTest.java
@@ -17,7 +17,6 @@ package org.apache.geode.distributed.internal.membership.gms.locator;
import static org.apache.geode.distributed.ConfigurationProperties.BIND_ADDRESS;
import static org.apache.geode.distributed.ConfigurationProperties.DISABLE_TCP;
import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
-import static org.apache.geode.distributed.internal.membership.gms.GMSMember.NORMAL_DM_TYPE;
import static org.apache.geode.distributed.internal.membership.gms.locator.GMSLocator.LOCATOR_FILE_STAMP;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.catchThrowable;
@@ -50,11 +49,11 @@ import org.apache.geode.distributed.internal.InternalDistributedSystem;
import org.apache.geode.distributed.internal.InternalLocator;
import org.apache.geode.distributed.internal.LocatorStats;
import org.apache.geode.distributed.internal.membership.MembershipManager;
-import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
import org.apache.geode.distributed.internal.membership.adapter.ServiceConfig;
import org.apache.geode.distributed.internal.membership.adapter.auth.GMSAuthenticator;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipBuilder;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipListener;
import org.apache.geode.distributed.internal.membership.gms.api.MessageListener;
@@ -164,7 +163,8 @@ public class GMSLocatorRecoveryIntegrationTest {
nonDefault.setProperty(LOCATORS, localHost.getHostAddress() + '[' + port + ']');
DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
- RemoteTransportConfig transport = new RemoteTransportConfig(config, NORMAL_DM_TYPE);
+ RemoteTransportConfig transport =
+ new RemoteTransportConfig(config, MemberIdentifier.NORMAL_DM_TYPE);
MembershipListener mockListener = mock(MembershipListener.class);
MessageListener mockMessageListener = mock(MessageListener.class);
@@ -193,8 +193,7 @@ public class GMSLocatorRecoveryIntegrationTest {
gmsLocator.init(null);
assertThat(gmsLocator.getMembers())
- .contains(
- ((GMSMemberAdapter) membershipManager.getLocalMember().getNetMember()).getGmsMember());
+ .contains(membershipManager.getLocalMember());
}
@Test
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
index 70605ae..a3a5ddd 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
@@ -54,13 +54,14 @@ import org.mockito.verification.Timeout;
import org.apache.geode.SystemConnectException;
import org.apache.geode.distributed.internal.ClusterDistributionManager;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
import org.apache.geode.distributed.internal.membership.adapter.ServiceConfig;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
import org.apache.geode.distributed.internal.membership.gms.Services;
import org.apache.geode.distributed.internal.membership.gms.Services.Stopper;
import org.apache.geode.distributed.internal.membership.gms.api.Authenticator;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.interfaces.HealthMonitor;
import org.apache.geode.distributed.internal.membership.gms.interfaces.Locator;
@@ -89,17 +90,17 @@ public class GMSJoinLeaveJUnitTest {
private MembershipConfig mockConfig;
private Authenticator authenticator;
private HealthMonitor healthMonitor;
- private GMSMember gmsJoinLeaveMemberId;
- private GMSMember[] mockMembers;
- private GMSMember mockOldMember;
+ private MemberIdentifier gmsJoinLeaveMemberId;
+ private MemberIdentifier[] mockMembers;
+ private MemberIdentifier mockOldMember;
private Properties credentials = new Properties();
private Messenger messenger;
private GMSJoinLeave gmsJoinLeave;
private Manager manager;
private Stopper stopper;
private TestLocator testLocator;
- private GMSMember removeMember = null;
- private GMSMember leaveMember = null;
+ private MemberIdentifier removeMember = null;
+ private MemberIdentifier leaveMember = null;
public void initMocks() {
initMocks(false);
@@ -119,7 +120,7 @@ public class GMSJoinLeaveJUnitTest {
when(mockConfig.getMemberTimeout()).thenReturn(2000L);
authenticator = mock(Authenticator.class);
- gmsJoinLeaveMemberId = new GMSMember("localhost", 8887);
+ gmsJoinLeaveMemberId = new InternalDistributedMember("localhost", 8887);
messenger = mock(Messenger.class);
when(messenger.getMemberID()).thenReturn(gmsJoinLeaveMemberId);
@@ -146,11 +147,12 @@ public class GMSJoinLeaveJUnitTest {
Timer t = new Timer(true);
when(services.getTimer()).thenReturn(t);
- mockMembers = new GMSMember[4];
+ mockMembers = new InternalDistributedMember[4];
for (int i = 0; i < mockMembers.length; i++) {
- mockMembers[i] = new GMSMember("localhost", 8888 + i);
+ mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
}
- mockOldMember = new GMSMember("localhost", 8700, Version.GFE_56);
+ mockOldMember = new InternalDistributedMember("localhost", 8700);
+ ((InternalDistributedMember) mockOldMember).setVersionObjectForTest(Version.GFE_56);
if (useTestGMSJoinLeave) {
gmsJoinLeave = new GMSJoinLeaveTest();
@@ -217,7 +219,7 @@ public class GMSJoinLeaveJUnitTest {
initMocks();
int viewId = 1;
- List<GMSMember> mbrs = new LinkedList<>();
+ List<MemberIdentifier> mbrs = new LinkedList<>();
mbrs.add(mockMembers[0]);
mbrs.add(mockMembers[1]);
mbrs.add(mockMembers[2]);
@@ -228,20 +230,20 @@ public class GMSJoinLeaveJUnitTest {
state.view = netView;
state.viewId = netView.getViewId();
- GMSMember coordinator = mockMembers[2];
- coordinator.setVmViewId(viewId);
+ MemberIdentifier coordinator = mockMembers[2];
+ coordinator.getMemberData().setVmViewId(viewId);
// already tried joining using members 0 and 1
- Set<GMSMember> set = new HashSet<>();
- mockMembers[0].setVmViewId(viewId - 1);
+ Set<MemberIdentifier> set = new HashSet<>();
+ mockMembers[0].getMemberData().setVmViewId(viewId - 1);
set.add(mockMembers[0]);
- mockMembers[1].setVmViewId(viewId - 1);
+ mockMembers[1].getMemberData().setVmViewId(viewId - 1);
set.add(mockMembers[1]);
state.alreadyTried = set;
state.hasContactedAJoinedLocator = true;
// simulate a response being received
- GMSMember sender = mockMembers[2];
+ MemberIdentifier sender = mockMembers[2];
FindCoordinatorResponse resp = new FindCoordinatorResponse(coordinator, sender, null, 0);
gmsJoinLeave.processMessage(resp);
// tell GMSJoinLeave that a unit test is running so it won't clear the
@@ -269,7 +271,7 @@ public class GMSJoinLeaveJUnitTest {
public void testViewWithoutMemberInitiatesForcedDisconnect() throws Exception {
initMocks();
GMSJoinLeaveTestHelper.becomeCoordinatorForTest(gmsJoinLeave);
- List<GMSMember> members = Arrays.asList(mockMembers);
+ List<MemberIdentifier> members = Arrays.asList(mockMembers);
GMSMembershipView v = new GMSMembershipView(mockMembers[0], 2, members);
InstallViewMessage message = getInstallViewMessage(v, null, false);
gmsJoinLeave.processMessage(message);
@@ -327,13 +329,13 @@ public class GMSJoinLeaveJUnitTest {
* prepares and install a view
*
*/
- private void prepareAndInstallView(GMSMember coordinator,
- List<GMSMember> members) throws IOException {
+ private void prepareAndInstallView(MemberIdentifier coordinator,
+ List<MemberIdentifier> members) throws IOException {
int viewId = 1;
// prepare the view
GMSMembershipView netView = new GMSMembershipView(coordinator, viewId, members);
- for (GMSMember member : netView.getMembers()) {
+ for (MemberIdentifier member : netView.getMembers()) {
netView.setPublicKey(member, member.toString());
}
InstallViewMessage installViewMessage = getInstallViewMessage(netView, credentials, true);
@@ -346,10 +348,10 @@ public class GMSJoinLeaveJUnitTest {
Assert.assertEquals(netView, gmsJoinLeave.getView());
}
- private List<GMSMember> createMemberList(GMSMember... members) {
- List<GMSMember> memberList =
- new ArrayList<GMSMember>(members.length);
- for (GMSMember member : members) {
+ private List<MemberIdentifier> createMemberList(MemberIdentifier... members) {
+ List<MemberIdentifier> memberList =
+ new ArrayList<MemberIdentifier>(members.length);
+ for (MemberIdentifier member : members) {
memberList.add(member);
}
return memberList;
@@ -387,7 +389,7 @@ public class GMSJoinLeaveJUnitTest {
initMocks();
final int viewInstallationTime = 15000;
- when(healthMonitor.checkIfAvailable(isA(GMSMember.class), isA(String.class),
+ when(healthMonitor.checkIfAvailable(isA(MemberIdentifier.class), isA(String.class),
isA(Boolean.class))).thenReturn(true);
gmsJoinLeave.delayViewCreationForTest(5000); // ensures multiple requests are queued for a view
@@ -425,7 +427,7 @@ public class GMSJoinLeaveJUnitTest {
initMocks();
prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
- List<GMSMember> mbrs = new LinkedList<>();
+ List<MemberIdentifier> mbrs = new LinkedList<>();
mbrs.add(mockMembers[0]);
mbrs.add(mockMembers[1]);
@@ -444,7 +446,7 @@ public class GMSJoinLeaveJUnitTest {
prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
int viewId = 2;
- List<GMSMember> mbrs = new LinkedList<>();
+ List<MemberIdentifier> mbrs = new LinkedList<>();
mbrs.add(mockMembers[1]);
mbrs.add(mockMembers[2]);
mbrs.add(mockMembers[3]);
@@ -465,7 +467,7 @@ public class GMSJoinLeaveJUnitTest {
prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
// test that a non-member can't remove another member
RemoveMemberMessage msg = new RemoveMemberMessage(mockMembers[0], mockMembers[1], reason);
- msg.setSender(new GMSMember("localhost", 9000));
+ msg.setSender(new InternalDistributedMember("localhost", 9000));
gmsJoinLeave.processMessage(msg);
assertTrue("RemoveMemberMessage should not have been added to view requests",
gmsJoinLeave.getViewRequests().size() == 0);
@@ -527,7 +529,7 @@ public class GMSJoinLeaveJUnitTest {
@Test
public void testDuplicateJoinRequestDoesNotCauseNewView() throws Exception {
initMocks();
- when(healthMonitor.checkIfAvailable(isA(GMSMember.class), isA(String.class),
+ when(healthMonitor.checkIfAvailable(isA(MemberIdentifier.class), isA(String.class),
isA(Boolean.class))).thenReturn(true);
gmsJoinLeave.unitTesting.add("noRandomViewChange");
prepareAndInstallView(gmsJoinLeaveMemberId,
@@ -547,16 +549,16 @@ public class GMSJoinLeaveJUnitTest {
GMSMembershipView view = gmsJoinLeave.getView();
assertTrue("expected member to be added: " + mockMembers[2] + "; view: " + view,
view.contains(mockMembers[2]));
- List<GMSMember> members = view.getMembers();
+ List<MemberIdentifier> members = view.getMembers();
int occurrences = 0;
- for (GMSMember mbr : members) {
+ for (MemberIdentifier mbr : members) {
if (mbr.equals(mockMembers[2])) {
occurrences += 1;
}
}
assertTrue("expected member to only be in the view once: " + mockMembers[2] + "; view: " + view,
occurrences == 1);
- verify(healthMonitor, times(5)).checkIfAvailable(isA(GMSMember.class),
+ verify(healthMonitor, times(5)).checkIfAvailable(isA(MemberIdentifier.class),
isA(String.class), isA(Boolean.class));
}
@@ -585,7 +587,7 @@ public class GMSJoinLeaveJUnitTest {
createMemberList(gmsJoinLeaveMemberId, mockMembers[0], mockMembers[1]));
reset(messenger);
RemoveMemberMessage msg = new RemoveMemberMessage(gmsJoinLeaveMemberId,
- new GMSMember("localhost", 10000), "removing for test");
+ new InternalDistributedMember("localhost", 10000), "removing for test");
msg.setSender(mockMembers[0]);
gmsJoinLeave.processMessage(msg);
verify(messenger).send(isA(RemoveMemberMessage.class));
@@ -595,12 +597,12 @@ public class GMSJoinLeaveJUnitTest {
public void testRemoveRequestCausesForcedDisconnectInRogue() throws Exception {
initMocks();
// gmsJoinLeave mistakenly uses an old viewID when joining, making it a rogue member
- gmsJoinLeaveMemberId.setVmViewId(-1);
- GMSMember previousMemberId =
- new GMSMember(gmsJoinLeaveMemberId.getInetAddress().getHostName(),
+ gmsJoinLeaveMemberId.getMemberData().setVmViewId(-1);
+ MemberIdentifier previousMemberId =
+ new InternalDistributedMember(gmsJoinLeaveMemberId.getInetAddress().getHostName(),
gmsJoinLeaveMemberId.getPort());
- previousMemberId.setVmViewId(0);
- previousMemberId.setUUID(gmsJoinLeaveMemberId.getUUID());
+ previousMemberId.getMemberData().setVmViewId(0);
+ previousMemberId.getMemberData().setUUID(gmsJoinLeaveMemberId.getMemberData().getUUID());
GMSMembershipView view = new GMSMembershipView(mockMembers[0], 1,
createMemberList(mockMembers[0], previousMemberId, mockMembers[1]));
InstallViewMessage viewMessage = new InstallViewMessage(view, 0, false);
@@ -647,7 +649,7 @@ public class GMSJoinLeaveJUnitTest {
String reason = "testing";
initMocks();
prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
- mockMembers[1].setVmViewId(gmsJoinLeave.getView().getViewId() - 1);
+ mockMembers[1].getMemberData().setVmViewId(gmsJoinLeave.getView().getViewId() - 1);
LeaveRequestMessage msg =
new LeaveRequestMessage(gmsJoinLeave.getMemberID(), mockMembers[1], reason);
msg.setSender(mockMembers[1]);
@@ -670,7 +672,7 @@ public class GMSJoinLeaveJUnitTest {
prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
GMSMembershipView view = gmsJoinLeave.getView();
view.add(gmsJoinLeaveMemberId);
- GMSMember creator = view.getCreator();
+ MemberIdentifier creator = view.getCreator();
LeaveRequestMessage msg = new LeaveRequestMessage(creator, creator, reason);
msg.setSender(creator);
gmsJoinLeave.processMessage(msg);
@@ -684,7 +686,7 @@ public class GMSJoinLeaveJUnitTest {
prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
GMSMembershipView view = gmsJoinLeave.getView();
view.add(gmsJoinLeaveMemberId);
- GMSMember creator = view.getCreator();
+ MemberIdentifier creator = view.getCreator();
RemoveMemberMessage msg = new RemoveMemberMessage(creator, creator, reason);
msg.setSender(creator);
gmsJoinLeave.processMessage(msg);
@@ -698,7 +700,7 @@ public class GMSJoinLeaveJUnitTest {
@Test
public void testBecomeCoordinatorThroughShutdownWhenOlderMemberCrashed() throws Exception {
initMocks();
- GMSMember A = mockMembers[0],
+ MemberIdentifier A = mockMembers[0],
B = gmsJoinLeaveMemberId,
C = mockMembers[1],
D = mockMembers[2],
@@ -722,7 +724,7 @@ public class GMSJoinLeaveJUnitTest {
@Test
public void testBecomeCoordinatorAndAcceptMemberWithViewID() throws Exception {
initMocks();
- GMSMember A = mockMembers[0],
+ MemberIdentifier A = mockMembers[0],
B = gmsJoinLeaveMemberId,
C = mockMembers[1],
D = mockMembers[2],
@@ -735,7 +737,7 @@ public class GMSJoinLeaveJUnitTest {
when(messenger.send(isA(InstallViewMessage.class), isA(GMSMembershipView.class)))
.thenAnswer((request) -> {
InstallViewMessage installViewMessage = request.getArgument(0);
- for (GMSMember recipient : installViewMessage.getRecipients()) {
+ for (MemberIdentifier recipient : installViewMessage.getRecipients()) {
ViewAckMessage viewAckMessage =
new ViewAckMessage(gmsJoinLeaveMemberId, installViewMessage.getView().getViewId(),
installViewMessage.isPreparing());
@@ -745,7 +747,7 @@ public class GMSJoinLeaveJUnitTest {
return null;
});
- E.setVmViewId(2);
+ E.getMemberData().setVmViewId(2);
gmsJoinLeave.recordViewRequest(new LeaveRequestMessage(B, C, "removing for test"));
@@ -769,7 +771,7 @@ public class GMSJoinLeaveJUnitTest {
GMSMembershipView oldView = gmsJoinLeave.getView();
oldView.add(gmsJoinLeaveMemberId);
GMSMembershipView view = new GMSMembershipView(oldView, oldView.getViewId() + 1);
- GMSMember creator = view.getCreator();
+ MemberIdentifier creator = view.getCreator();
view.remove(creator);
InstallViewMessage msg = getInstallViewMessage(view, creator, false);
msg.setSender(creator);
@@ -785,7 +787,7 @@ public class GMSJoinLeaveJUnitTest {
GMSMembershipView oldView = gmsJoinLeave.getView();
oldView.add(gmsJoinLeaveMemberId);
GMSMembershipView view = new GMSMembershipView(oldView, oldView.getViewId() + 1);
- GMSMember creator = view.getCreator();
+ MemberIdentifier creator = view.getCreator();
LeaveRequestMessage leaveRequestMessage =
new LeaveRequestMessage(gmsJoinLeaveMemberId, mockMembers[0], "leaving for test");
gmsJoinLeave.processMessage(leaveRequestMessage);
@@ -802,7 +804,7 @@ public class GMSJoinLeaveJUnitTest {
prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
GMSMembershipView oldView = gmsJoinLeave.getView();
oldView.add(gmsJoinLeaveMemberId);
- GMSMember creator = oldView.getCreator();
+ MemberIdentifier creator = oldView.getCreator();
GMSJoinLeaveTestHelper.becomeCoordinatorForTest(gmsJoinLeave);
GMSMembershipView view = new GMSMembershipView(2, gmsJoinLeave.getView().getViewId() + 1);
view.setCreator(creator);
@@ -829,20 +831,20 @@ public class GMSJoinLeaveJUnitTest {
// set up a view with sufficient members, then create a new view
// where enough weight is lost to cause a network partition
- List<GMSMember> mbrs = new LinkedList<>();
+ List<MemberIdentifier> mbrs = new LinkedList<>();
mbrs.add(mockMembers[0]);
mbrs.add(mockMembers[1]);
mbrs.add(mockMembers[2]);
mbrs.add(gmsJoinLeaveMemberId);
- mockMembers[1].setMemberWeight((byte) 20);
+ mockMembers[1].getMemberData().setMemberWeight((byte) 20);
GMSMembershipView newView =
new GMSMembershipView(mockMembers[0], gmsJoinLeave.getView().getViewId() + 1, mbrs);
InstallViewMessage installViewMessage = getInstallViewMessage(newView, credentials, false);
gmsJoinLeave.processMessage(installViewMessage);
- Set<GMSMember> crashes = new HashSet<>();
+ Set<MemberIdentifier> crashes = new HashSet<>();
crashes.add(mockMembers[1]);
crashes.add(mockMembers[2]);
mbrs = new LinkedList<>(mbrs);
@@ -877,15 +879,15 @@ public class GMSJoinLeaveJUnitTest {
// set up a view with sufficient members, then create a new view
// where enough weight is lost to cause a network partition
- List<GMSMember> mbrs = new LinkedList<>();
- Set<GMSMember> shutdowns = new HashSet<>();
- Set<GMSMember> crashes = new HashSet<>();
+ List<MemberIdentifier> mbrs = new LinkedList<>();
+ Set<MemberIdentifier> shutdowns = new HashSet<>();
+ Set<MemberIdentifier> crashes = new HashSet<>();
mbrs.add(mockMembers[0]);
mbrs.add(mockMembers[1]);
mbrs.add(mockMembers[2]);
mbrs.add(gmsJoinLeaveMemberId);
- mockMembers[1].setMemberWeight((byte) 20);
+ mockMembers[1].getMemberData().setMemberWeight((byte) 20);
GMSMembershipView newView =
new GMSMembershipView(mockMembers[0], gmsJoinLeave.getView().getViewId() + 1, mbrs,
@@ -928,7 +930,7 @@ public class GMSJoinLeaveJUnitTest {
@Test
public void testNoViewAckCausesRemovalMessage() throws Exception {
initMocks(true);
- when(healthMonitor.checkIfAvailable(isA(GMSMember.class), isA(String.class),
+ when(healthMonitor.checkIfAvailable(isA(MemberIdentifier.class), isA(String.class),
isA(Boolean.class))).thenReturn(false);
prepareAndInstallView(mockMembers[0], createMemberList(mockMembers[0], gmsJoinLeaveMemberId));
GMSMembershipView oldView = gmsJoinLeave.getView();
@@ -950,7 +952,7 @@ public class GMSJoinLeaveJUnitTest {
// wait for suspect processing
verify(healthMonitor, timeout(10000).atLeast(1)).checkIfAvailable(
- isA(GMSMember.class),
+ isA(MemberIdentifier.class),
isA(String.class), isA(Boolean.class));
// verify(messenger, atLeast(1)).send(isA(RemoveMemberMessage.class));
}
@@ -1015,19 +1017,19 @@ public class GMSJoinLeaveJUnitTest {
public void testCoordinatorGetsConflictingViewFromLocator() throws Exception {
// create the GMSJoinLeave instance we'll be testing
initMocks(false);
- GMSMember otherMember = mockMembers[0];
- gmsJoinLeaveMemberId.setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
- List<GMSMember> members = createMemberList(gmsJoinLeaveMemberId, otherMember);
+ MemberIdentifier otherMember = mockMembers[0];
+ gmsJoinLeaveMemberId.getMemberData().setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
+ List<MemberIdentifier> members = createMemberList(gmsJoinLeaveMemberId, otherMember);
prepareAndInstallView(gmsJoinLeaveMemberId, members);
GMSMembershipView installedView = gmsJoinLeave.getView();
gmsJoinLeave.unitTesting.add("noRandomViewChange"); // keep view numbers predictable
// create a view coming from the locator that conflicts with the installed view
- GMSMember locatorMemberId = new GMSMember("localhost",
+ MemberIdentifier locatorMemberId = new InternalDistributedMember("localhost",
mockMembers[mockMembers.length - 1].getPort() + 1);
- locatorMemberId.setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- List<GMSMember> newMemberList = new ArrayList<>(members);
+ locatorMemberId.getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ List<MemberIdentifier> newMemberList = new ArrayList<>(members);
newMemberList.add(locatorMemberId);
GMSMembershipView locatorView =
new GMSMembershipView(locatorMemberId, installedView.getViewId() + 10, newMemberList);
@@ -1096,8 +1098,8 @@ public class GMSJoinLeaveJUnitTest {
@Test
public void testViewBroadcaster() throws Exception {
initMocks();
- List<GMSMember> members = new ArrayList<>(Arrays.asList(mockMembers));
- gmsJoinLeaveMemberId.setVmViewId(1);
+ List<MemberIdentifier> members = new ArrayList<>(Arrays.asList(mockMembers));
+ gmsJoinLeaveMemberId.getMemberData().setVmViewId(1);
members.add(gmsJoinLeaveMemberId);
prepareAndInstallView(gmsJoinLeaveMemberId, members);
GMSJoinLeaveTestHelper.becomeCoordinatorForTest(gmsJoinLeave);
@@ -1106,8 +1108,8 @@ public class GMSJoinLeaveJUnitTest {
verify(messenger).sendUnreliably(isA(InstallViewMessage.class));
}
- private void installView(int viewId, GMSMember coordinator,
- List<GMSMember> members) throws IOException {
+ private void installView(int viewId, MemberIdentifier coordinator,
+ List<MemberIdentifier> members) throws IOException {
// prepare the view
GMSMembershipView netView = new GMSMembershipView(coordinator, viewId, members);
InstallViewMessage installViewMessage = getInstallViewMessage(netView, credentials, false);
@@ -1156,7 +1158,7 @@ public class GMSJoinLeaveJUnitTest {
initMocks(false);
System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
gmsJoinLeave.join();
- Set<GMSMember> recips = new HashSet<>();
+ Set<MemberIdentifier> recips = new HashSet<>();
recips.add(mockMembers[0]);
recips.add(mockMembers[1]);
recips.add(mockMembers[2]);
@@ -1187,7 +1189,7 @@ public class GMSJoinLeaveJUnitTest {
initMocks(false);
System.setProperty(GMSJoinLeave.BYPASS_DISCOVERY_PROPERTY, "true");
gmsJoinLeave.join();
- Set<GMSMember> recips = new HashSet<>();
+ Set<MemberIdentifier> recips = new HashSet<>();
recips.add(mockMembers[0]);
recips.add(mockMembers[1]);
recips.add(mockMembers[2]);
@@ -1196,9 +1198,9 @@ public class GMSJoinLeaveJUnitTest {
prepareProcessor.initialize(1, recips);
assertTrue("Prepare processor should be waiting ",
gmsJoinLeave.testPrepareProcessorWaiting());
- Set<GMSMember> pendingLeaves = new HashSet<>();
+ Set<MemberIdentifier> pendingLeaves = new HashSet<>();
pendingLeaves.add(mockMembers[0]);
- Set<GMSMember> pendingRemovals = new HashSet<>();
+ Set<MemberIdentifier> pendingRemovals = new HashSet<>();
pendingRemovals.add(mockMembers[1]);
prepareProcessor.processPendingRequests(pendingLeaves, pendingRemovals);
@@ -1221,7 +1223,7 @@ public class GMSJoinLeaveJUnitTest {
// gmsJoinLeaveMemberId.getNetMember().setPreferredForCoordinator(false);
// JoinRequestMessage reqMsg = new JoinRequestMessage(gmsJoinLeaveMemberId, mockMembers[0], null,
// 56734);
- // GMSMember ids = new GMSMember("localhost", 97898);
+ // GMSMember ids = new InternalDistributedMember("localhost", 97898);
// ids.getNetMember().setPreferredForCoordinator(true);
// gmsJoinLeave.processMessage(reqMsg);
// ArgumentCaptor<JoinResponseMessage> ac = ArgumentCaptor.forClass(JoinResponseMessage.class);
@@ -1313,7 +1315,7 @@ public class GMSJoinLeaveJUnitTest {
// a new member is joining
GMSMembershipView preparedView =
new GMSMembershipView(gmsJoinLeave.getView(), gmsJoinLeave.getView().getViewId() + 5);
- mockMembers[1].setVmViewId(preparedView.getViewId());
+ mockMembers[1].getMemberData().setVmViewId(preparedView.getViewId());
preparedView.add(mockMembers[1]);
InstallViewMessage msg = getInstallViewMessage(preparedView, null, true);
@@ -1349,17 +1351,17 @@ public class GMSJoinLeaveJUnitTest {
@Test
public void testPublicKeyForNewMemberFromPreparedViewIsInstalledInNewView() throws Exception {
initMocks(false);
- GMSMember newMember = mockMembers[1];
+ MemberIdentifier newMember = mockMembers[1];
prepareAndInstallView(gmsJoinLeaveMemberId,
createMemberList(gmsJoinLeaveMemberId, mockMembers[0]));
// a new member is joining
GMSMembershipView preparedView =
new GMSMembershipView(gmsJoinLeave.getView(), gmsJoinLeave.getView().getViewId() + 5);
- for (GMSMember member : preparedView.getMembers()) {
+ for (MemberIdentifier member : preparedView.getMembers()) {
preparedView.setPublicKey(member, member.toString());
}
- newMember.setVmViewId(preparedView.getViewId());
+ newMember.getMemberData().setVmViewId(preparedView.getViewId());
preparedView.add(newMember);
preparedView.setPublicKey(newMember, newMember.toString());
@@ -1394,9 +1396,9 @@ public class GMSJoinLeaveJUnitTest {
}
private GMSMembershipView createView() {
- List<GMSMember> mbrs = new LinkedList<>();
- Set<GMSMember> shutdowns = new HashSet<>();
- Set<GMSMember> crashes = new HashSet<>();
+ List<MemberIdentifier> mbrs = new LinkedList<>();
+ Set<MemberIdentifier> shutdowns = new HashSet<>();
+ Set<MemberIdentifier> crashes = new HashSet<>();
mbrs.add(mockMembers[0]);
mbrs.add(mockMembers[1]);
mbrs.add(mockMembers[2]);
@@ -1410,7 +1412,7 @@ public class GMSJoinLeaveJUnitTest {
@Test
public void testCoordinatorFindRequestSuccess() throws Exception {
initMocks(false);
- HashSet<GMSMember> registrants = new HashSet<>();
+ HashSet<MemberIdentifier> registrants = new HashSet<>();
registrants.add(mockMembers[0]);
FindCoordinatorResponse fcr = new FindCoordinatorResponse(mockMembers[0], mockMembers[0], false,
null, registrants, false, true, null);
@@ -1431,7 +1433,7 @@ public class GMSJoinLeaveJUnitTest {
public void testCoordinatorFindRequestFailure() throws Exception {
try {
initMocks(false);
- HashSet<GMSMember> registrants = new HashSet<>();
+ HashSet<MemberIdentifier> registrants = new HashSet<>();
registrants.add(mockMembers[0]);
FindCoordinatorResponse fcr = new FindCoordinatorResponse(mockMembers[0], mockMembers[0],
false, null, registrants, false, true, null);
@@ -1474,7 +1476,7 @@ public class GMSJoinLeaveJUnitTest {
}
@Override
- boolean checkIfAvailable(GMSMember fmbr) {
+ boolean checkIfAvailable(MemberIdentifier fmbr) {
if (removeMember != null) {
try {
if (removeMember.equals(fmbr)) {
@@ -1562,13 +1564,13 @@ public class GMSJoinLeaveJUnitTest {
gmsJoinLeave.installView(newView);
}
- private void processJoinMessage(GMSMember coordinator,
- GMSMember newMember, int port) {
+ private void processJoinMessage(MemberIdentifier coordinator,
+ MemberIdentifier newMember, int port) {
JoinRequestMessage reqMsg = new JoinRequestMessage(coordinator, newMember, null, port, 0);
gmsJoinLeave.processMessage(reqMsg);
}
- private void processRemoveMessage(GMSMember rMember) {
+ private void processRemoveMessage(MemberIdentifier rMember) {
RemoveMemberMessage msg =
new RemoveMemberMessage(gmsJoinLeave.getMemberID(), rMember, "testing");
msg.setSender(gmsJoinLeave.getMemberID());
@@ -1576,7 +1578,7 @@ public class GMSJoinLeaveJUnitTest {
gmsJoinLeave.processMessage(msg);
}
- private void processLeaveMessage(GMSMember rMember) {
+ private void processLeaveMessage(MemberIdentifier rMember) {
LeaveRequestMessage msg =
new LeaveRequestMessage(gmsJoinLeave.getMemberID(), rMember, "testing");
msg.setSender(rMember);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java
index 9501f72..fd36a0b 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncryptJUnitTest.java
@@ -31,10 +31,11 @@ import org.junit.experimental.categories.Category;
import org.apache.geode.distributed.ConfigurationProperties;
import org.apache.geode.distributed.internal.ClusterDistributionManager;
import org.apache.geode.distributed.internal.DistributionConfigImpl;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
import org.apache.geode.distributed.internal.membership.adapter.ServiceConfig;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
import org.apache.geode.test.junit.categories.MembershipTest;
@@ -48,7 +49,7 @@ public class GMSEncryptJUnitTest {
Services services;
- GMSMember mockMembers[];
+ MemberIdentifier mockMembers[];
GMSMembershipView netView;
@@ -71,12 +72,12 @@ public class GMSEncryptJUnitTest {
services = mock(Services.class);
when(services.getConfig()).thenReturn(membershipConfig);
- mockMembers = new GMSMember[4];
+ mockMembers = new MemberIdentifier[4];
for (int i = 0; i < mockMembers.length; i++) {
- mockMembers[i] = new GMSMember("localhost", 8888 + i);
+ mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
}
int viewId = 1;
- List<GMSMember> mbrs = new LinkedList<>();
+ List<MemberIdentifier> mbrs = new LinkedList<>();
mbrs.add(mockMembers[0]);
mbrs.add(mockMembers[1]);
mbrs.add(mockMembers[2]);
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java
index 1820055..c3e9caf 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumCheckerJUnitTest.java
@@ -40,22 +40,23 @@ import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.test.junit.categories.MembershipTest;
@Category({MembershipTest.class})
public class GMSQuorumCheckerJUnitTest {
- private GMSMember[] mockMembers;
+ private MemberIdentifier[] mockMembers;
private JChannel channel;
private JGAddress address;
@Before
public void initMocks() {
- mockMembers = new GMSMember[12];
+ mockMembers = new MemberIdentifier[12];
for (int i = 0; i < mockMembers.length; i++) {
- mockMembers[i] = new GMSMember("localhost", 8888 + i);
+ mockMembers[i] = new InternalDistributedMember("localhost", 8888 + i);
}
channel = mock(JChannel.class);
address = mock(JGAddress.class);
@@ -135,8 +136,8 @@ public class GMSQuorumCheckerJUnitTest {
@Test
public void testQuorumChecker10Servers2Locators4ServersLost() throws Exception {
GMSMembershipView view = prepareView();
- mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[0].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[1].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
Set<Integer> pongResponders = new HashSet<>();
for (int i = 0; i < mockMembers.length; i++) {
@@ -162,8 +163,8 @@ public class GMSQuorumCheckerJUnitTest {
@Test
public void testQuorumChecker10Servers2Locators4ServersAnd1LocatorLost() throws Exception {
GMSMembershipView view = prepareView();
- mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[0].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[1].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
Set<Integer> pongResponders = new HashSet<>();
for (int i = 0; i < mockMembers.length; i++) {
@@ -193,8 +194,8 @@ public class GMSQuorumCheckerJUnitTest {
public void testQuorumChecker10Servers2Locators5ServersAnd2LocatorsButNotLeadMemberLost()
throws Exception {
GMSMembershipView view = prepareView();
- mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[0].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[1].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
Set<Integer> pongResponders = new HashSet<>();
for (int i = 0; i < mockMembers.length; i++) {
@@ -226,8 +227,8 @@ public class GMSQuorumCheckerJUnitTest {
public void testQuorumChecker10Servers2Locators5ServerAnd1LocatorWithLeadMemberLost()
throws Exception {
GMSMembershipView view = prepareView();
- mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[0].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[1].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
Set<Integer> pongResponders = new HashSet<>();
for (int i = 0; i < mockMembers.length; i++) {
@@ -257,8 +258,8 @@ public class GMSQuorumCheckerJUnitTest {
public void testQuorumChecker2Servers2LocatorsLeadMemberLost() throws Exception {
int numMembers = 4;
GMSMembershipView view = prepareView(numMembers);
- mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[0].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[1].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
Set<Integer> pongResponders = new HashSet<>();
for (int i = 0; i < numMembers; i++) {
@@ -281,8 +282,8 @@ public class GMSQuorumCheckerJUnitTest {
public void testQuorumChecker2Servers2LocatorsLeadMemberAnd1LocatorLost() throws Exception {
int numMembers = 4;
GMSMembershipView view = prepareView(numMembers);
- mockMembers[0].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- mockMembers[1].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[0].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ mockMembers[1].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
Set<Integer> pongResponders = new HashSet<>();
for (int i = 0; i < numMembers; i++) {
@@ -308,7 +309,7 @@ public class GMSQuorumCheckerJUnitTest {
private GMSMembershipView prepareView(int numMembers) {
int viewId = 1;
- List<GMSMember> mbrs = new LinkedList<>();
+ List<MemberIdentifier> mbrs = new LinkedList<>();
for (int i = 0; i < numMembers; i++) {
mbrs.add(mockMembers[i]);
}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
index 508da37..8dc3edc 100755
--- a/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessengerJUnitTest.java
@@ -74,11 +74,12 @@ import org.apache.geode.distributed.ConfigurationProperties;
import org.apache.geode.distributed.DistributedSystemDisconnectedException;
import org.apache.geode.distributed.internal.DistributionConfigImpl;
import org.apache.geode.distributed.internal.DistributionStats;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
import org.apache.geode.distributed.internal.membership.adapter.ServiceConfig;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
import org.apache.geode.distributed.internal.membership.gms.Services.Stopper;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.interfaces.GMSMessage;
import org.apache.geode.distributed.internal.membership.gms.interfaces.HealthMonitor;
@@ -144,7 +145,7 @@ public class JGroupsMessengerJUnitTest {
nonDefault.put(ACK_WAIT_THRESHOLD, "1");
nonDefault.putAll(addProp);
DistributionConfigImpl config = new DistributionConfigImpl(nonDefault);
- tconfig = new RemoteTransportConfig(config, GMSMember.NORMAL_DM_TYPE);
+ tconfig = new RemoteTransportConfig(config, MemberIdentifier.NORMAL_DM_TYPE);
stopper = mock(Stopper.class);
when(stopper.isCancelInProgress()).thenReturn(false);
@@ -214,7 +215,7 @@ public class JGroupsMessengerJUnitTest {
messenger.installView(v);
messenger.handleJGroupsIOException(new IOException("je m'en fiche"),
new JGAddress(v.getMembers().get(1)));
- verify(healthMonitor).checkIfAvailable(isA(GMSMember.class), isA(String.class),
+ verify(healthMonitor).checkIfAvailable(isA(MemberIdentifier.class), isA(String.class),
isA(Boolean.class));
}
@@ -228,13 +229,13 @@ public class JGroupsMessengerJUnitTest {
messenger.installView(v);
messenger.handleJGroupsIOException(new IOException("fichez-moi le camp"),
new JGAddress(v.getMembers().get(1)));
- verify(healthMonitor, never()).checkIfAvailable(isA(GMSMember.class),
+ verify(healthMonitor, never()).checkIfAvailable(isA(MemberIdentifier.class),
isA(String.class), isA(Boolean.class));
}
private GMSMembershipView createView() {
- GMSMember sender = messenger.getMemberID();
- List<GMSMember> mbrs = new ArrayList<>();
+ MemberIdentifier sender = messenger.getMemberID();
+ List<MemberIdentifier> mbrs = new ArrayList<>();
mbrs.add(sender);
mbrs.add(createAddress(100));
mbrs.add(createAddress(101));
@@ -269,11 +270,11 @@ public class JGroupsMessengerJUnitTest {
initMocks(false);
BufferDataOutputStream out =
new BufferDataOutputStream(500, Version.getCurrentVersion());
- GMSMember mbr = createAddress(8888);
- mbr.setMemberWeight((byte) 40);
+ MemberIdentifier mbr = createAddress(8888);
+ mbr.getMemberData().setMemberWeight((byte) 40);
mbr.toData(out, mock(SerializationContext.class));
DataInputStream in = new DataInputStream(new ByteArrayInputStream(out.toByteArray()));
- mbr = new GMSMember();
+ mbr = new InternalDistributedMember();
mbr.fromData(in, mock(DeserializationContext.class));
assertEquals(40, mbr.getMemberWeight());
}
@@ -283,7 +284,7 @@ public class JGroupsMessengerJUnitTest {
for (int i = 0; i < 2; i++) {
boolean enableMcast = (i == 1);
initMocks(enableMcast);
- GMSMember mbr = createAddress(8888);
+ MemberIdentifier mbr = createAddress(8888);
HeartbeatMessage msg =
mock(HeartbeatMessage.class);
when(msg.getRecipients()).thenReturn(Collections.singletonList(mbr));
@@ -326,7 +327,7 @@ public class JGroupsMessengerJUnitTest {
JChannel realChannel = messenger.myChannel;
messenger.myChannel = mockChannel;
try {
- GMSMember mbr = createAddress(8888);
+ MemberIdentifier mbr = createAddress(8888);
HeartbeatMessage msg = mock(HeartbeatMessage.class);
when(msg.getRecipients()).thenReturn(Collections.singletonList(mbr));
when(msg.getMulticast()).thenReturn(enableMcast);
@@ -367,7 +368,7 @@ public class JGroupsMessengerJUnitTest {
when(services.getShutdownCause()).thenReturn(shutdownCause);
try {
- GMSMember mbr = createAddress(8888);
+ MemberIdentifier mbr = createAddress(8888);
HeartbeatMessage msg = mock(HeartbeatMessage.class);
when(msg.getRecipients()).thenReturn(Collections.singletonList(mbr));
when(msg.getMulticast()).thenReturn(enableMcast);
@@ -402,7 +403,7 @@ public class JGroupsMessengerJUnitTest {
JChannel realChannel = messenger.myChannel;
messenger.myChannel = mockChannel;
try {
- GMSMember mbr = createAddress(8888);
+ MemberIdentifier mbr = createAddress(8888);
HeartbeatMessage msg =
mock(HeartbeatMessage.class);
when(msg.getRecipients()).thenReturn(Collections.singletonList(mbr));
@@ -425,7 +426,7 @@ public class JGroupsMessengerJUnitTest {
for (int i = 0; i < 2; i++) {
boolean enableMcast = (i == 1);
initMocks(enableMcast);
- GMSMember mbr = createAddress(8888);
+ MemberIdentifier mbr = createAddress(8888);
HeartbeatMessage msg =
mock(HeartbeatMessage.class);
when(msg.getRecipients()).thenReturn(Collections.singletonList(mbr));
@@ -462,12 +463,12 @@ public class JGroupsMessengerJUnitTest {
MessageHandler mh = mock(MessageHandler.class);
messenger.addHandler(JoinRequestMessage.class, mh);
- GMSMember addr = messenger.getMemberID();
+ MemberIdentifier addr = messenger.getMemberID();
GMSMembershipView v = new GMSMembershipView(addr);
when(joinLeave.getView()).thenReturn(v);
- GMSMember sender = createAddress(8888);
+ MemberIdentifier sender = createAddress(8888);
JoinRequestMessage msg = new JoinRequestMessage(messenger.localAddress, sender, null, -1, 0);
@@ -509,13 +510,13 @@ public class JGroupsMessengerJUnitTest {
MessageHandler mh = mock(MessageHandler.class);
messenger.addHandler(JoinRequestMessage.class, mh);
- GMSMember sender = messenger.getMemberID();
+ MemberIdentifier sender = messenger.getMemberID();
GMSMembershipView v = new GMSMembershipView(sender);
when(joinLeave.getView()).thenReturn(v);
messenger.installView(v);
// send a big message and expect fragmentation
- GMSMember recipient = broadcastMessage ? null : messenger.localAddress;
+ MemberIdentifier recipient = broadcastMessage ? null : messenger.localAddress;
services.getSerializer().registerDSFID(ByteHolder.DSFID, ByteHolder.class);
JoinRequestMessage msg = new JoinRequestMessage(recipient, sender,
new ByteHolder(
@@ -553,7 +554,7 @@ public class JGroupsMessengerJUnitTest {
int seqno = 1;
for (Message m : messages) {
if (jgroupsWillUseMulticast) {
- m.setSrc(messenger.localAddress.getUUID());
+ m.setSrc(messenger.localAddress.getMemberData().getUUID());
} else {
m.setSrc(fakeMember);
UNICAST3.Header oldHeader = (UNICAST3.Header) m.getHeader(unicastHeaderId);
@@ -572,15 +573,15 @@ public class JGroupsMessengerJUnitTest {
@Test
public void testSendToMultipleMembers() throws Exception {
initMocks(false);
- GMSMember sender = messenger.getMemberID();
- GMSMember other = createAddress(8888);
+ MemberIdentifier sender = messenger.getMemberID();
+ MemberIdentifier other = createAddress(8888);
GMSMembershipView v = new GMSMembershipView(sender);
v.add(other);
when(joinLeave.getView()).thenReturn(v);
messenger.installView(v);
- List<GMSMember> recipients = v.getMembers();
+ List<MemberIdentifier> recipients = v.getMembers();
HeartbeatMessage msg = new HeartbeatMessage();
msg.setRecipients(recipients);
@@ -746,7 +747,7 @@ public class JGroupsMessengerJUnitTest {
@Test
public void testMessageFiltering() throws Exception {
initMocks(true);
- GMSMember mbr = createAddress(8888);
+ MemberIdentifier mbr = createAddress(8888);
GMSMembershipView view = new GMSMembershipView(mbr);
// the digest should be set in an outgoing join response
@@ -777,7 +778,7 @@ public class JGroupsMessengerJUnitTest {
public void testPingPong() throws Exception {
initMocks(false);
GMSPingPonger pinger = messenger.pingPonger;
- GMSMember mbr = createAddress(8888);
+ MemberIdentifier mbr = createAddress(8888);
JGAddress addr = new JGAddress(mbr);
Message pingMessage = pinger.createPingMessage(null, addr);
@@ -817,7 +818,7 @@ public class JGroupsMessengerJUnitTest {
@Test
public void testJGroupsIOExceptionHandler() throws Exception {
initMocks(false);
- GMSMember mbr = createAddress(8888);
+ MemberIdentifier mbr = createAddress(8888);
GMSMembershipView v = new GMSMembershipView(mbr);
v.add(messenger.getMemberID());
messenger.installView(v);
@@ -850,14 +851,14 @@ public class JGroupsMessengerJUnitTest {
receiver.receive(msg);
// now create a view and a real distribution-message
- GMSMember myAddress = messenger.getMemberID();
- GMSMember other = createAddress(8888);
+ MemberIdentifier myAddress = messenger.getMemberID();
+ MemberIdentifier other = createAddress(8888);
GMSMembershipView v = new GMSMembershipView(myAddress);
v.add(other);
when(joinLeave.getView()).thenReturn(v);
messenger.installView(v);
- List<GMSMember> recipients = v.getMembers();
+ List<MemberIdentifier> recipients = v.getMembers();
HeartbeatMessage dmsg = new HeartbeatMessage();
dmsg.setRecipients(recipients);
@@ -876,7 +877,7 @@ public class JGroupsMessengerJUnitTest {
initMocks(false);
JChannel channel = messenger.myChannel;
tconfig.setOldDSMembershipInfo(new MembershipInformation(channel,
- Collections.singleton(new GMSMember("localhost", 10000)),
+ Collections.singleton(new InternalDistributedMember("localhost", 10000)),
new ConcurrentLinkedQueue<>()));
JGroupsMessenger newMessenger = new JGroupsMessenger();
newMessenger.init(services);
@@ -914,7 +915,7 @@ public class JGroupsMessengerJUnitTest {
public void testWaitForMessageStateSucceeds() throws Exception {
initMocks(true/* multicast */);
JGroupsMessenger.MessageTracker tracker = mock(JGroupsMessenger.MessageTracker.class);
- GMSMember mbr = createAddress(1234);
+ MemberIdentifier mbr = createAddress(1234);
messenger.scheduledMcastSeqnos.put(mbr, tracker);
when(tracker.get()).thenReturn(0l, 2l, 49l, 50l, 80l);
Map state = new HashMap();
@@ -940,7 +941,7 @@ public class JGroupsMessengerJUnitTest {
// message 50 will never arrive
Map state = new HashMap();
state.put("JGroups.mcastState", Long.valueOf(50));
- GMSMember mbr = createAddress(1234);
+ MemberIdentifier mbr = createAddress(1234);
messenger.scheduledMcastSeqnos.put(mbr, new JGroupsMessenger.MessageTracker(30));
messenger.waitForMessageState(mbr, state);
fail("expected a GemFireIOException to be thrown");
@@ -949,9 +950,9 @@ public class JGroupsMessengerJUnitTest {
}
}
- private GMSMembershipView createView(GMSMember otherMbr) {
- GMSMember sender = messenger.getMemberID();
- List<GMSMember> mbrs = new ArrayList<>();
+ private GMSMembershipView createView(MemberIdentifier otherMbr) {
+ MemberIdentifier sender = messenger.getMemberID();
+ List<MemberIdentifier> mbrs = new ArrayList<>();
mbrs.add(sender);
mbrs.add(otherMbr);
GMSMembershipView v = new GMSMembershipView(sender, 1, mbrs);
@@ -960,7 +961,7 @@ public class JGroupsMessengerJUnitTest {
@Test
public void testEncryptedFindCoordinatorRequest() throws Exception {
- GMSMember otherMbr = new GMSMember("localhost", 8888);
+ MemberIdentifier otherMbr = new InternalDistributedMember("localhost", 8888);
Properties p = new Properties();
final String udpDhalgo = "AES:128";
@@ -976,9 +977,9 @@ public class JGroupsMessengerJUnitTest {
messenger.initClusterKey();
FindCoordinatorRequest gfmsg = new FindCoordinatorRequest(messenger.getMemberID(),
- new ArrayList<GMSMember>(2), 1,
+ new ArrayList<MemberIdentifier>(2), 1,
messenger.getPublicKey(messenger.getMemberID()), 1, "");
- List<GMSMember> recipients = new ArrayList<>();
+ List<MemberIdentifier> recipients = new ArrayList<>();
recipients.add(otherMbr);
gfmsg.setRecipients(recipients);
@@ -1001,7 +1002,7 @@ public class JGroupsMessengerJUnitTest {
@Test
public void testEncryptedFindCoordinatorResponse() throws Exception {
- GMSMember otherMbr = new GMSMember("localhost", 8888);
+ MemberIdentifier otherMbr = new InternalDistributedMember("localhost", 8888);
Properties p = new Properties();
@@ -1019,7 +1020,7 @@ public class JGroupsMessengerJUnitTest {
FindCoordinatorResponse gfmsg = new FindCoordinatorResponse(messenger.getMemberID(),
messenger.getMemberID(), messenger.getClusterSecretKey(), 1);
- List<GMSMember> recipients = new ArrayList<>();
+ List<MemberIdentifier> recipients = new ArrayList<>();
recipients.add(otherMbr);
gfmsg.setRecipients(recipients);
@@ -1044,7 +1045,7 @@ public class JGroupsMessengerJUnitTest {
@Test
public void testEncryptedJoinRequest() throws Exception {
- GMSMember otherMbr = new GMSMember("localhost", 8888);
+ MemberIdentifier otherMbr = new InternalDistributedMember("localhost", 8888);
Properties p = new Properties();
p.put(ConfigurationProperties.SECURITY_UDP_DHALGO, AES_128);
@@ -1079,7 +1080,7 @@ public class JGroupsMessengerJUnitTest {
@Test
public void testEncryptedJoinResponse() throws Exception {
- GMSMember otherMbr = new GMSMember("localhost", 8888);
+ MemberIdentifier otherMbr = new InternalDistributedMember("localhost", 8888);
Properties p = new Properties();
p.put(ConfigurationProperties.SECURITY_UDP_DHALGO, AES_128);
@@ -1136,11 +1137,11 @@ public class JGroupsMessengerJUnitTest {
}
- private GMSMember createAddress(int port) {
- GMSMember gms = new GMSMember("localhost", port);
- gms.setUUID(UUID.randomUUID());
- gms.setVmKind(GMSMember.NORMAL_DM_TYPE);
- gms.setVersionOrdinal(Version.getCurrentVersion().ordinal());
+ private MemberIdentifier createAddress(int port) {
+ MemberIdentifier gms = new InternalDistributedMember("localhost", port);
+ gms.getMemberData().setUUID(UUID.randomUUID());
+ gms.getMemberData().setVmKind(MemberIdentifier.NORMAL_DM_TYPE);
+ gms.getMemberData().setVersionOrdinal(Version.getCurrentVersion().ordinal());
return gms;
}
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/FilterProfileConcurrencyTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/FilterProfileConcurrencyTest.java
index c95d9e6..cdd5f48 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/FilterProfileConcurrencyTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/FilterProfileConcurrencyTest.java
@@ -27,7 +27,6 @@ import org.junit.runner.RunWith;
import org.apache.geode.distributed.DistributedMember;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.MemberAttributes;
import org.apache.geode.internal.cache.tier.InterestType;
import org.apache.geode.internal.serialization.Version;
import org.apache.geode.internal.util.BlobHelper;
@@ -63,7 +62,7 @@ public class FilterProfileConcurrencyTest {
private FilterProfile createFilterProfile() throws UnknownHostException {
DistributedMember member = new InternalDistributedMember(InetAddress.getLocalHost(), 0, false,
- false, MemberAttributes.DEFAULT);
+ false);
return new FilterProfile(null, member, true);
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
index ff94f77..8cab023 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
@@ -21,6 +21,7 @@ import java.net.UnknownHostException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
+import java.util.Comparator;
import java.util.Date;
import java.util.HashMap;
import java.util.HashSet;
@@ -62,6 +63,9 @@ import org.apache.geode.distributed.internal.membership.MembershipManager;
import org.apache.geode.distributed.internal.membership.MembershipView;
import org.apache.geode.distributed.internal.membership.adapter.ServiceConfig;
import org.apache.geode.distributed.internal.membership.adapter.auth.GMSAuthenticator;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberData;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifierFactory;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipBuilder;
import org.apache.geode.internal.Assert;
import org.apache.geode.internal.InternalDataSerializer;
@@ -118,16 +122,16 @@ public class ClusterDistributionManager implements DistributionManager {
/** The DM type for regular distribution managers */
- public static final int NORMAL_DM_TYPE = 10;
+ public static final int NORMAL_DM_TYPE = MemberIdentifier.NORMAL_DM_TYPE;
/** The DM type for locator distribution managers */
- public static final int LOCATOR_DM_TYPE = 11;
+ public static final int LOCATOR_DM_TYPE = MemberIdentifier.LOCATOR_DM_TYPE;
/** The DM type for Console (admin-only) distribution managers */
- public static final int ADMIN_ONLY_DM_TYPE = 12;
+ public static final int ADMIN_ONLY_DM_TYPE = MemberIdentifier.ADMIN_ONLY_DM_TYPE;
/** The DM type for stand-alone members */
- public static final int LONER_DM_TYPE = 13;
+ public static final int LONER_DM_TYPE = MemberIdentifier.LONER_DM_TYPE;
@@ -456,6 +460,7 @@ public class ClusterDistributionManager implements DistributionManager {
.setMembershipListener(listener)
.setConfig(new ServiceConfig(transport, system.getConfig()))
.setSerializer(InternalDataSerializer.getDSFIDSerializer())
+ .setMemberIDFactory(new ClusterDistributionManagerIDFactory())
.create();
sb.append(System.currentTimeMillis() - start);
@@ -2821,4 +2826,23 @@ public class ClusterDistributionManager implements DistributionManager {
return stopper;
}
+ static class ClusterDistributionManagerIDFactory implements MemberIdentifierFactory {
+ static final Comparator<MemberIdentifier> idComparator = new Comparator() {
+ @Override
+ public int compare(Object o1, Object o2) {
+ return ((DistributedMember) o1).compareTo((DistributedMember) o2);
+ }
+ };
+
+ @Override
+ public MemberIdentifier create(MemberData memberInfo) {
+ return new InternalDistributedMember(memberInfo);
+ }
+
+ @Override
+ public Comparator<MemberIdentifier> getComparator() {
+ return idComparator;
+ }
+ }
+
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
index 12f14f0..9395b72 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/LonerDistributionManager.java
@@ -46,8 +46,8 @@ import org.apache.geode.distributed.DurableClientAttributes;
import org.apache.geode.distributed.Role;
import org.apache.geode.distributed.internal.locks.ElderState;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.MemberAttributes;
import org.apache.geode.distributed.internal.membership.MembershipManager;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberDataBuilder;
import org.apache.geode.i18n.LogWriterI18n;
import org.apache.geode.internal.cache.InternalCache;
import org.apache.geode.internal.logging.InternalLogWriter;
@@ -1216,7 +1216,7 @@ public class LonerDistributionManager implements DistributionManager {
}
result = new InternalDistributedMember(host, lonerPort, name, uniqueString,
ClusterDistributionManager.LONER_DM_TYPE,
- MemberAttributes.parseGroups(config.getRoles(), config.getGroups()), dac);
+ MemberDataBuilder.parseGroups(config.getRoles(), config.getGroups()), dac);
} catch (UnknownHostException ex) {
throw new InternalGemFireError(
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalDistributedMember.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalDistributedMember.java
index 0808863..08963fa 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalDistributedMember.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalDistributedMember.java
@@ -42,6 +42,9 @@ import org.apache.geode.distributed.Role;
import org.apache.geode.distributed.internal.ClusterDistributionManager;
import org.apache.geode.distributed.internal.DistributionAdvisor.ProfileId;
import org.apache.geode.distributed.internal.ServerLocation;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberData;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberDataBuilder;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.Assert;
import org.apache.geode.internal.InternalDataSerializer;
import org.apache.geode.internal.OSProcess;
@@ -56,11 +59,12 @@ import org.apache.geode.internal.serialization.Version;
/**
* This is the fundamental representation of a member of a GemFire distributed system.
*/
-public class InternalDistributedMember implements DistributedMember, Externalizable,
+public class InternalDistributedMember
+ implements DistributedMember, MemberIdentifier, Externalizable,
DataSerializableFixedID, ProfileId, VersionSource<DistributedMember> {
private static final long serialVersionUID = -2785249969777296507L;
- protected NetMember netMbr; // the underlying member object
+ private MemberData memberData; // the underlying member object
/**
* whether this is a partial member ID (without roles, durable attributes). We use partial IDs in
@@ -96,7 +100,12 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
private static final int VERSION_BIT = 0x8;
public int getVmPid() {
- return netMbr.getProcessId();
+ return memberData.getProcessId();
+ }
+
+ @Override
+ public int compareTo(DistributedMember o) {
+ return compareTo(o, false, true);
}
@FunctionalInterface
@@ -121,12 +130,12 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
Version.GFE_71, Version.GFE_90};
private void defaultToCurrentHost() {
- netMbr.setProcessId(OSProcess.getId());
+ memberData.setProcessId(OSProcess.getId());
try {
if (SocketCreator.resolve_dns) {
- netMbr.setHostName(SocketCreator.getHostName(SocketCreator.getLocalHost()));
+ memberData.setHostName(SocketCreator.getHostName(SocketCreator.getLocalHost()));
} else {
- netMbr.setHostName(SocketCreator.getLocalHost().getHostAddress());
+ memberData.setHostName(SocketCreator.getLocalHost().getHostAddress());
}
} catch (UnknownHostException ee) {
throw new InternalGemFireError(ee);
@@ -138,7 +147,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
public InternalDistributedMember() {}
/**
- * Construct a InternalDistributedMember. All fields are specified.
+ * Construct a InternalDistributedMember
* <p>
*
* This, and the following constructor are the only valid ways to create an ID for a distributed
@@ -146,41 +155,35 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
* network-partition-detection.
*
* @param i the inet address
- * @param p the membership port
+ * @param membershipPort the membership port
* @param splitBrainEnabled whether this feature is enabled for the member
* @param canBeCoordinator whether the member is eligible to be the membership coordinator
- * @param attr the member's attributes
*/
- public InternalDistributedMember(InetAddress i, int p, boolean splitBrainEnabled,
- boolean canBeCoordinator, MemberAttributes attr) {
+ public InternalDistributedMember(InetAddress i, int membershipPort, boolean splitBrainEnabled,
+ boolean canBeCoordinator) {
String hostName = SocketCreator.resolve_dns ? SocketCreator.getHostName(i) : i.getHostAddress();
- this.netMbr = NetMemberFactory.newNetMember(i, hostName, p, splitBrainEnabled, canBeCoordinator,
- Version.CURRENT_ORDINAL,
- attr);
-
- short version = netMbr.getVersionOrdinal();
- try {
- this.versionObj = Version.fromOrdinal(version);
- } catch (UnsupportedSerializationVersionException e) {
- this.versionObj = Version.CURRENT;
- }
- // checkHostName();
+ this.memberData = MemberDataBuilder.newBuilder(i, hostName)
+ .setMembershipPort(membershipPort)
+ .setNetworkPartitionDetectionEnabled(splitBrainEnabled)
+ .setPreferredForCoordinator(canBeCoordinator)
+ .build();
+ this.versionObj = Version.CURRENT;
}
/**
- * Construct a InternalDistributedMember based on the given NetMember.
+ * Construct a InternalDistributedMember based on the given member data.
*
*/
- public InternalDistributedMember(NetMember m) {
- netMbr = m;
+ public InternalDistributedMember(MemberData m) {
+ memberData = m;
- if (netMbr.getHostName() == null || netMbr.isPartial()) {
+ if (memberData.getHostName() == null || memberData.isPartial()) {
String hostName = SocketCreator.resolve_dns ? SocketCreator.getHostName(m.getInetAddress())
: m.getInetAddress().getHostAddress();
- netMbr.setHostName(hostName);
+ memberData.setHostName(hostName);
}
short version = m.getVersionOrdinal();
@@ -194,14 +197,14 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
}
/**
- * Replace the current NetMember with the given member. This can be used to fill out an
- * InternalDistributedMember that was created from a partial NetMember created by
+ * Replace the current member data with the given member data. This can be used to fill out an
+ * InternalDistributedMember that was created from a partial data created by
* readEssentialData.
*
- * @param m the replacement NetMember
+ * @param m the replacement member data
*/
- public void setNetMember(NetMember m) {
- this.netMbr = m;
+ public void setMemberData(MemberData m) {
+ this.memberData = m;
}
/**
@@ -213,17 +216,20 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
* DOES NOT PROPERLY INITIALIZE ATTRIBUTES NEEDED FOR P2P FUNCTIONALITY. </b>
*
*
- * @param i the hostname, must be for the current host
+ * @param i the hostname, stored in the member ID but not resolved - local host inet addr is used
* @param p the membership listening port
- * @throws UnknownHostException if the given hostname cannot be resolved
+ * @throws RuntimeException if the given hostname cannot be resolved
*/
+ @VisibleForTesting
public InternalDistributedMember(String i, int p) {
- this(NetMemberFactory.newNetMember(i, p));
+ this(MemberDataBuilder.newBuilderForLocalHost(i)
+ .setMembershipPort(p)
+ .build());
}
/**
- * Creates a new InternalDistributedMember for use in notifying membership listeners. The version
- * information in the ID is set to Version.CURRENT.
+ * Creates a new InternalDistributedMember for use in notifying listeners in client
+ * caches. The version information in the ID is set to Version.CURRENT.
*
* @param location the coordinates of the server
*/
@@ -236,9 +242,11 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
throw new ServerConnectivityException("Unable to resolve server location " + location, e);
}
- netMbr =
- NetMemberFactory.newNetMember(addr, location.getHostName(), location.getPort(), false, true,
- Version.CURRENT_ORDINAL, MemberAttributes.DEFAULT);
+ memberData = MemberDataBuilder.newBuilder(addr, location.getHostName())
+ .setMembershipPort(location.getPort())
+ .setNetworkPartitionDetectionEnabled(false)
+ .setPreferredForCoordinator(true)
+ .build();
versionObj = Version.CURRENT;
}
@@ -253,8 +261,8 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
* INITIALIZE THE ID. </b>
*
* @param host the hostname, must be for the current host
- * @param p the membership listening port
- * @param n gemfire properties connection name
+ * @param p the membership port
+ * @param n member name
* @param u unique string used make the member more unique
* @param vmKind the dmType
* @param groups the server groups / roles
@@ -264,34 +272,39 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
*/
public InternalDistributedMember(String host, int p, String n, String u, int vmKind,
String[] groups, DurableClientAttributes attr) throws UnknownHostException {
- MemberAttributes mattr = new MemberAttributes(p, org.apache.geode.internal.OSProcess.getId(),
- vmKind, -1, n, groups, attr);
InetAddress addr = SocketCreator.toInetAddress(host);
- netMbr = NetMemberFactory
- .newNetMember(addr, host, p, false, true, Version.CURRENT_ORDINAL, mattr);
+ MemberDataBuilder builder = MemberDataBuilder.newBuilder(addr, host)
+ .setName(n)
+ .setMembershipPort(p)
+ .setDirectPort(p)
+ .setPreferredForCoordinator(false)
+ .setNetworkPartitionDetectionEnabled(true)
+ .setVmKind(vmKind)
+ .setGroups(groups);
+ if (attr != null) {
+ builder.setDurableId(attr.getId())
+ .setDurableTimeout(attr.getTimeout());
+ }
+ memberData = builder.build();
defaultToCurrentHost();
- netMbr.setName(n);
this.uniqueTag = u;
- netMbr.setVmKind(vmKind);
- netMbr.setDirectPort(p);
- netMbr.setDurableClientAttributes(attr);
- netMbr.setGroups(groups);
}
/**
- * Create a InternalDistributedMember referring to the current host (as defined by the given
- * address).
+ * Create a InternalDistributedMember
* <p>
*
* <b> THIS METHOD IS FOR TESTING ONLY. DO NOT USE IT TO CREATE IDs FOR USE IN THE PRODUCT. IT
* DOES NOT PROPERLY INITIALIZE ATTRIBUTES NEEDED FOR P2P FUNCTIONALITY. </b>
*
*
- * @param i the hostname, must be for the current host
+ * @param i the host address
* @param p the membership listening port
*/
public InternalDistributedMember(InetAddress i, int p) {
- netMbr = NetMemberFactory.newNetMember(i, p);
+ memberData = MemberDataBuilder.newBuilder(i, "localhost")
+ .setMembershipPort(p)
+ .build();
defaultToCurrentHost();
}
@@ -308,7 +321,8 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
* false to create a temporary id for the OTHER side of a connection)
*/
public InternalDistributedMember(InetAddress addr, int p, boolean isCurrentHost) {
- netMbr = NetMemberFactory.newNetMember(addr, p);
+ memberData = MemberDataBuilder.newBuilder(addr, "localhost")
+ .setMembershipPort(p).build();
if (isCurrentHost) {
defaultToCurrentHost();
}
@@ -320,11 +334,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
* @return the underlying host address
*/
public InetAddress getInetAddress() {
- return netMbr.getInetAddress();
- }
-
- public NetMember getNetMember() {
- return netMbr;
+ return memberData.getInetAddress();
}
/**
@@ -333,16 +343,20 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
* @return the underlying membership port
*/
public int getPort() {
- return netMbr.getPort();
+ return memberData.getPort();
}
+ @Override
+ public short getVersionOrdinal() {
+ return versionObj == null ? memberData.getVersionOrdinal() : versionObj.ordinal();
+ }
/**
* Returns the port on which the direct channel runs
*/
public int getDirectChannelPort() {
assert !this.isPartial;
- return netMbr.getDirectPort();
+ return memberData.getDirectChannelPort();
}
/**
@@ -352,7 +366,12 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
* @see ClusterDistributionManager#NORMAL_DM_TYPE
*/
public int getVmKind() {
- return netMbr.getVmKind();
+ return memberData.getVmKind();
+ }
+
+ @Override
+ public int getMemberWeight() {
+ return memberData.getMemberWeight();
}
/**
@@ -360,7 +379,12 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
* this is limited to 16 bits.
*/
public int getVmViewId() {
- return netMbr.getVmViewId();
+ return memberData.getVmViewId();
+ }
+
+ @Override
+ public boolean preferredForCoordinator() {
+ return memberData.preferredForCoordinator();
}
/**
@@ -376,7 +400,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
synchronized (this.rolesLock) {
tmpRolesSet = this.rolesSet;
if (tmpRolesSet == null) {
- final String[] tmpRoles = netMbr.getGroups();
+ final String[] tmpRoles = memberData.getGroups();
// convert array of string role names to array of Roles...
if (tmpRoles == null || tmpRoles.length == 0) {
tmpRolesSet = Collections.emptySet();
@@ -396,21 +420,21 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
@Override
public List<String> getGroups() {
- return Collections.unmodifiableList(Arrays.asList(netMbr.getGroups()));
+ return Collections.unmodifiableList(Arrays.asList(memberData.getGroups()));
}
public void setGroups(String[] newGroups) {
assert !this.isPartial;
assert newGroups != null;
synchronized (this.rolesLock) {
- netMbr.setGroups(newGroups);
+ memberData.setGroups(newGroups);
this.rolesSet = null;
this.cachedToString = null;
}
}
public void setVmViewId(int p) {
- netMbr.setVmViewId(p);
+ memberData.setVmViewId(p);
cachedToString = null;
}
@@ -420,7 +444,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
*/
@Override
public String getName() {
- String result = netMbr.getName();
+ String result = memberData.getName();
if (result == null) {
result = "";
}
@@ -433,34 +457,14 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
@Override
public DurableClientAttributes getDurableClientAttributes() {
assert !this.isPartial;
- DurableClientAttributes attributes = netMbr.getDurableClientAttributes();
- if (attributes == null) {
- attributes = new DurableClientAttributes("", 300);
- netMbr.setDurableClientAttributes(attributes);
+ String durableId = memberData.getDurableId();
+ if (durableId == null || durableId.isEmpty()) {
+ return new DurableClientAttributes("", 300);
}
- return netMbr.getDurableClientAttributes();
- }
-
- /**
- * implements the java.lang.Comparable interface
- *
- * @see java.lang.Comparable
- * @param o - the Object to be compared
- * @return a negative integer, zero, or a positive integer as this object is less than, equal to,
- * or greater than the specified object.
- * @exception java.lang.ClassCastException - if the specified object's type prevents it from being
- * compared to this Object.
- */
- @Override
- public int compareTo(DistributedMember o) {
- return compareTo(o, true);
+ return new DurableClientAttributes(durableId, memberData.getDurableTimeout());
}
- public int compareTo(DistributedMember o, boolean checkNetMembersIfEqual) {
- return compareTo(o, checkNetMembersIfEqual, true);
- }
-
- public int compareTo(DistributedMember o, boolean checkNetMembersIfEqual, boolean verifyViewId) {
+ public int compareTo(DistributedMember o, boolean compareMemberData, boolean compareViewIds) {
if (this == o) {
return 0;
}
@@ -523,7 +527,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
}
if (this.uniqueTag == null && other.uniqueTag == null) {
- if (verifyViewId) {
+ if (compareViewIds) {
// not loners, so look at P2P view ID
int thisViewId = getVmViewId();
int otherViewId = other.getVmViewId();
@@ -546,14 +550,11 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
}
}
- if (checkNetMembersIfEqual && this.netMbr != null && other.netMbr != null) {
- return this.netMbr.compareAdditionalData(other.netMbr);
+ if (compareMemberData && this.memberData != null && other.memberData != null) {
+ return this.memberData.compareAdditionalData(other.memberData);
} else {
return 0;
}
-
- // purposely avoid comparing roles
- // @todo Add durableClientAttributes to compare
}
/**
@@ -610,8 +611,8 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
return false;
}
- if (this.netMbr != null && other.netMbr != null) {
- if (0 != this.netMbr.compareAdditionalData(other.netMbr)) {
+ if (this.memberData != null && other.memberData != null) {
+ if (0 != this.memberData.compareAdditionalData(other.memberData)) {
return false;
}
}
@@ -625,7 +626,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
@Override
public int hashCode() {
int result = 0;
- result = result + netMbr.getInetAddress().hashCode();
+ result = result + memberData.getInetAddress().hashCode();
result = result + getPort();
return result;
}
@@ -653,17 +654,13 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
addFixedToString(sb);
// add version if not current
- short version = netMbr.getVersionOrdinal();
+ short version = memberData.getVersionOrdinal();
if (version != Version.CURRENT.ordinal()) {
sb.append("(version:").append(Version.toString(version)).append(')');
}
// leave out Roles on purpose
- // if (netMbr instanceof GMSMember) {
- // sb.append("(UUID=").append(((GMSMember)netMbr).getUUID()).append(")");
- // }
-
result = sb.toString();
cachedToString = result;
}
@@ -679,15 +676,15 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
if (add.isMulticastAddress())
host = add.getHostAddress();
else {
- String hostName = netMbr.getHostName();
+ String hostName = memberData.getHostName();
host = SocketCreator.resolve_dns ? shortName(hostName) : hostName;
}
sb.append(host);
String myName = getName();
- int vmPid = netMbr.getProcessId();
- int vmKind = netMbr.getVmKind();
+ int vmPid = memberData.getProcessId();
+ int vmKind = memberData.getVmKind();
if (vmPid > 0 || vmKind != ClusterDistributionManager.NORMAL_DM_TYPE || !"".equals(myName)) {
sb.append("(");
@@ -722,7 +719,8 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
sb.append(vmStr);
sb.append(")");
}
- if (vmKind != ClusterDistributionManager.LONER_DM_TYPE && netMbr.preferredForCoordinator()) {
+ if (vmKind != ClusterDistributionManager.LONER_DM_TYPE
+ && memberData.preferredForCoordinator()) {
sb.append("<ec>");
}
int vmViewId = getVmViewId();
@@ -768,7 +766,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
*/
@Override
public void writeExternal(ObjectOutput out) throws IOException {
- Assert.assertTrue(netMbr.getVmKind() > 0);
+ Assert.assertTrue(memberData.getVmKind() > 0);
// do it the way we like
byte[] address = getInetAddress().getAddress();
@@ -777,12 +775,12 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
out.write(address);
out.writeInt(getPort());
- DataSerializer.writeString(netMbr.getHostName(), out);
+ DataSerializer.writeString(memberData.getHostName(), out);
int flags = 0;
- if (netMbr.isNetworkPartitionDetectionEnabled())
+ if (memberData.isNetworkPartitionDetectionEnabled())
flags |= NPD_ENABLED_BIT;
- if (netMbr.preferredForCoordinator())
+ if (memberData.preferredForCoordinator())
flags |= COORD_ENABLED_BIT;
if (this.isPartial)
flags |= PARTIAL_ID_BIT;
@@ -791,20 +789,21 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
flags |= VERSION_BIT;
out.writeByte((byte) (flags & 0xff));
- out.writeInt(netMbr.getDirectPort());
- out.writeInt(netMbr.getProcessId());
- out.writeInt(netMbr.getVmKind());
- out.writeInt(netMbr.getVmViewId());
- DataSerializer.writeStringArray(netMbr.getGroups(), out);
+ out.writeInt(memberData.getDirectChannelPort());
+ out.writeInt(memberData.getProcessId());
+ out.writeInt(memberData.getVmKind());
+ out.writeInt(memberData.getVmViewId());
+ DataSerializer.writeStringArray(memberData.getGroups(), out);
- DataSerializer.writeString(netMbr.getName(), out);
+ DataSerializer.writeString(memberData.getName(), out);
DataSerializer.writeString(this.uniqueTag, out);
- DurableClientAttributes attributes = netMbr.getDurableClientAttributes();
- DataSerializer.writeString(attributes == null ? "" : attributes.getId(), out);
- DataSerializer.writeInteger(Integer.valueOf(attributes == null ? 300 : attributes.getTimeout()),
+ String durableId = memberData.getDurableId();
+ DataSerializer.writeString(durableId == null ? "" : durableId, out);
+ DataSerializer.writeInteger(
+ Integer.valueOf(durableId == null ? 300 : memberData.getDurableTimeout()),
out);
- Version.writeOrdinal(out, netMbr.getVersionOrdinal(), true);
- netMbr.writeAdditionalData(out);
+ Version.writeOrdinal(out, memberData.getVersionOrdinal(), true);
+ memberData.writeAdditionalData(out);
}
/**
@@ -837,23 +836,31 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
this.uniqueTag = DataSerializer.readString(in);
String durableId = DataSerializer.readString(in);
int durableTimeout = in.readInt();
- DurableClientAttributes durableClientAttributes =
- new DurableClientAttributes(durableId, durableTimeout);
short version = readVersion(flags, in);
- netMbr = NetMemberFactory.newNetMember(inetAddr, hostName, port, sbEnabled, elCoord, version,
- new MemberAttributes(dcPort, vmPid, vmKind, vmViewId, name, groups,
- durableClientAttributes));
+ memberData = MemberDataBuilder.newBuilder(inetAddr, hostName)
+ .setMembershipPort(port)
+ .setDirectPort(dcPort)
+ .setName(name)
+ .setNetworkPartitionDetectionEnabled(sbEnabled)
+ .setPreferredForCoordinator(elCoord)
+ .setVersionOrdinal(version)
+ .setVmPid(vmPid)
+ .setVmKind(vmKind)
+ .setVmViewId(vmViewId)
+ .setGroups(groups)
+ .setDurableId(durableId)
+ .setDurableTimeout(durableTimeout)
+ .build();
if (version >= Version.GFE_90.ordinal()) {
try {
- netMbr.readAdditionalData(in);
+ memberData.readAdditionalData(in);
} catch (java.io.EOFException e) {
// old version
}
}
-
- Assert.assertTrue(netMbr.getVmKind() > 0);
+ Assert.assertTrue(memberData.getVmKind() > 0);
}
@Override
@@ -865,8 +872,8 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
public void toData(DataOutput out,
SerializationContext context) throws IOException {
toDataPre_GFE_9_0_0_0(out, context);
- if (netMbr.getVersionOrdinal() >= Version.GFE_90.ordinal()) {
- getNetMember().writeAdditionalData(out);
+ if (memberData.getVersionOrdinal() >= Version.GFE_90.ordinal()) {
+ getMemberData().writeAdditionalData(out);
}
}
@@ -880,12 +887,12 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
DataSerializer.writeInetAddress(getInetAddress(), out);
out.writeInt(getPort());
- DataSerializer.writeString(netMbr.getHostName(), out);
+ DataSerializer.writeString(memberData.getHostName(), out);
int flags = 0;
- if (netMbr.isNetworkPartitionDetectionEnabled())
+ if (memberData.isNetworkPartitionDetectionEnabled())
flags |= NPD_ENABLED_BIT;
- if (netMbr.preferredForCoordinator())
+ if (memberData.preferredForCoordinator())
flags |= COORD_ENABLED_BIT;
if (this.isPartial)
flags |= PARTIAL_ID_BIT;
@@ -895,31 +902,31 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
out.writeByte((byte) (flags & 0xff));
- out.writeInt(netMbr.getDirectPort());
- out.writeInt(netMbr.getProcessId());
- int vmKind = netMbr.getVmKind();
+ out.writeInt(memberData.getDirectChannelPort());
+ out.writeInt(memberData.getProcessId());
+ int vmKind = memberData.getVmKind();
out.writeByte(vmKind);
- DataSerializer.writeStringArray(netMbr.getGroups(), out);
+ DataSerializer.writeStringArray(memberData.getGroups(), out);
- DataSerializer.writeString(netMbr.getName(), out);
+ DataSerializer.writeString(memberData.getName(), out);
if (vmKind == ClusterDistributionManager.LONER_DM_TYPE) {
DataSerializer.writeString(this.uniqueTag, out);
} else { // added in 6.5 for unique identifiers in P2P
- DataSerializer.writeString(String.valueOf(netMbr.getVmViewId()), out);
+ DataSerializer.writeString(String.valueOf(memberData.getVmViewId()), out);
}
- DurableClientAttributes durableClientAttributes = netMbr.getDurableClientAttributes();
- DataSerializer
- .writeString(durableClientAttributes == null ? "" : durableClientAttributes.getId(), out);
- DataSerializer.writeInteger(Integer.valueOf(
- durableClientAttributes == null ? 300 : durableClientAttributes.getTimeout()), out);
+ String durableId = memberData.getDurableId();
+ DataSerializer.writeString(durableId == null ? "" : durableId, out);
+ DataSerializer.writeInteger(
+ Integer.valueOf(durableId == null ? 300 : memberData.getDurableTimeout()),
+ out);
- short version = netMbr.getVersionOrdinal();
+ short version = memberData.getVersionOrdinal();
Version.writeOrdinal(out, version, true);
}
public void toDataPre_GFE_7_1_0_0(DataOutput out, SerializationContext context)
throws IOException {
- Assert.assertTrue(netMbr.getVmKind() > 0);
+ Assert.assertTrue(memberData.getVmKind() > 0);
// disabled to allow post-connect setting of the port for loner systems
// Assert.assertTrue(getPort() > 0);
// if (this.getPort() == 0) {
@@ -933,34 +940,34 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
DataSerializer.writeInetAddress(getInetAddress(), out);
out.writeInt(getPort());
- DataSerializer.writeString(netMbr.getHostName(), out);
+ DataSerializer.writeString(memberData.getHostName(), out);
int flags = 0;
- if (netMbr.isNetworkPartitionDetectionEnabled())
+ if (memberData.isNetworkPartitionDetectionEnabled())
flags |= NPD_ENABLED_BIT;
- if (netMbr.preferredForCoordinator())
+ if (memberData.preferredForCoordinator())
flags |= COORD_ENABLED_BIT;
if (this.isPartial)
flags |= PARTIAL_ID_BIT;
out.writeByte((byte) (flags & 0xff));
- out.writeInt(netMbr.getDirectPort());
- out.writeInt(netMbr.getProcessId());
- out.writeByte(netMbr.getVmKind());
- DataSerializer.writeStringArray(netMbr.getGroups(), out);
+ out.writeInt(memberData.getDirectChannelPort());
+ out.writeInt(memberData.getProcessId());
+ out.writeByte(memberData.getVmKind());
+ DataSerializer.writeStringArray(memberData.getGroups(), out);
- DataSerializer.writeString(netMbr.getName(), out);
- int vmKind = netMbr.getVmKind();
+ DataSerializer.writeString(memberData.getName(), out);
+ int vmKind = memberData.getVmKind();
if (vmKind == ClusterDistributionManager.LONER_DM_TYPE) {
DataSerializer.writeString(this.uniqueTag, out);
} else { // added in 6.5 for unique identifiers in P2P
- DataSerializer.writeString(String.valueOf(netMbr.getVmViewId()), out);
+ DataSerializer.writeString(String.valueOf(memberData.getVmViewId()), out);
}
- DurableClientAttributes durableClientAttributes = netMbr.getDurableClientAttributes();
- DataSerializer
- .writeString(durableClientAttributes == null ? "" : durableClientAttributes.getId(), out);
- DataSerializer.writeInteger(Integer.valueOf(
- durableClientAttributes == null ? 300 : durableClientAttributes.getTimeout()), out);
+ String durableId = memberData.getDurableId();
+ DataSerializer.writeString(durableId == null ? "" : durableId, out);
+ DataSerializer.writeInteger(
+ Integer.valueOf(durableId == null ? 300 : memberData.getDurableTimeout()),
+ out);
}
@@ -970,9 +977,9 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
fromDataPre_GFE_9_0_0_0(in, context);
// just in case this is just a non-versioned read
// from a file we ought to check the version
- if (getNetMember().getVersionOrdinal() >= Version.GFE_90.ordinal()) {
+ if (getMemberData().getVersionOrdinal() >= Version.GFE_90.ordinal()) {
try {
- netMbr.readAdditionalData(in);
+ memberData.readAdditionalData(in);
} catch (EOFException e) {
// nope - it's from a pre-GEODE client or WAN site
}
@@ -1012,17 +1019,25 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
String durableId = DataSerializer.readString(in);
int durableTimeout = in.readInt();
- DurableClientAttributes durableClientAttributes =
- durableId.length() > 0 ? new DurableClientAttributes(durableId, durableTimeout) : null;
short version = readVersion(flags, in);
- MemberAttributes attr = new MemberAttributes(dcPort, vmPid, vmKind, vmViewId, name, groups,
- durableClientAttributes);
- netMbr =
- NetMemberFactory.newNetMember(inetAddr, hostName, port, sbEnabled, elCoord, version, attr);
-
- Assert.assertTrue(netMbr.getVmKind() > 0);
+ memberData = MemberDataBuilder.newBuilder(inetAddr, hostName)
+ .setMembershipPort(port)
+ .setDirectPort(dcPort)
+ .setName(name)
+ .setNetworkPartitionDetectionEnabled(sbEnabled)
+ .setPreferredForCoordinator(elCoord)
+ .setVersionOrdinal(version)
+ .setVmPid(vmPid)
+ .setVmKind(vmKind)
+ .setVmViewId(vmViewId)
+ .setGroups(groups)
+ .setDurableId(durableId)
+ .setDurableTimeout(durableTimeout)
+ .build();
+
+ Assert.assertTrue(memberData.getVmKind() > 0);
// Assert.assertTrue(getPort() > 0);
}
@@ -1059,17 +1074,25 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
String durableId = DataSerializer.readString(in);
int durableTimeout = in.readInt();
- DurableClientAttributes durableClientAttributes =
- durableId.length() > 0 ? new DurableClientAttributes(durableId, durableTimeout) : null;
short version = readVersion(flags, in);
- MemberAttributes attr = new MemberAttributes(dcPort, vmPid, vmKind, vmViewId, name, groups,
- durableClientAttributes);
- netMbr =
- NetMemberFactory.newNetMember(inetAddr, hostName, port, sbEnabled, elCoord, version, attr);
-
- Assert.assertTrue(netMbr.getVmKind() > 0);
+ memberData = MemberDataBuilder.newBuilder(inetAddr, hostName)
+ .setMembershipPort(port)
+ .setDirectPort(dcPort)
+ .setName(name)
+ .setNetworkPartitionDetectionEnabled(sbEnabled)
+ .setPreferredForCoordinator(elCoord)
+ .setVersionOrdinal(version)
+ .setVmPid(vmPid)
+ .setVmKind(vmKind)
+ .setVmViewId(vmViewId)
+ .setGroups(groups)
+ .setDurableId(durableId)
+ .setDurableTimeout(durableTimeout)
+ .build();
+
+ Assert.assertTrue(memberData.getVmKind() > 0);
}
/** this reads an ID written with writeEssentialData */
@@ -1106,46 +1129,52 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
String name = DataSerializer.readString(in);
- MemberAttributes attr = new MemberAttributes(-1, -1, vmKind, vmViewId, name, null, null);
- netMbr = NetMemberFactory.newNetMember(inetAddr, hostName, port, sbEnabled, elCoord,
- InternalDataSerializer.getVersionForDataStream(in).ordinal(), attr);
+ memberData = MemberDataBuilder.newBuilder(inetAddr, hostName)
+ .setMembershipPort(port)
+ .setName(name)
+ .setNetworkPartitionDetectionEnabled(sbEnabled)
+ .setPreferredForCoordinator(elCoord)
+ .setVersionOrdinal(InternalDataSerializer.getVersionForDataStream(in).ordinal())
+ .setVmKind(vmKind)
+ .setVmViewId(vmViewId)
+ .build();
if (InternalDataSerializer.getVersionForDataStream(in).compareTo(Version.GFE_90) == 0) {
- netMbr.readAdditionalData(in);
+ memberData.readAdditionalData(in);
}
}
@Override
public void writeEssentialData(DataOutput out) throws IOException {
- Assert.assertTrue(netMbr.getVmKind() > 0);
+ Assert.assertTrue(memberData.getVmKind() > 0);
DataSerializer.writeInetAddress(getInetAddress(), out);
out.writeInt(getPort());
int flags = 0;
- if (netMbr.isNetworkPartitionDetectionEnabled())
+ if (memberData.isNetworkPartitionDetectionEnabled())
flags |= NPD_ENABLED_BIT;
- if (netMbr.preferredForCoordinator())
+ if (memberData.preferredForCoordinator())
flags |= COORD_ENABLED_BIT;
flags |= PARTIAL_ID_BIT;
out.writeByte((byte) (flags & 0xff));
// out.writeInt(dcPort);
- byte vmKind = netMbr.getVmKind();
+ byte vmKind = memberData.getVmKind();
out.writeByte(vmKind);
if (vmKind == ClusterDistributionManager.LONER_DM_TYPE) {
DataSerializer.writeString(this.uniqueTag, out);
} else { // added in 6.5 for unique identifiers in P2P
- DataSerializer.writeString(String.valueOf(netMbr.getVmViewId()), out);
+ DataSerializer.writeString(String.valueOf(memberData.getVmViewId()), out);
}
// write name last to fix bug 45160
- DataSerializer.writeString(netMbr.getName(), out);
+ DataSerializer.writeString(memberData.getName(), out);
Version outputVersion = InternalDataSerializer.getVersionForDataStream(out);
if (0 <= outputVersion.compareTo(Version.GFE_90)
&& outputVersion.compareTo(Version.GEODE_1_1_0) < 0) {
- netMbr.writeAdditionalData(out);
+ memberData.writeAdditionalData(out);
}
}
@@ -1154,19 +1183,29 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
* information to help form a unique ID
*/
public void setPort(int p) {
- assert netMbr.getVmKind() == ClusterDistributionManager.LONER_DM_TYPE;
- this.netMbr.setPort(p);
+ assert memberData.getVmKind() == ClusterDistributionManager.LONER_DM_TYPE;
+ this.memberData.setPort(p);
cachedToString = null;
}
@Override
+ public MemberData getMemberData() {
+ return memberData;
+ }
+
+ @Override
+ public String getHostName() {
+ return memberData.getHostName();
+ }
+
+ @Override
public String getHost() {
- return this.netMbr.getInetAddress().getCanonicalHostName();
+ return this.memberData.getInetAddress().getCanonicalHostName();
}
@Override
public int getProcessId() {
- return netMbr.getProcessId();
+ return memberData.getProcessId();
}
@Override
@@ -1180,7 +1219,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
addFixedToString(sb);
// add version if not current
- short version = netMbr.getVersionOrdinal();
+ short version = memberData.getVersionOrdinal();
if (version != Version.CURRENT.ordinal()) {
sb.append("(version:").append(Version.toString(version)).append(')');
}
@@ -1190,7 +1229,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
public void setVersionObjectForTest(Version v) {
this.versionObj = v;
- netMbr.setVersion(v);
+ memberData.setVersion(v);
}
public Version getVersionObject() {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberAttributes.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberAttributes.java
deleted file mode 100755
index bbca5a3..0000000
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MemberAttributes.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.distributed.internal.membership;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.StringTokenizer;
-
-import org.apache.geode.annotations.Immutable;
-import org.apache.geode.distributed.DurableClientAttributes;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
-
-/**
- * The attributes of a distributed member. This is largely deprecated as GMSMember holds all of this
- * information.
- *
- * @since GemFire 5.0
- */
-public class MemberAttributes {
-
- @Immutable
- public static final MemberAttributes INVALID =
- new MemberAttributes(-1, -1, -1, -1, null, null, null);
- @Immutable
- public static final MemberAttributes DEFAULT =
- new MemberAttributes(-1, -1, ClusterDistributionManager.NORMAL_DM_TYPE, -1, null, null, null);
-
- private int dcPort;
- private int vmPid;
- private int vmKind;
- private int vmViewId;
- private String name;
- private String[] groups;
- private DurableClientAttributes durableClientAttributes;
-
- /** Constructs new MemberAttributes from parameters. */
- public MemberAttributes(int dcPort, int vmPid, int vmKind, int vmViewId, String p_name,
- String[] p_groups, DurableClientAttributes durableClientAttributes) {
- String[] l_groups = p_groups;
- this.dcPort = dcPort;
- this.vmPid = vmPid;
- this.vmKind = vmKind;
- this.vmViewId = vmViewId;
- if (l_groups == null) {
- l_groups = new String[0];
- }
- this.groups = l_groups;
- if (p_name == null) {
- this.name = "";
- } else {
- this.name = p_name;
- }
- this.durableClientAttributes = durableClientAttributes;
- }
-
- /** Returns direct channel port. */
- public int getPort() {
- return this.dcPort;
- }
-
- /** Returns VM PID. */
- public int getVmPid() {
- return this.vmPid;
- }
-
- /** Returns VM Kind (enumerated constants in DistributionManager). */
- public int getVmKind() {
- return this.vmKind;
- }
-
- /** Returns the name of the member. */
- public String getName() {
- return this.name;
- }
-
- /** Returns the groups of the member. */
- public String[] getGroups() {
- return this.groups;
- }
-
- /** Returns the durable client attributes. */
- public DurableClientAttributes getDurableClientAttributes() {
- return this.durableClientAttributes;
- }
-
- /** Parses comma-separated-roles/groups into array of groups (strings). */
- public static String[] parseGroups(String csvRoles, String csvGroups) {
- List<String> groups = new ArrayList<String>();
- parseCsv(groups, csvRoles);
- parseCsv(groups, csvGroups);
- return (String[]) groups.toArray(new String[groups.size()]);
- }
-
-
- private static void parseCsv(List<String> groups, String csv) {
- if (csv == null || csv.length() == 0) {
- return;
- }
- StringTokenizer st = new StringTokenizer(csv, ",");
- while (st.hasMoreTokens()) {
- String groupName = st.nextToken().trim();
- if (!groups.contains(groupName)) { // only add each group once
- groups.add(groupName);
- }
- }
- }
-
- /**
- * @return the membership view number in which this member was born
- */
- public int getVmViewId() {
- return this.vmViewId;
- }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipView.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipView.java
index 52f0122..475980c 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipView.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/MembershipView.java
@@ -183,7 +183,7 @@ public class MembershipView {
public InternalDistributedMember getCoordinator() {
for (InternalDistributedMember addr : members) {
- if (addr.getNetMember().preferredForCoordinator()) {
+ if (addr.getMemberData().preferredForCoordinator()) {
return addr;
}
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetMember.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetMember.java
deleted file mode 100755
index 7be9250..0000000
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetMember.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.distributed.internal.membership;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.net.InetAddress;
-
-import org.apache.geode.distributed.DurableClientAttributes;
-import org.apache.geode.internal.serialization.Version;
-
-/**
- * This is the SPI for the basic element of membership provided in Geode.
- *
- *
- */
-public interface NetMember extends Comparable<NetMember> {
-
- InetAddress getInetAddress();
-
- int getPort();
-
- void setPort(int p);
-
- short getVersionOrdinal();
-
- /**
- * return a flag stating whether the member has network partition detection enabled
- *
- * @since GemFire 5.6
- */
- boolean isNetworkPartitionDetectionEnabled();
-
- void setNetworkPartitionDetectionEnabled(boolean enabled);
-
- /**
- * return a flag stating whether the member can be the membership coordinator
- *
- * @since GemFire 5.6
- */
- boolean preferredForCoordinator();
-
- /**
- * Set whether this member ID is preferred for coordinator. This is mostly useful for unit tests
- * because it does not distribute this status to other members in the distributed system.
- *
- */
- void setPreferredForCoordinator(boolean preferred);
-
- byte getMemberWeight();
-
- void setVersion(Version v);
-
- int getProcessId();
-
- void setProcessId(int id);
-
- byte getVmKind();
-
- void setVmKind(int kind);
-
- int getVmViewId();
-
- void setVmViewId(int id);
-
- int getDirectPort();
-
- void setDirectPort(int port);
-
- String getName();
-
- void setName(String name);
-
- DurableClientAttributes getDurableClientAttributes();
-
- void setDurableClientAttributes(DurableClientAttributes attributes);
-
- String[] getGroups();
-
- void setGroups(String[] groups);
-
- /** whether this NetMember has additional data to be serialized as part of a DistributedMember */
- boolean hasAdditionalData();
-
- /** write identity information not known by DistributedMember instances */
- void writeAdditionalData(DataOutput out) throws IOException;
-
- /** read identity information not known by DistributedMember instances */
- void readAdditionalData(DataInput in) throws ClassNotFoundException, IOException;
-
- /** compare data that is not known to DistributedMember instances */
- int compareAdditionalData(NetMember other);
-
- void setDurableTimeout(int newValue);
-
- void setHostName(String hostName);
-
- String getHostName();
-
- /** is this a partial ID created without full identifier information? */
- boolean isPartial();
-}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetMemberFactory.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetMemberFactory.java
deleted file mode 100644
index c60f31f..0000000
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetMemberFactory.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.distributed.internal.membership;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-
-import org.apache.geode.distributed.DurableClientAttributes;
-import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
-import org.apache.geode.internal.net.SocketCreator;
-import org.apache.geode.internal.serialization.Version;
-
-public class NetMemberFactory {
- /**
- * Return a new NetMember, possibly for a different host
- *
- * @param host the name of the host for the specified NetMember, the current host (hopefully) if
- * there are any problems.
- * @param port the membership port
- * @param splitBrainEnabled whether the member has this feature enabled
- * @param canBeCoordinator whether the member can be membership coordinator
- * @param payload the payload for this member
- * @return the new NetMember
- */
- public static NetMember newNetMember(InetAddress host, String hostName, int port,
- boolean splitBrainEnabled,
- boolean canBeCoordinator, short version,
- MemberAttributes payload) {
- DurableClientAttributes durableClientAttributes = payload.getDurableClientAttributes();
- String durableId = null;
- int durableTimeout = 0;
- if (durableClientAttributes != null) {
- durableId = durableClientAttributes.getId();
- durableTimeout = durableClientAttributes.getTimeout();
- }
- return new GMSMemberAdapter(
- new GMSMember(host, hostName, port, payload.getVmPid(), (byte) payload.getVmKind(),
- payload.getPort(), payload.getVmViewId(), payload.getName(), payload.getGroups(),
- durableId, durableTimeout, splitBrainEnabled, canBeCoordinator, version, 0, 0));
- }
-
- /**
- * Return a new NetMember representing current host
- *
- * @param host an InetAddress referring to the current host
- * @param port the membership port being used
- * @return the new NetMember
- */
- public static NetMember newNetMember(InetAddress host, int port) {
- DurableClientAttributes durableClientAttributes =
- MemberAttributes.DEFAULT.getDurableClientAttributes();
- String durableId = null;
- int durableTimeout = 0;
- if (durableClientAttributes != null) {
- durableId = durableClientAttributes.getId();
- durableTimeout = durableClientAttributes.getTimeout();
- }
- return new GMSMemberAdapter(new GMSMember(host, host.getHostName(), port,
- MemberAttributes.DEFAULT.getVmPid(), (byte) MemberAttributes.DEFAULT
- .getVmKind(),
- MemberAttributes.DEFAULT.getPort(), MemberAttributes.DEFAULT.getVmViewId(),
- MemberAttributes.DEFAULT
- .getName(),
- MemberAttributes.DEFAULT.getGroups(),
- durableId, durableTimeout, false, true, Version.CURRENT_ORDINAL, 0, 0));
- }
-
- /**
- * Return a new NetMember representing current host
- *
- * @param host a String referring to the current host
- * @param port the membership port being used
- * @return the new member
- */
- public static NetMember newNetMember(String host, int port) {
- InetAddress inetAddr = null;
- try {
- inetAddr = SocketCreator.getLocalHost();
- } catch (UnknownHostException e2) {
- throw new RuntimeException("Unable to create an identifier for testing for " + host, e2);
- }
- DurableClientAttributes durableClientAttributes =
- MemberAttributes.DEFAULT.getDurableClientAttributes();
- String durableId = null;
- int durableTimeout = 0;
- if (durableClientAttributes != null) {
- durableId = durableClientAttributes.getId();
- durableTimeout = durableClientAttributes.getTimeout();
- }
- return new GMSMemberAdapter(new GMSMember(inetAddr, inetAddr.getHostName(),
- port, MemberAttributes.DEFAULT.getVmPid(), (byte) MemberAttributes.DEFAULT.getVmKind(),
- MemberAttributes.DEFAULT.getPort(), MemberAttributes.DEFAULT.getVmViewId(),
- MemberAttributes.DEFAULT
- .getName(),
- MemberAttributes.DEFAULT.getGroups(),
- durableId, durableTimeout, false, true, Version.CURRENT_ORDINAL, 0, 0));
- }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMemberAdapter.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMemberAdapter.java
deleted file mode 100644
index 01d56ec..0000000
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMemberAdapter.java
+++ /dev/null
@@ -1,230 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.distributed.internal.membership.adapter;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.net.InetAddress;
-
-import org.apache.geode.distributed.DurableClientAttributes;
-import org.apache.geode.distributed.internal.membership.NetMember;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
-import org.apache.geode.internal.serialization.Version;
-
-/**
- * GMSMemberAdapter implements the NetMember interface required by InternalDistributedMember
- * to supply it with membership details. GMSMemberAdapter delegates to a GMSMember, which
- * is the identifier class for group membership services.
- */
-public class GMSMemberAdapter implements NetMember {
-
- private GMSMember gmsMember;
- private DurableClientAttributes durableClientAttributes;
-
- public GMSMemberAdapter(GMSMember gmsMember) {
- this.gmsMember = gmsMember;
- String durableId = gmsMember.getDurableId();
- if (durableId != null) {
- this.durableClientAttributes =
- new DurableClientAttributes(durableId, gmsMember.getDurableTimeout());
- }
- }
-
- public GMSMember getGmsMember() {
- return gmsMember;
- }
-
- @Override
- public InetAddress getInetAddress() {
- return gmsMember.getInetAddress();
- }
-
- @Override
- public int getPort() {
- return gmsMember.getPort();
- }
-
- @Override
- public void setPort(int p) {
- gmsMember.setPort(p);
- }
-
- @Override
- public short getVersionOrdinal() {
- return gmsMember.getVersionOrdinal();
- }
-
- @Override
- public boolean isNetworkPartitionDetectionEnabled() {
- return gmsMember.isNetworkPartitionDetectionEnabled();
- }
-
- @Override
- public void setNetworkPartitionDetectionEnabled(boolean enabled) {
- gmsMember.setNetworkPartitionDetectionEnabled(enabled);
- }
-
- @Override
- public boolean preferredForCoordinator() {
- return gmsMember.preferredForCoordinator();
- }
-
- @Override
- public void setPreferredForCoordinator(boolean preferred) {
- gmsMember.setPreferredForCoordinator(preferred);
- }
-
- @Override
- public byte getMemberWeight() {
- return gmsMember.getMemberWeight();
- }
-
- @Override
- public void setVersion(Version v) {
- gmsMember.setVersion(v);
- }
-
- @Override
- public int getProcessId() {
- return gmsMember.getProcessId();
- }
-
- @Override
- public void setProcessId(int id) {
- gmsMember.setProcessId(id);
- }
-
- @Override
- public byte getVmKind() {
- return gmsMember.getVmKind();
- }
-
- @Override
- public void setVmKind(int kind) {
- gmsMember.setVmKind(kind);
- }
-
- @Override
- public int getVmViewId() {
- return gmsMember.getVmViewId();
- }
-
- @Override
- public void setVmViewId(int id) {
- gmsMember.setVmViewId(id);
- }
-
- @Override
- public int getDirectPort() {
- return gmsMember.getDirectPort();
- }
-
- @Override
- public void setDirectPort(int port) {
- gmsMember.setDirectPort(port);
- }
-
- @Override
- public String getName() {
- return gmsMember.getName();
- }
-
- @Override
- public void setName(String name) {
- gmsMember.setName(name);
- }
-
- @Override
- public DurableClientAttributes getDurableClientAttributes() {
- return this.durableClientAttributes;
- }
-
- @Override
- public void setDurableTimeout(int newValue) {
- if (durableClientAttributes != null) {
- durableClientAttributes.updateTimeout(newValue);
- gmsMember.setDurableTimeout(newValue);
- }
- }
-
- @Override
- public void setHostName(String hostName) {
- gmsMember.setHostName(hostName);
- }
-
- @Override
- public String getHostName() {
- return gmsMember.getHostName();
- }
-
- @Override
- public boolean isPartial() {
- return gmsMember.isPartial();
- }
-
- @Override
- public void setDurableClientAttributes(DurableClientAttributes attributes) {
- durableClientAttributes = attributes;
- if (attributes != null) {
- gmsMember.setDurableId(attributes.getId());
- gmsMember.setDurableTimeout(attributes.getTimeout());
- }
- }
-
- @Override
- public String[] getGroups() {
- return gmsMember.getGroups();
- }
-
- @Override
- public void setGroups(String[] groups) {
- gmsMember.setGroups(groups);
- }
-
- @Override
- public boolean hasAdditionalData() {
- return gmsMember.hasAdditionalData();
- }
-
- @Override
- public void writeAdditionalData(DataOutput out) throws IOException {
- gmsMember.writeAdditionalData(out);
- }
-
- @Override
- public void readAdditionalData(DataInput in) throws ClassNotFoundException, IOException {
- gmsMember.readAdditionalData(in);
- }
-
- @Override
- public int compareAdditionalData(NetMember other) {
- return gmsMember.compareAdditionalData(((GMSMemberAdapter) other).getGmsMember());
- }
-
- @Override
- public int compareTo(NetMember o) {
- return gmsMember.compareTo(((GMSMemberAdapter) o).getGmsMember());
- }
-
- @Override
- public String toString() {
- return "adapter(" + gmsMember + ")";
- }
-
- public void setGmsMember(GMSMember canonicalID) {
- gmsMember = canonicalID;
- }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManager.java
index 9e7e3b3..7bd7256 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMembershipManager.java
@@ -72,10 +72,11 @@ import org.apache.geode.distributed.internal.membership.MembershipManager;
import org.apache.geode.distributed.internal.membership.MembershipTestHook;
import org.apache.geode.distributed.internal.membership.MembershipView;
import org.apache.geode.distributed.internal.membership.QuorumChecker;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMemberData;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
import org.apache.geode.distributed.internal.membership.gms.SuspectMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipListener;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipStatistics;
@@ -417,7 +418,7 @@ public class GMSMembershipManager implements MembershipManager {
public void messageReceived(DistributionMessage msg) {
// bug 36851 - notify failure detection that we've had contact from a member
services.getHealthMonitor()
- .contactedBy(((GMSMemberAdapter) msg.getSender().getNetMember()).getGmsMember());
+ .contactedBy(msg.getSender());
handleOrDeferMessage(msg);
}
@@ -481,7 +482,7 @@ public class GMSMembershipManager implements MembershipManager {
InternalDistributedMember m = newView.getMembers().get(i);
// Once a member has been seen via a view, remove them from the
- // newborn set. Replace the netmember of the surpriseMember ID
+ // newborn set. Replace the member data of the surpriseMember ID
// in case it was a partial ID and is being retained by DistributionManager
// or some other object
boolean wasSurprise = surpriseMembers.containsKey(m);
@@ -490,7 +491,7 @@ public class GMSMembershipManager implements MembershipManager {
surpriseMembers.entrySet().iterator(); iterator.hasNext();) {
Entry<InternalDistributedMember, Long> entry = iterator.next();
if (entry.getKey().equals(m)) {
- entry.getKey().setNetMember(m.getNetMember());
+ entry.getKey().setMemberData((GMSMemberData) m.getMemberData());
iterator.remove();
break;
}
@@ -708,14 +709,13 @@ public class GMSMembershipManager implements MembershipManager {
return result;
}
- private MembershipView createGeodeView(GMSMember gmsCreator, int viewId,
- List<GMSMember> gmsMembers,
- Set<GMSMember> gmsShutdowns, Set<GMSMember> gmsCrashes) {
- InternalDistributedMember geodeCreator =
- new InternalDistributedMember(new GMSMemberAdapter(gmsCreator));
+ private MembershipView createGeodeView(MemberIdentifier gmsCreator, int viewId,
+ List<MemberIdentifier> gmsMembers,
+ Set<MemberIdentifier> gmsShutdowns, Set<MemberIdentifier> gmsCrashes) {
+ InternalDistributedMember geodeCreator = (InternalDistributedMember) gmsCreator;
List<InternalDistributedMember> geodeMembers = new ArrayList<>(gmsMembers.size());
- for (GMSMember member : gmsMembers) {
- geodeMembers.add(new InternalDistributedMember(new GMSMemberAdapter(member)));
+ for (MemberIdentifier member : gmsMembers) {
+ geodeMembers.add((InternalDistributedMember) member);
}
Set<InternalDistributedMember> geodeShutdownMembers =
gmsMemberCollectionToInternalDistributedMemberSet(gmsShutdowns);
@@ -726,16 +726,16 @@ public class GMSMembershipManager implements MembershipManager {
}
private Set<InternalDistributedMember> gmsMemberCollectionToInternalDistributedMemberSet(
- Collection<GMSMember> gmsMembers) {
+ Collection<MemberIdentifier> gmsMembers) {
if (gmsMembers.size() == 0) {
return Collections.emptySet();
} else if (gmsMembers.size() == 1) {
return Collections.singleton(
- new InternalDistributedMember(new GMSMemberAdapter(gmsMembers.iterator().next())));
+ (InternalDistributedMember) gmsMembers.iterator().next());
} else {
Set<InternalDistributedMember> idmMembers = new HashSet<>(gmsMembers.size());
- for (GMSMember member : gmsMembers) {
- idmMembers.add(new InternalDistributedMember(new GMSMemberAdapter((member))));
+ for (MemberIdentifier member : gmsMembers) {
+ idmMembers.add((InternalDistributedMember) member);
}
return idmMembers;
}
@@ -743,16 +743,16 @@ public class GMSMembershipManager implements MembershipManager {
private List<InternalDistributedMember> gmsMemberListToInternalDistributedMemberList(
- List<GMSMember> gmsMembers) {
+ List<MemberIdentifier> gmsMembers) {
if (gmsMembers.size() == 0) {
return Collections.emptyList();
} else if (gmsMembers.size() == 1) {
return Collections
- .singletonList(new InternalDistributedMember(new GMSMemberAdapter(gmsMembers.get(0))));
+ .singletonList((InternalDistributedMember) gmsMembers.get(0));
} else {
List<InternalDistributedMember> idmMembers = new ArrayList<>(gmsMembers.size());
- for (GMSMember member : gmsMembers) {
- idmMembers.add(new InternalDistributedMember(new GMSMemberAdapter((member))));
+ for (MemberIdentifier member : gmsMembers) {
+ idmMembers.add((InternalDistributedMember) member);
}
return idmMembers;
}
@@ -1072,10 +1072,10 @@ public class GMSMembershipManager implements MembershipManager {
*/
public void replacePartialIdentifierInMessage(DistributionMessage msg) {
InternalDistributedMember sender = msg.getSender();
- GMSMember oldID = ((GMSMemberAdapter) sender.getNetMember()).getGmsMember();
- GMSMember newID = this.services.getJoinLeave().getMemberID(oldID);
+ MemberIdentifier oldID = sender;
+ MemberIdentifier newID = this.services.getJoinLeave().getMemberID(oldID);
if (newID != null && newID != oldID) {
- sender.setNetMember(new GMSMemberAdapter(newID));
+ sender.setMemberData(newID.getMemberData());
sender.setIsPartial(false);
} else {
// the DM's view also has surprise members, so let's check it as well
@@ -1121,8 +1121,8 @@ public class GMSMembershipManager implements MembershipManager {
}
}
- private InternalDistributedMember gmsMemberToDMember(GMSMember gmsMember) {
- return new InternalDistributedMember(new GMSMemberAdapter(gmsMember));
+ private InternalDistributedMember gmsMemberToDMember(MemberIdentifier gmsMember) {
+ return (InternalDistributedMember) gmsMember;
}
/**
@@ -1409,10 +1409,8 @@ public class GMSMembershipManager implements MembershipManager {
synchronized (this.shutdownMembers) {
this.shutdownMembers.put(id, id);
services.getHealthMonitor()
- .memberShutdown(
- ((GMSMemberAdapter) ((InternalDistributedMember) id).getNetMember()).getGmsMember(),
- reason);
- services.getJoinLeave().memberShutdown(getGMSMember((InternalDistributedMember) id), reason);
+ .memberShutdown((MemberIdentifier) id, reason);
+ services.getJoinLeave().memberShutdown((MemberIdentifier) id, reason);
}
}
@@ -1483,7 +1481,7 @@ public class GMSMembershipManager implements MembershipManager {
logger.warn("Membership: requesting removal of {}. Reason={}",
new Object[] {mbr, reason});
try {
- services.getJoinLeave().remove(getGMSMember((InternalDistributedMember) mbr), reason);
+ services.getJoinLeave().remove((MemberIdentifier) mbr, reason);
} catch (RuntimeException e) {
Throwable problem = e;
if (services.getShutdownCause() != null) {
@@ -1541,7 +1539,7 @@ public class GMSMembershipManager implements MembershipManager {
public boolean verifyMember(DistributedMember mbr, String reason) {
return mbr != null && memberExists(mbr)
&& this.services.getHealthMonitor()
- .checkIfAvailable(getGMSMember((InternalDistributedMember) mbr), reason, false);
+ .checkIfAvailable((MemberIdentifier) mbr, reason, false);
}
/**
@@ -1624,7 +1622,7 @@ public class GMSMembershipManager implements MembershipManager {
InternalDistributedMember member = (InternalDistributedMember) it_mem.next();
Throwable th = (Throwable) it_causes.next();
- if (!view.contains(getGMSMember(member)) || (th instanceof ShunnedMemberException)) {
+ if (!view.contains(member) || (th instanceof ShunnedMemberException)) {
continue;
}
logger.fatal(String.format("Failed to send message <%s> to member <%s> view, %s",
@@ -1645,13 +1643,6 @@ public class GMSMembershipManager implements MembershipManager {
return null;
}
- /**
- * retrieve the GMS member ID held in a Geode InternalDistributedMember
- */
- private GMSMember getGMSMember(InternalDistributedMember member) {
- return ((GMSMemberAdapter) member.getNetMember()).getGmsMember();
- }
-
/*
* (non-Javadoc)
*
@@ -1751,7 +1742,7 @@ public class GMSMembershipManager implements MembershipManager {
if (useMcast || tcpDisabled || sendViaMessenger) {
checkAddressesForUUIDs(destinations);
- Set<GMSMember> failures = services.getMessenger().send(new GMSMessageAdapter(msg));
+ Set<MemberIdentifier> failures = services.getMessenger().send(new GMSMessageAdapter(msg));
if (failures == null || failures.size() == 0) {
return Collections.emptySet();
}
@@ -1771,12 +1762,10 @@ public class GMSMembershipManager implements MembershipManager {
void checkAddressesForUUIDs(InternalDistributedMember[] addresses) {
GMSMembershipView view = services.getJoinLeave().getView();
for (int i = 0; i < addresses.length; i++) {
- InternalDistributedMember m = addresses[i];
- if (m != null) {
- GMSMemberAdapter adapter = (GMSMemberAdapter) m.getNetMember();
- GMSMember id = adapter.getGmsMember();
- if (!id.hasUUID()) {
- adapter.setGmsMember(view.getCanonicalID(id));
+ InternalDistributedMember id = addresses[i];
+ if (id != null) {
+ if (!id.getMemberData().hasUUID()) {
+ id.setMemberData(view.getCanonicalID(id).getMemberData());
}
}
}
@@ -2063,7 +2052,7 @@ public class GMSMembershipManager implements MembershipManager {
if (dc != null) {
dc.getChannelStates(member, result);
}
- services.getMessenger().getMessageState(getGMSMember((InternalDistributedMember) member),
+ services.getMessenger().getMessageState((MemberIdentifier) member,
result,
includeMulticast);
return result;
@@ -2078,8 +2067,7 @@ public class GMSMembershipManager implements MembershipManager {
if (dc != null) {
dc.waitForChannelState(otherMember, state);
}
- services.getMessenger().waitForMessageState(
- getGMSMember((InternalDistributedMember) otherMember), state);
+ services.getMessenger().waitForMessageState((MemberIdentifier) otherMember, state);
if (services.getConfig().isMulticastEnabled()
&& !services.getConfig().getDisableTcp()) {
@@ -2465,7 +2453,7 @@ public class GMSMembershipManager implements MembershipManager {
directChannel = new DirectChannel(GMSMembershipManager.this, dcReceiver, dm);
dcPort = directChannel.getPort();
}
- services.getMessenger().getMemberID().setDirectPort(dcPort);
+ services.getMessenger().getMemberID().getMemberData().setDirectChannelPort(dcPort);
}
/* Service interface */
@@ -2564,8 +2552,7 @@ public class GMSMembershipManager implements MembershipManager {
}
GMSMembershipManager.this.address =
- new InternalDistributedMember(
- new GMSMemberAdapter(services.getMessenger().getMemberID()));
+ (InternalDistributedMember) services.getMessenger().getMemberID();
if (directChannel != null) {
directChannel.setLocalAddr(address);
@@ -2581,8 +2568,8 @@ public class GMSMembershipManager implements MembershipManager {
}
@Override
- public void memberSuspected(GMSMember initiator,
- GMSMember suspect, String reason) {
+ public void memberSuspected(MemberIdentifier initiator,
+ MemberIdentifier suspect, String reason) {
SuspectMember s = new SuspectMember(initiator, suspect, reason);
handleOrDeferSuspect(s);
}
@@ -2634,7 +2621,7 @@ public class GMSMembershipManager implements MembershipManager {
/** this is invoked by JoinLeave when there is a loss of quorum in the membership system */
@Override
- public void quorumLost(Collection<GMSMember> failures, GMSMembershipView view) {
+ public void quorumLost(Collection<MemberIdentifier> failures, GMSMembershipView view) {
// notify of quorum loss if split-brain detection is enabled (meaning we'll shut down) or
// if the loss is more than one member
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMessageAdapter.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMessageAdapter.java
index fdfcf6c..89016ed 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMessageAdapter.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/GMSMessageAdapter.java
@@ -17,19 +17,18 @@ package org.apache.geode.distributed.internal.membership.adapter;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
-import java.util.stream.Collectors;
import org.apache.geode.distributed.internal.DistributionMessage;
import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
import org.apache.geode.distributed.internal.OperationExecutors;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.interfaces.GMSMessage;
import org.apache.geode.internal.cache.DirectReplyMessage;
import org.apache.geode.internal.serialization.DataSerializableFixedID;
/**
- * GMSMessageAdapter wraps a Geode DistributionMessage to be sent via the GMS Messenger
+ * GMSMessageAdapter wraps a Geode DistributionMessage to be sent via the GMS Messenger (JGroups)
*/
public class GMSMessageAdapter implements GMSMessage {
@@ -40,15 +39,14 @@ public class GMSMessageAdapter implements GMSMessage {
}
@Override
- public void setRecipient(GMSMember member) {
- geodeMessage.setRecipient(new InternalDistributedMember(new GMSMemberAdapter(member)));
+ public void setRecipient(MemberIdentifier member) {
+ geodeMessage.setRecipient((InternalDistributedMember) member);
}
@Override
- public void setRecipients(List<GMSMember> recipients) {
- geodeMessage.setRecipients(
- recipients.stream().map(GMSMemberAdapter::new).map(InternalDistributedMember::new).collect(
- Collectors.toList()));
+ public void setRecipients(List<MemberIdentifier> recipients) {
+ throw new UnsupportedOperationException(
+ "setting recipients is not allowed on a message wrapper");
}
@Override
@@ -65,16 +63,13 @@ public class GMSMessageAdapter implements GMSMessage {
}
@Override
- public List<GMSMember> getRecipients() {
+ public List<MemberIdentifier> getRecipients() {
InternalDistributedMember[] recipients = geodeMessage.getRecipients();
if (recipients == null
|| recipients.length == 1 && recipients[0] == DistributionMessage.ALL_RECIPIENTS) {
return Collections.singletonList(null);
}
- return Arrays.asList(recipients).stream()
- .map(recipient -> (GMSMemberAdapter) recipient.getNetMember())
- .map(GMSMemberAdapter::getGmsMember).collect(
- Collectors.toList());
+ return Arrays.asList(recipients);
}
@Override
@@ -98,13 +93,13 @@ public class GMSMessageAdapter implements GMSMessage {
}
@Override
- public void setSender(GMSMember sender) {
- geodeMessage.setSender(new InternalDistributedMember(new GMSMemberAdapter(sender)));
+ public void setSender(MemberIdentifier sender) {
+ geodeMessage.setSender((InternalDistributedMember) sender);
}
@Override
- public GMSMember getSender() {
- return ((GMSMemberAdapter) geodeMessage.getSender().getNetMember()).getGmsMember();
+ public MemberIdentifier getSender() {
+ return geodeMessage.getSender();
}
@Override
@@ -122,7 +117,7 @@ public class GMSMessageAdapter implements GMSMessage {
return geodeMessage.toString();
}
- public DataSerializableFixedID getGeodeMessage() {
+ DataSerializableFixedID getGeodeMessage() {
return geodeMessage;
}
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/auth/GMSAuthenticator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/auth/GMSAuthenticator.java
index 3f35e38..f357f2f 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/auth/GMSAuthenticator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/adapter/auth/GMSAuthenticator.java
@@ -24,9 +24,8 @@ import org.apache.commons.lang3.StringUtils;
import org.apache.geode.LogWriter;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.api.Authenticator;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.cache.tier.sockets.Handshake;
import org.apache.geode.internal.security.CallbackInstantiator;
import org.apache.geode.internal.security.SecurityService;
@@ -62,14 +61,14 @@ public class GMSAuthenticator implements Authenticator {
* failure message
*/
@Override
- public String authenticate(GMSMember member, Properties credentials) {
+ public String authenticate(MemberIdentifier member, Properties credentials) {
return authenticate(member, credentials, this.securityProps);
}
/**
* Method is package protected to be used in testing.
*/
- String authenticate(GMSMember member, Properties credentials, Properties secProps) {
+ String authenticate(MemberIdentifier member, Properties credentials, Properties secProps) {
// For older systems, locator might be started without cache, so secureService may not be
// initialized here. We need to check if the passed in secProps has peer authenticator or not at
@@ -105,7 +104,7 @@ public class GMSAuthenticator implements Authenticator {
/**
* Method is package protected to be used in testing.
*/
- Principal invokeAuthenticator(Properties securityProps, GMSMember member,
+ Principal invokeAuthenticator(Properties securityProps, MemberIdentifier member,
Properties credentials) throws AuthenticationFailedException {
String authMethod = securityProps.getProperty(SECURITY_PEER_AUTHENTICATOR);
org.apache.geode.security.Authenticator auth = null;
@@ -116,8 +115,7 @@ public class GMSAuthenticator implements Authenticator {
// this.securityProps contains security-ldap-basedn but security-ldap-baseDomainName is
// expected
auth.init(this.securityProps, logWriter, securityLogWriter);
- return auth.authenticate(credentials,
- new InternalDistributedMember(new GMSMemberAdapter(member)));
+ return auth.authenticate(credentials, (InternalDistributedMember) member);
} catch (GemFireSecurityException gse) {
throw gse;
@@ -139,7 +137,7 @@ public class GMSAuthenticator implements Authenticator {
* @return the credentials
*/
@Override
- public Properties getCredentials(GMSMember member) {
+ public Properties getCredentials(MemberIdentifier member) {
try {
return getCredentials(member, securityProps);
@@ -155,10 +153,10 @@ public class GMSAuthenticator implements Authenticator {
/**
* For testing only.
*/
- Properties getCredentials(GMSMember member, Properties secProps) {
+ Properties getCredentials(MemberIdentifier member, Properties secProps) {
String authMethod = secProps.getProperty(SECURITY_PEER_AUTH_INIT);
return Handshake.getCredentials(authMethod, secProps,
- new InternalDistributedMember(new GMSMemberAdapter(member)),
+ (InternalDistributedMember) member,
true,
logWriter,
securityLogWriter);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMember.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberData.java
old mode 100755
new mode 100644
similarity index 74%
rename from geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMember.java
rename to geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberData.java
index d51fd96..3e04317
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMember.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberData.java
@@ -23,8 +23,8 @@ import java.net.InetAddress;
import org.jgroups.util.UUID;
import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberData;
import org.apache.geode.internal.net.SocketCreator;
-import org.apache.geode.internal.serialization.DataSerializableFixedID;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
import org.apache.geode.internal.serialization.StaticSerialization;
@@ -33,7 +33,7 @@ import org.apache.geode.internal.serialization.Version;
/**
* GMSMember is the membership identifier class for Group Membership Services.
*/
-public class GMSMember implements DataSerializableFixedID {
+public class GMSMemberData implements MemberData, Comparable<GMSMemberData> {
/** The type for regular members */
public static final int NORMAL_DM_TYPE = 10;
@@ -71,11 +71,10 @@ public class GMSMember implements DataSerializableFixedID {
return isPartial;
}
- // Used only by Externalization
- public GMSMember() {}
+ public GMSMemberData() {}
@VisibleForTesting
- public GMSMember(String localhost, int udpPort, Version version) {
+ public GMSMemberData(String localhost, int udpPort, Version version) {
this.hostName = localhost;
this.inetAddr = SocketCreator.toInetAddress(localhost);
this.udpPort = udpPort;
@@ -96,7 +95,7 @@ public class GMSMember implements DataSerializableFixedID {
* @param p the membership listening port
*/
@VisibleForTesting
- public GMSMember(String i, int p) {
+ public GMSMemberData(String i, int p) {
this(i, p, Version.getCurrentVersion());
}
@@ -112,12 +111,13 @@ public class GMSMember implements DataSerializableFixedID {
* @param msbs - most significant bytes of UUID
* @param lsbs - least significant bytes of UUID
*/
- public GMSMember(InetAddress i, String hostName, int p, int processId, byte vmKind,
+ public GMSMemberData(InetAddress i, String hostName, int p, int processId, byte vmKind,
int directPort, int vmViewId,
String name, String[] groups,
String durableId, int durableTimeout,
- boolean networkPartitionDetectionEnabled, boolean preferredForCoordinator, short version,
- long msbs, long lsbs) {
+ boolean networkPartitionDetectionEnabled, boolean preferredForCoordinator,
+ short version,
+ long msbs, long lsbs, byte memberWeight) {
this.inetAddr = i;
this.hostName = hostName;
this.udpPort = p;
@@ -134,9 +134,10 @@ public class GMSMember implements DataSerializableFixedID {
this.versionOrdinal = version;
this.uuidMSBs = msbs;
this.uuidLSBs = lsbs;
+ this.memberWeight = memberWeight;
}
- public GMSMember(InetAddress i, int p, short version, long msbs, long lsbs, int viewId) {
+ public GMSMemberData(InetAddress i, int p, short version, long msbs, long lsbs, int viewId) {
this.inetAddr = i;
this.hostName = i.getHostName();
this.udpPort = p;
@@ -148,11 +149,11 @@ public class GMSMember implements DataSerializableFixedID {
/**
- * Clone a GMSMember
+ * Clone a GMSMemberInfo
*
* @param other the member to create a copy of
*/
- public GMSMember(GMSMember other) {
+ public GMSMemberData(GMSMemberData other) {
this.hostName = other.hostName;
this.udpPort = other.udpPort;
this.preferredForCoordinator = other.preferredForCoordinator;
@@ -173,42 +174,51 @@ public class GMSMember implements DataSerializableFixedID {
}
+ @Override
public int getPort() {
return this.udpPort;
}
+ @Override
public boolean preferredForCoordinator() {
return this.preferredForCoordinator;
}
+ @Override
public void setPreferredForCoordinator(boolean preferred) {
this.preferredForCoordinator = preferred;
}
+ @Override
public String getDurableId() {
return durableId;
}
+ @Override
public int getDurableTimeout() {
return durableTimeout;
}
+ @Override
public InetAddress getInetAddress() {
return this.inetAddr;
}
+ @Override
public short getVersionOrdinal() {
return this.versionOrdinal;
}
+ @Override
public void setVersionOrdinal(short versionOrdinal) {
this.versionOrdinal = versionOrdinal;
}
+ @Override
public void setUUID(UUID u) {
this.uuidLSBs = u.getLeastSignificantBits();
this.uuidMSBs = u.getMostSignificantBits();
@@ -217,6 +227,7 @@ public class GMSMember implements DataSerializableFixedID {
/**
* return the jgroups logical address for this member, if it's been established
*/
+ @Override
public UUID getUUID() {
if (this.uuidLSBs == 0 && this.uuidMSBs == 0) {
return null;
@@ -224,10 +235,12 @@ public class GMSMember implements DataSerializableFixedID {
return new UUID(this.uuidMSBs, this.uuidLSBs);
}
+ @Override
public long getUuidMSBs() {
return this.uuidMSBs;
}
+ @Override
public long getUuidLSBs() {
return this.uuidLSBs;
}
@@ -246,11 +259,18 @@ public class GMSMember implements DataSerializableFixedID {
* compared to this Object.
*/
- public int compareTo(GMSMember o) {
+ @Override
+ public int compareTo(GMSMemberData o) {
return compareTo(o, true);
}
- public int compareTo(GMSMember o, boolean compareUUIDs) {
+ @Override
+ public int compareTo(MemberData o, boolean compareUUIDs) {
+ return compareTo(o, compareUUIDs, true);
+ }
+
+ @Override
+ public int compareTo(MemberData o, boolean compareUUIDs, boolean compareViewIds) {
if (o == this) {
return 0;
}
@@ -260,7 +280,7 @@ public class GMSMember implements DataSerializableFixedID {
"GMSMember.compareTo(): comparison between different classes");
}
byte[] myAddr = inetAddr.getAddress();
- GMSMember his = o;
+ GMSMemberData his = (GMSMemberData) o;
byte[] hisAddr = his.inetAddr.getAddress();
if (myAddr != hisAddr) {
for (int idx = 0; idx < myAddr.length; idx++) {
@@ -286,7 +306,7 @@ public class GMSMember implements DataSerializableFixedID {
// bug #41983, address of kill-9'd member is reused
// before it can be ejected from membership
- if (this.vmViewId >= 0 && his.vmViewId >= 0) {
+ if (compareViewIds && this.vmViewId >= 0 && his.vmViewId >= 0) {
if (this.vmViewId < his.vmViewId) {
result = -1;
} else if (his.vmViewId < this.vmViewId) {
@@ -308,7 +328,9 @@ public class GMSMember implements DataSerializableFixedID {
}
- public int compareAdditionalData(GMSMember his) {
+ @Override
+ public int compareAdditionalData(MemberData o) {
+ GMSMemberData his = (GMSMemberData) o;
int result = 0;
if (this.uuidMSBs != 0 && his.uuidMSBs != 0) {
if (this.uuidMSBs < his.uuidMSBs) {
@@ -327,10 +349,10 @@ public class GMSMember implements DataSerializableFixedID {
@Override
public boolean equals(Object obj) {
// GemStone fix for 29125
- if ((obj == null) || !(obj instanceof GMSMember)) {
+ if ((obj == null) || !(obj instanceof GMSMemberData)) {
return false;
}
- return compareTo((GMSMember) obj) == 0;
+ return compareTo((GMSMemberData) obj) == 0;
}
@Override
@@ -345,7 +367,7 @@ public class GMSMember implements DataSerializableFixedID {
public String toString() {
StringBuilder sb = new StringBuilder(100);
- sb.append("GMSMember[");
+ sb.append("MemberData[");
if (name != null && name.length() > 0) {
sb.append("name=").append(name);
}
@@ -360,111 +382,127 @@ public class GMSMember implements DataSerializableFixedID {
}
+ @Override
public boolean isNetworkPartitionDetectionEnabled() {
return networkPartitionDetectionEnabled;
}
+ @Override
public byte getMemberWeight() {
return memberWeight;
}
+ @Override
public InetAddress getInetAddr() {
return inetAddr;
}
+ @Override
public int getProcessId() {
return processId;
}
+ @Override
public byte getVmKind() {
return vmKind;
}
+ @Override
public int getVmViewId() {
return vmViewId;
}
+ @Override
public void setVmViewId(int id) {
this.vmViewId = id;
}
- public int getDirectPort() {
+ @Override
+ public int getDirectChannelPort() {
return directPort;
}
+ @Override
public String getName() {
return name;
}
+ @Override
public String[] getRoles() {
return groups;
}
+ @Override
public void setUdpPort(int udpPort) {
this.udpPort = udpPort;
}
+ @Override
public void setNetworkPartitionDetectionEnabled(boolean networkPartitionDetectionEnabled) {
this.networkPartitionDetectionEnabled = networkPartitionDetectionEnabled;
}
+ @Override
public void setMemberWeight(byte memberWeight) {
this.memberWeight = memberWeight;
}
+ @Override
public void setInetAddr(InetAddress inetAddr) {
this.inetAddr = inetAddr;
}
+ @Override
public void setProcessId(int processId) {
this.processId = processId;
}
+ @Override
public void setVmKind(int vmKind) {
this.vmKind = (byte) vmKind;
}
+ @Override
public void setVersion(Version v) {
this.versionOrdinal = v.ordinal();
}
- public void setBirthViewId(int birthViewId) {
- this.vmViewId = birthViewId;
- }
-
-
- public void setDirectPort(int directPort) {
+ @Override
+ public void setDirectChannelPort(int directPort) {
this.directPort = directPort;
}
+ @Override
public void setName(String name) {
this.name = name;
}
+ @Override
public String[] getGroups() {
return groups;
}
-
+ @Override
public void setGroups(String[] groups) {
this.groups = groups;
}
+ @Override
public void setPort(int p) {
this.udpPort = p;
}
@@ -472,68 +510,16 @@ public class GMSMember implements DataSerializableFixedID {
/**
* checks to see if this address has UUID information needed to send messages via JGroups
*/
+ @Override
public boolean hasUUID() {
return !(this.uuidLSBs == 0 && this.uuidMSBs == 0);
}
- @Override
- public Version[] getSerializationVersions() {
- return null;
- }
-
- @Override
- public int getDSFID() {
- return GMSMEMBER;
- }
-
static final int NPD_ENABLED_BIT = 0x01;
static final int PREFERRED_FOR_COORD_BIT = 0x02;
static final int VERSION_BIT = 0x8;
- static final int LONER_VM_TYPE = 13; // from ClusterDistributionManager
-
@Override
- public void toData(DataOutput out,
- SerializationContext context) throws IOException {
- StaticSerialization.writeInetAddress(getInetAddress(), out);
- out.writeInt(getPort());
-
- StaticSerialization.writeString(hostName, out);
-
- int flags = 0;
- if (isNetworkPartitionDetectionEnabled())
- flags |= NPD_ENABLED_BIT;
- if (preferredForCoordinator())
- flags |= PREFERRED_FOR_COORD_BIT;
- // always write product version but enable reading from older versions
- // that do not have it
- flags |= VERSION_BIT;
-
- out.writeByte((byte) (flags & 0xff));
-
- out.writeInt(getDirectPort());
- out.writeInt(getProcessId());
- int vmKind = getVmKind();
- out.writeByte(vmKind);
- StaticSerialization.writeStringArray(getGroups(), out);
-
- StaticSerialization.writeString(getName(), out);
- if (vmKind == LONER_VM_TYPE) {
- StaticSerialization.writeString("", out);
- } else { // added in 6.5 for unique identifiers in P2P
- StaticSerialization.writeString(String.valueOf(getVmViewId()), out);
- }
- StaticSerialization
- .writeString(durableId == null ? "" : durableId, out);
- out.writeInt(durableId == null ? 300 : durableTimeout);
-
- Version.writeOrdinal(out, versionOrdinal, true);
-
- if (versionOrdinal >= Version.GFE_90.ordinal()) {
- writeAdditionalData(out);
- }
- }
-
public void writeEssentialData(DataOutput out,
SerializationContext context) throws IOException {
Version.writeOrdinal(out, this.versionOrdinal, true);
@@ -556,62 +542,16 @@ public class GMSMember implements DataSerializableFixedID {
}
@Override
- public void fromData(DataInput in,
- DeserializationContext context) throws IOException, ClassNotFoundException {
- inetAddr = StaticSerialization.readInetAddress(in);
- udpPort = in.readInt();
-
- this.hostName = StaticSerialization.readString(in);
-
- int flags = in.readUnsignedByte();
- preferredForCoordinator = (flags & PREFERRED_FOR_COORD_BIT) != 0;
- this.networkPartitionDetectionEnabled = (flags & NPD_ENABLED_BIT) != 0;
-
- directPort = in.readInt();
- processId = in.readInt();
- vmKind = (byte) in.readUnsignedByte();
- groups = StaticSerialization.readStringArray(in);
- vmViewId = -1;
-
- name = StaticSerialization.readString(in);
- if (vmKind == LONER_DM_TYPE) {
- StaticSerialization.readString(in);
- } else {
- String str = StaticSerialization.readString(in);
- if (str != null) { // backward compatibility from earlier than 6.5
- vmViewId = Integer.parseInt(str);
- }
- }
-
- durableId = StaticSerialization.readString(in);
- durableTimeout = in.readInt();
-
- versionOrdinal = readVersion(flags, in, context);
-
- if (versionOrdinal >= Version.GFE_90.ordinal()) {
- readAdditionalData(in);
- }
- }
-
- private short readVersion(int flags, DataInput in,
- DeserializationContext context) throws IOException {
- if ((flags & VERSION_BIT) != 0) {
- return Version.readOrdinal(in);
- } else {
- // prior to 7.1 member IDs did not serialize their version information
- Version v = context.getSerializationVersion();
- return v.ordinal();
- }
- }
-
public String getHostName() {
return hostName;
}
+ @Override
public void setHostName(String hostName) {
this.hostName = hostName;
}
+ @Override
public void readEssentialData(DataInput in,
DeserializationContext context) throws IOException, ClassNotFoundException {
this.versionOrdinal = Version.readOrdinal(in);
@@ -637,11 +577,13 @@ public class GMSMember implements DataSerializableFixedID {
}
+ @Override
public boolean hasAdditionalData() {
return uuidMSBs != 0 || uuidLSBs != 0 || memberWeight != 0;
}
+ @Override
public void writeAdditionalData(DataOutput out) throws IOException {
out.writeLong(uuidMSBs);
out.writeLong(uuidLSBs);
@@ -649,6 +591,7 @@ public class GMSMember implements DataSerializableFixedID {
}
+ @Override
public void readAdditionalData(DataInput in) throws ClassNotFoundException, IOException {
try {
this.uuidMSBs = in.readLong();
@@ -659,50 +602,13 @@ public class GMSMember implements DataSerializableFixedID {
}
}
- private String formatUUID() {
- UUID uuid = getUUID();
- return ";uuid=" + (uuid == null ? "none" : getUUID().toStringLong());
- }
-
+ @Override
public void setDurableTimeout(int newValue) {
durableTimeout = newValue;
}
+ @Override
public void setDurableId(String id) {
durableId = id;
}
-
-
- public static class GMSMemberWrapper {
- GMSMember mbr;
-
- public GMSMemberWrapper(GMSMember m) {
- this.mbr = m;
- }
-
- public GMSMember getMbr() {
- return mbr;
- }
-
- @Override
- public int hashCode() {
- return mbr.hashCode();
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj == null || !(obj instanceof GMSMemberWrapper)) {
- return false;
- }
- GMSMember other = ((GMSMemberWrapper) obj).mbr;
- return mbr.compareTo(other) == 0;
- }
-
- @Override
- public String toString() {
- return "GMSMemberWrapper [mbr=" + mbr + "]";
- }
- }
-
-
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMembershipView.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMembershipView.java
index 570f83b..d481c73 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMembershipView.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSMembershipView.java
@@ -33,6 +33,7 @@ import java.util.stream.Collectors;
import org.apache.logging.log4j.Logger;
import org.apache.geode.annotations.Immutable;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.logging.LogService;
import org.apache.geode.internal.serialization.DataSerializableFixedID;
import org.apache.geode.internal.serialization.DeserializationContext;
@@ -41,8 +42,11 @@ import org.apache.geode.internal.serialization.StaticSerialization;
import org.apache.geode.internal.serialization.Version;
/**
- * The GMSMembershipView class represents a membership view. Note that this class is not
- * synchronized, so take
+ * The GMSMembershipView class represents a membership view For Geode this is translated
+ * into a MembershipView by GMSMembershipManager. The membership identifiers are GMSMembers,
+ * which for Geode means they are InternalDistributedMember instances.
+ *
+ * Note that this class is not synchronized, so take
* that under advisement if you decide to modify a view with add() or remove().
*
*/
@@ -50,14 +54,14 @@ public class GMSMembershipView implements DataSerializableFixedID {
private static final Logger logger = LogService.getLogger();
private int viewId;
- private List<GMSMember> members;
+ private List<MemberIdentifier> members;
// TODO this should be a List
- private final Map<GMSMember, Object> publicKeys = new ConcurrentHashMap<>();
+ private final Map<MemberIdentifier, Object> publicKeys = new ConcurrentHashMap<>();
private int[] failureDetectionPorts = new int[10];
- private Set<GMSMember> shutdownMembers;
- private Set<GMSMember> crashedMembers;
- private GMSMember creator;
- private Set<GMSMember> hashedMembers;
+ private Set<MemberIdentifier> shutdownMembers;
+ private Set<MemberIdentifier> crashedMembers;
+ private MemberIdentifier creator;
+ private Set<MemberIdentifier> hashedMembers;
private final Object membersLock = new Object();
@Immutable
public static final Random RANDOM = new Random();
@@ -73,7 +77,7 @@ public class GMSMembershipView implements DataSerializableFixedID {
Arrays.fill(failureDetectionPorts, -1);
}
- public GMSMembershipView(GMSMember creator) {
+ public GMSMembershipView(MemberIdentifier creator) {
viewId = 0;
members = new ArrayList<>(4);
members.add(creator);
@@ -84,8 +88,8 @@ public class GMSMembershipView implements DataSerializableFixedID {
Arrays.fill(failureDetectionPorts, -1);
}
- public GMSMembershipView(GMSMember creator, int viewId,
- List<GMSMember> members) {
+ public GMSMembershipView(MemberIdentifier creator, int viewId,
+ List<MemberIdentifier> members) {
this.viewId = viewId;
this.members = new ArrayList<>(members);
hashedMembers = new HashSet<>(this.members);
@@ -128,9 +132,9 @@ public class GMSMembershipView implements DataSerializableFixedID {
this.publicKeys.putAll(other.publicKeys);
}
- public GMSMembershipView(GMSMember creator, int viewId,
- List<GMSMember> mbrs, Set<GMSMember> shutdowns,
- Set<GMSMember> crashes) {
+ public GMSMembershipView(MemberIdentifier creator, int viewId,
+ List<MemberIdentifier> mbrs, Set<MemberIdentifier> shutdowns,
+ Set<MemberIdentifier> crashes) {
this.creator = creator;
this.viewId = viewId;
this.members = mbrs;
@@ -147,19 +151,19 @@ public class GMSMembershipView implements DataSerializableFixedID {
}
- public GMSMember getCreator() {
+ public MemberIdentifier getCreator() {
return this.creator;
}
- public void setCreator(GMSMember creator) {
+ public void setCreator(MemberIdentifier creator) {
this.creator = creator;
}
- public Object getPublicKey(GMSMember mbr) {
+ public Object getPublicKey(MemberIdentifier mbr) {
return publicKeys.get(mbr);
}
- public void setPublicKey(GMSMember mbr, Object key) {
+ public void setPublicKey(MemberIdentifier mbr, Object key) {
if (mbr != null && key != null) {
publicKeys.put(mbr, key);
}
@@ -181,7 +185,7 @@ public class GMSMembershipView implements DataSerializableFixedID {
return this.failureDetectionPorts;
}
- public int getFailureDetectionPort(GMSMember mbr) {
+ public int getFailureDetectionPort(MemberIdentifier mbr) {
int idx = members.indexOf(mbr);
if (idx < 0 || idx >= failureDetectionPorts.length) {
return -1;
@@ -190,7 +194,7 @@ public class GMSMembershipView implements DataSerializableFixedID {
}
- public void setFailureDetectionPort(GMSMember mbr, int port) {
+ public void setFailureDetectionPort(MemberIdentifier mbr, int port) {
int idx = members.indexOf(mbr);
if (idx < 0) {
throw new IllegalArgumentException("element not found in members list:" + mbr);
@@ -207,7 +211,7 @@ public class GMSMembershipView implements DataSerializableFixedID {
if (ports != null) {
int idx = 0;
int portsSize = ports.length;
- for (GMSMember mbr : otherView.getMembers()) {
+ for (MemberIdentifier mbr : otherView.getMembers()) {
if (contains(mbr)) {
// unit tests create views w/o failure detection ports, so we must check the length
// of the array
@@ -240,7 +244,7 @@ public class GMSMembershipView implements DataSerializableFixedID {
return this.members.get(i);
}
- public void add(GMSMember mbr) {
+ public void add(MemberIdentifier mbr) {
this.hashedMembers.add(mbr);
this.members.add(mbr);
int idx = members.size() - 1;
@@ -248,15 +252,15 @@ public class GMSMembershipView implements DataSerializableFixedID {
this.failureDetectionPorts[idx] = -1;
}
- public void addCrashedMembers(Set<GMSMember> mbr) {
+ public void addCrashedMembers(Set<MemberIdentifier> mbr) {
this.crashedMembers.addAll(mbr);
}
/**
* return members that are i this view but not the given old view
*/
- public List<GMSMember> getNewMembers(GMSMembershipView olderView) {
- List<GMSMember> result = new ArrayList<>(members);
+ public List<MemberIdentifier> getNewMembers(GMSMembershipView olderView) {
+ List<MemberIdentifier> result = new ArrayList<>(members);
result.removeAll(olderView.getMembers());
return result;
}
@@ -264,14 +268,14 @@ public class GMSMembershipView implements DataSerializableFixedID {
/**
* return members added in this view
*/
- public List<GMSMember> getNewMembers() {
- List<GMSMember> result = new ArrayList<>(5);
+ public List<MemberIdentifier> getNewMembers() {
+ List<MemberIdentifier> result = new ArrayList<>(5);
result.addAll(this.members.stream().filter(mbr -> mbr.getVmViewId() == this.viewId)
.collect(Collectors.toList()));
return result;
}
- public boolean remove(GMSMember mbr) {
+ public boolean remove(MemberIdentifier mbr) {
this.hashedMembers.remove(mbr);
int idx = this.members.indexOf(mbr);
if (idx >= 0) {
@@ -282,12 +286,12 @@ public class GMSMembershipView implements DataSerializableFixedID {
return this.members.remove(mbr);
}
- public void removeAll(Collection<GMSMember> ids) {
+ public void removeAll(Collection<MemberIdentifier> ids) {
this.hashedMembers.removeAll(ids);
ids.forEach(this::remove);
}
- public boolean contains(GMSMember mbr) {
+ public boolean contains(MemberIdentifier mbr) {
return this.hashedMembers.contains(mbr);
}
@@ -295,18 +299,18 @@ public class GMSMembershipView implements DataSerializableFixedID {
return this.members.size();
}
- public GMSMember getLeadMember() {
- for (GMSMember mbr : this.members) {
- if (mbr.getVmKind() == GMSMember.NORMAL_DM_TYPE) {
+ public MemberIdentifier getLeadMember() {
+ for (MemberIdentifier mbr : this.members) {
+ if (mbr.getVmKind() == MemberIdentifier.NORMAL_DM_TYPE) {
return mbr;
}
}
return null;
}
- public GMSMember getCoordinator() {
+ public MemberIdentifier getCoordinator() {
synchronized (membersLock) {
- for (GMSMember addr : members) {
+ for (MemberIdentifier addr : members) {
if (addr.preferredForCoordinator()) {
return addr;
}
@@ -321,18 +325,18 @@ public class GMSMembershipView implements DataSerializableFixedID {
/**
* Returns the coordinator of this view, rejecting any in the given collection of IDs
*/
- public GMSMember getCoordinator(
- Collection<GMSMember> rejections) {
+ public MemberIdentifier getCoordinator(
+ Collection<MemberIdentifier> rejections) {
if (rejections == null) {
return getCoordinator();
}
synchronized (membersLock) {
- for (GMSMember addr : members) {
+ for (MemberIdentifier addr : members) {
if (addr.preferredForCoordinator() && !rejections.contains(addr)) {
return addr;
}
}
- for (GMSMember addr : members) {
+ for (MemberIdentifier addr : members) {
if (!rejections.contains(addr)) {
return addr;
}
@@ -351,14 +355,14 @@ public class GMSMembershipView implements DataSerializableFixedID {
* @param maxNumberDesired number of preferred coordinators to return
* @return list of preferred coordinators
*/
- public List<GMSMember> getPreferredCoordinators(
- Set<GMSMember> filter, GMSMember localAddress,
+ public List<MemberIdentifier> getPreferredCoordinators(
+ Set<MemberIdentifier> filter, MemberIdentifier localAddress,
int maxNumberDesired) {
- List<GMSMember> results = new ArrayList<>();
- List<GMSMember> notPreferredCoordinatorList = new ArrayList<>();
+ List<MemberIdentifier> results = new ArrayList<>();
+ List<MemberIdentifier> notPreferredCoordinatorList = new ArrayList<>();
synchronized (membersLock) {
- for (GMSMember addr : members) {
+ for (MemberIdentifier addr : members) {
if (addr.equals(localAddress)) {
continue;// this is must to add
}
@@ -375,7 +379,7 @@ public class GMSMembershipView implements DataSerializableFixedID {
results.add(localAddress);// to add local address
if (results.size() < maxNumberDesired && notPreferredCoordinatorList.size() > 0) {
- Iterator<GMSMember> it = notPreferredCoordinatorList.iterator();
+ Iterator<MemberIdentifier> it = notPreferredCoordinatorList.iterator();
while (it.hasNext() && results.size() < maxNumberDesired) {
results.add(it.next());
}
@@ -387,29 +391,29 @@ public class GMSMembershipView implements DataSerializableFixedID {
/* NetView implementation method */
- public List<GMSMember> getGMSMembers() {
- return (List<GMSMember>) (List<?>) Collections.unmodifiableList(this.members);
+ public List<MemberIdentifier> getGMSMembers() {
+ return (List<MemberIdentifier>) (List<?>) Collections.unmodifiableList(this.members);
}
- public List<GMSMember> getMembers() {
+ public List<MemberIdentifier> getMembers() {
return Collections.unmodifiableList(this.members);
}
- public Set<GMSMember> getShutdownMembers() {
+ public Set<MemberIdentifier> getShutdownMembers() {
return this.shutdownMembers;
}
- public Set<GMSMember> getCrashedMembers() {
+ public Set<MemberIdentifier> getCrashedMembers() {
return this.crashedMembers;
}
/** check to see if the given address is next in line to be coordinator */
- public boolean shouldBeCoordinator(GMSMember who) {
- Iterator<GMSMember> it = this.members.iterator();
- GMSMember firstNonPreferred = null;
+ public boolean shouldBeCoordinator(MemberIdentifier who) {
+ Iterator<MemberIdentifier> it = this.members.iterator();
+ MemberIdentifier firstNonPreferred = null;
while (it.hasNext()) {
- GMSMember mbr = it.next();
+ MemberIdentifier mbr = it.next();
if (mbr.preferredForCoordinator()) {
return mbr.equals(who);
} else if (firstNonPreferred == null) {
@@ -424,20 +428,20 @@ public class GMSMembershipView implements DataSerializableFixedID {
*/
public int memberWeight() {
int result = 0;
- GMSMember lead = getLeadMember();
- for (GMSMember mbr : this.members) {
+ MemberIdentifier lead = getLeadMember();
+ for (MemberIdentifier mbr : this.members) {
result += mbr.getMemberWeight();
switch (mbr.getVmKind()) {
- case GMSMember.NORMAL_DM_TYPE:
+ case MemberIdentifier.NORMAL_DM_TYPE:
result += 10;
if (lead != null && mbr.equals(lead)) {
result += 5;
}
break;
- case GMSMember.LOCATOR_DM_TYPE:
+ case MemberIdentifier.LOCATOR_DM_TYPE:
result += 3;
break;
- case GMSMember.ADMIN_ONLY_DM_TYPE:
+ case MemberIdentifier.ADMIN_ONLY_DM_TYPE:
break;
default:
throw new IllegalStateException("Unknown member type: " + mbr.getVmKind());
@@ -452,23 +456,23 @@ public class GMSMembershipView implements DataSerializableFixedID {
*/
public int getCrashedMemberWeight(GMSMembershipView oldView) {
int result = 0;
- GMSMember lead = oldView.getLeadMember();
- for (GMSMember mbr : this.crashedMembers) {
+ MemberIdentifier lead = oldView.getLeadMember();
+ for (MemberIdentifier mbr : this.crashedMembers) {
if (!oldView.contains(mbr)) {
continue;
}
result += mbr.getMemberWeight();
switch (mbr.getVmKind()) {
- case GMSMember.NORMAL_DM_TYPE:
+ case MemberIdentifier.NORMAL_DM_TYPE:
result += 10;
if (lead != null && mbr.equals(lead)) {
result += 5;
}
break;
- case GMSMember.LOCATOR_DM_TYPE:
+ case MemberIdentifier.LOCATOR_DM_TYPE:
result += 3;
break;
- case GMSMember.ADMIN_ONLY_DM_TYPE:
+ case MemberIdentifier.ADMIN_ONLY_DM_TYPE:
break;
default:
throw new IllegalStateException("Unknown member type: " + mbr.getVmKind());
@@ -481,10 +485,10 @@ public class GMSMembershipView implements DataSerializableFixedID {
* returns the members of this views crashedMembers collection that were members of the given
* view. Admin-only members are not counted
*/
- public Set<GMSMember> getActualCrashedMembers(GMSMembershipView oldView) {
- Set<GMSMember> result = new HashSet<>(this.crashedMembers.size());
+ public Set<MemberIdentifier> getActualCrashedMembers(GMSMembershipView oldView) {
+ Set<MemberIdentifier> result = new HashSet<>(this.crashedMembers.size());
result.addAll(this.crashedMembers.stream()
- .filter(mbr -> (mbr.getVmKind() != GMSMember.ADMIN_ONLY_DM_TYPE))
+ .filter(mbr -> (mbr.getVmKind() != MemberIdentifier.ADMIN_ONLY_DM_TYPE))
.filter(mbr -> oldView == null || oldView.contains(mbr)).collect(Collectors.toList()));
return result;
}
@@ -493,24 +497,24 @@ public class GMSMembershipView implements DataSerializableFixedID {
* logs the weight of failed members wrt the given previous view
*/
public void logCrashedMemberWeights(GMSMembershipView oldView, Logger log) {
- GMSMember lead = oldView.getLeadMember();
- for (GMSMember mbr : this.crashedMembers) {
+ MemberIdentifier lead = oldView.getLeadMember();
+ for (MemberIdentifier mbr : this.crashedMembers) {
if (!oldView.contains(mbr)) {
continue;
}
int mbrWeight = mbr.getMemberWeight();
switch (mbr.getVmKind()) {
- case GMSMember.NORMAL_DM_TYPE:
+ case MemberIdentifier.NORMAL_DM_TYPE:
if (lead != null && mbr.equals(lead)) {
mbrWeight += 15;
} else {
mbrWeight += 10;
}
break;
- case GMSMember.LOCATOR_DM_TYPE:
+ case MemberIdentifier.LOCATOR_DM_TYPE:
mbrWeight += 3;
break;
- case GMSMember.ADMIN_ONLY_DM_TYPE:
+ case MemberIdentifier.ADMIN_ONLY_DM_TYPE:
break;
default:
throw new IllegalStateException("Unknown member type: " + mbr.getVmKind());
@@ -520,12 +524,12 @@ public class GMSMembershipView implements DataSerializableFixedID {
}
public String toString() {
- GMSMember lead = getLeadMember();
+ MemberIdentifier lead = getLeadMember();
StringBuilder sb = new StringBuilder(200);
sb.append("View[").append(creator).append('|').append(viewId).append("] members: [");
boolean first = true;
- for (GMSMember mbr : this.members) {
+ for (MemberIdentifier mbr : this.members) {
if (!first)
sb.append(", ");
sb.append(mbr);
@@ -537,7 +541,7 @@ public class GMSMembershipView implements DataSerializableFixedID {
if (!this.shutdownMembers.isEmpty()) {
sb.append("] shutdown: [");
first = true;
- for (GMSMember mbr : this.shutdownMembers) {
+ for (MemberIdentifier mbr : this.shutdownMembers) {
if (!first)
sb.append(", ");
sb.append(mbr);
@@ -547,7 +551,7 @@ public class GMSMembershipView implements DataSerializableFixedID {
if (!this.crashedMembers.isEmpty()) {
sb.append("] crashed: [");
first = true;
- for (GMSMember mbr : this.crashedMembers) {
+ for (MemberIdentifier mbr : this.crashedMembers) {
if (!first)
sb.append(", ");
sb.append(mbr);
@@ -571,9 +575,9 @@ public class GMSMembershipView implements DataSerializableFixedID {
* Returns the ID from this view that is equal to the argument. If no such ID exists the argument
* is returned.
*/
- public synchronized GMSMember getCanonicalID(GMSMember id) {
+ public synchronized MemberIdentifier getCanonicalID(MemberIdentifier id) {
if (hashedMembers.contains(id)) {
- for (GMSMember m : this.members) {
+ for (MemberIdentifier m : this.members) {
if (id.equals(m)) {
return m;
}
@@ -629,7 +633,7 @@ public class GMSMembershipView implements DataSerializableFixedID {
}
/** this will deserialize as an ArrayList */
- private void writeAsArrayList(List<GMSMember> list, DataOutput out,
+ private void writeAsArrayList(List<MemberIdentifier> list, DataOutput out,
SerializationContext context) throws IOException {
int size;
if (list == null) {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSUtil.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSUtil.java
index 7e79c74..ea83dbd 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSUtil.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/GMSUtil.java
@@ -17,8 +17,6 @@ package org.apache.geode.distributed.internal.membership.gms;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
@@ -31,6 +29,7 @@ import java.util.StringTokenizer;
import org.apache.logging.log4j.Logger;
import org.apache.geode.GemFireConfigException;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.membership.HostAddress;
import org.apache.geode.internal.logging.LogService;
import org.apache.geode.internal.net.SocketCreator;
@@ -65,31 +64,19 @@ public class GMSUtil {
return parseLocators(locatorsString, addr);
}
- public static GMSMember readMemberID(DataInput in,
+ public static MemberIdentifier readMemberID(DataInput in,
DeserializationContext context) throws IOException, ClassNotFoundException {
- Object id = context.getDeserializer().readObject(in);
- if (id == null || id instanceof GMSMember) {
- return (GMSMember) id;
- }
- // return ((GMSMemberAdapter)((InternalDistributedMember)id).getNetMember()).getGmsMember();
- try {
- Method getNetMember = id.getClass().getMethod("getNetMember");
- Object netMember = getNetMember.invoke(id);
- Method getGmsMember = netMember.getClass().getMethod("getGmsMember");
- return (GMSMember) getGmsMember.invoke(netMember);
- } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
- throw new IllegalStateException("Unable to deserialize a member ID", e);
- }
+ return (MemberIdentifier) context.getDeserializer().readObject(in);
}
- public static Set<GMSMember> readHashSetOfMemberIDs(DataInput in,
+ public static Set<MemberIdentifier> readHashSetOfMemberIDs(DataInput in,
DeserializationContext context)
throws IOException, ClassNotFoundException {
int size = StaticSerialization.readArrayLength(in);
if (size == -1) {
return null;
}
- Set<GMSMember> result = new HashSet<>();
+ Set<MemberIdentifier> result = new HashSet<>();
for (int i = 0; i < size; i++) {
result.add(readMemberID(in, context));
}
@@ -222,27 +209,27 @@ public class GMSUtil {
return sb.toString();
}
- public static List<GMSMember> readArrayOfIDs(DataInput in,
+ public static List<MemberIdentifier> readArrayOfIDs(DataInput in,
DeserializationContext context)
throws IOException, ClassNotFoundException {
int size = StaticSerialization.readArrayLength(in);
if (size == -1) {
return null;
}
- List<GMSMember> result = new ArrayList<>(size);
+ List<MemberIdentifier> result = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
result.add(readMemberID(in, context));
}
return result;
}
- private static void writeAsInternalDistributedMember(GMSMember suspect, DataOutput out,
+ private static void writeAsInternalDistributedMember(MemberIdentifier suspect, DataOutput out,
SerializationContext context) throws IOException {
context.getSerializer().writeDSFID(suspect, DataSerializableFixedID.DISTRIBUTED_MEMBER,
out);
}
- public static void writeMemberID(GMSMember id, DataOutput out,
+ public static void writeMemberID(MemberIdentifier id, DataOutput out,
SerializationContext context) throws IOException {
if (id == null) {
context.getSerializer().writeObject(id, out);
@@ -256,7 +243,7 @@ public class GMSUtil {
}
}
- public static void writeSetOfMemberIDs(Set<GMSMember> set, DataOutput out,
+ public static void writeSetOfMemberIDs(Set<MemberIdentifier> set, DataOutput out,
SerializationContext context) throws IOException {
int size;
if (set == null) {
@@ -266,7 +253,7 @@ public class GMSUtil {
}
StaticSerialization.writeArrayLength(size, out);
if (size > 0) {
- for (GMSMember member : set) {
+ for (MemberIdentifier member : set) {
GMSUtil.writeMemberID(member, out, context);
}
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MemberDataBuilderImpl.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MemberDataBuilderImpl.java
new file mode 100644
index 0000000..f7b770b
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MemberDataBuilderImpl.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.distributed.internal.membership.gms;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.geode.distributed.internal.membership.gms.api.MemberData;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberDataBuilder;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
+import org.apache.geode.internal.net.SocketCreator;
+import org.apache.geode.internal.serialization.Version;
+
+public class MemberDataBuilderImpl implements MemberDataBuilder {
+
+ private static final String EMPTY_STRING = "";
+
+ private InetAddress inetAddress;
+ private String hostName;
+ private int membershipPort = -1;
+ private int directChannelPort = -1;
+ private int vmPid = -1;
+ private int vmKind = MemberIdentifier.NORMAL_DM_TYPE;
+ private int vmViewId = -1;
+ private String name = EMPTY_STRING;
+ private String[] groups;
+ private String durableId;
+ private int durableTimeout = -1;
+ private boolean preferredForCoordinator = true;
+ private boolean networkPartitionDetectionEnabled;
+ private short versionOrdinal = Version.CURRENT_ORDINAL;
+ private long uuidMostSignificantBits = 0;
+ private long uuidLeastSignificantBits = 0;
+
+ public void setMemberWeight(byte memberWeight) {
+ this.memberWeight = memberWeight;
+ }
+
+ private byte memberWeight = 0;
+
+ /**
+ * Create a builder for the given host machine and host name
+ */
+ public static MemberDataBuilderImpl newBuilder(InetAddress hostAddress, String hostName) {
+ return new MemberDataBuilderImpl(hostAddress, hostName);
+ }
+
+ /**
+ * Create a builder for the machine hosting this process
+ */
+ public static MemberDataBuilderImpl newBuilderForLocalHost(String hostName) {
+ return new MemberDataBuilderImpl(hostName);
+ }
+
+ private MemberDataBuilderImpl(InetAddress hostAddress, String hostName) {
+ inetAddress = hostAddress;
+ this.hostName = hostName;
+ }
+
+ private MemberDataBuilderImpl(String fakeHostName) {
+ try {
+ inetAddress = SocketCreator.getLocalHost();
+ } catch (UnknownHostException e2) {
+ throw new RuntimeException("Unable to resolve local host address", e2);
+ }
+ hostName = fakeHostName;
+ }
+
+ public MemberDataBuilderImpl setMembershipPort(int membershipPort) {
+ this.membershipPort = membershipPort;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setDirectPort(int directChannelPort) {
+ this.directChannelPort = directChannelPort;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setVmPid(int vmPid) {
+ this.vmPid = vmPid;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setVmKind(int vmKind) {
+ this.vmKind = vmKind;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setVmViewId(int vmViewId) {
+ this.vmViewId = vmViewId;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setName(String name) {
+ this.name = name;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setGroups(String[] groups) {
+ this.groups = groups;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setDurableId(String durableId) {
+ this.durableId = durableId;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setDurableTimeout(int durableTimeout) {
+ this.durableTimeout = durableTimeout;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setPreferredForCoordinator(boolean preferredForCoordinator) {
+ this.preferredForCoordinator = preferredForCoordinator;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setNetworkPartitionDetectionEnabled(
+ boolean networkPartitionDetectionEnabled) {
+ this.networkPartitionDetectionEnabled = networkPartitionDetectionEnabled;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setVersionOrdinal(short versionOrdinal) {
+ this.versionOrdinal = versionOrdinal;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setUuidMostSignificantBits(long uuidMostSignificantBits) {
+ this.uuidMostSignificantBits = uuidMostSignificantBits;
+ return this;
+ }
+
+ public MemberDataBuilderImpl setUuidLeastSignificantBits(long uuidLeastSignificantBits) {
+ this.uuidLeastSignificantBits = uuidLeastSignificantBits;
+ return this;
+ }
+
+ public MemberData build() {
+ return new GMSMemberData(inetAddress, hostName,
+ membershipPort, vmPid, (byte) vmKind, directChannelPort,
+ vmViewId, name, groups, durableId, durableTimeout,
+ networkPartitionDetectionEnabled, preferredForCoordinator, versionOrdinal,
+ uuidMostSignificantBits, uuidLeastSignificantBits, memberWeight);
+ }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/Authenticator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MemberIdentifierFactoryImpl.java
old mode 100755
new mode 100644
similarity index 50%
copy from geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/Authenticator.java
copy to geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MemberIdentifierFactoryImpl.java
index 6e73b2e..c8509d8
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/Authenticator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MemberIdentifierFactoryImpl.java
@@ -12,29 +12,25 @@
* or implied. See the License for the specific language governing permissions and limitations under
* the License.
*/
-package org.apache.geode.distributed.internal.membership.gms.api;
+package org.apache.geode.distributed.internal.membership.gms;
-import java.util.Properties;
+import java.util.Comparator;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberData;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifierFactory;
-public interface Authenticator {
+class MemberIdentifierFactoryImpl implements MemberIdentifierFactory {
+ static final Comparator<MemberIdentifier> idComparator =
+ (o1, o2) -> ((MemberIdentifierImpl) o1).compareTo((MemberIdentifierImpl) o2);
- /**
- * Authenticate peer member
- *
- * @param member the member to be authenticated
- * @param credentials the credentials used in authentication
- * @return null if authentication succeed (including no authenticator case), otherwise, return
- * failure message
- */
- String authenticate(GMSMember member, Properties credentials);
+ @Override
+ public MemberIdentifier create(MemberData memberInfo) {
+ return new MemberIdentifierImpl(memberInfo);
+ }
- /**
- * Get credential object for the given GemFire distributed member.
- *
- * @param member the target distributed member
- * @return the credentials
- */
- Properties getCredentials(GMSMember member);
+ @Override
+ public Comparator<MemberIdentifier> getComparator() {
+ return idComparator;
+ }
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MemberIdentifierImpl.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MemberIdentifierImpl.java
new file mode 100644
index 0000000..0571402
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MemberIdentifierImpl.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.distributed.internal.membership.gms;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.geode.distributed.internal.membership.gms.api.MemberData;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.internal.serialization.Version;
+
+/**
+ * Unless overridden with a MemberIdentifierFactory GMS will use this as the member identifier
+ * implementation. Geode uses InternalDistributedMember.
+ */
+public class MemberIdentifierImpl implements MemberIdentifier, Comparable<MemberIdentifierImpl> {
+ private MemberData memberData;
+
+ public MemberIdentifierImpl(MemberData memberInfo) {
+ memberData = memberInfo;
+ }
+
+ @Override
+ public MemberData getMemberData() {
+ return memberData;
+ }
+
+ @Override
+ public String getHostName() {
+ return memberData.getHostName();
+ }
+
+ @Override
+ public InetAddress getInetAddress() {
+ return memberData.getInetAddress();
+ }
+
+ @Override
+ public int getPort() {
+ return memberData.getPort();
+ }
+
+ @Override
+ public short getVersionOrdinal() {
+ return memberData.getVersionOrdinal();
+ }
+
+ @Override
+ public int getVmViewId() {
+ return memberData.getVmViewId();
+ }
+
+ @Override
+ public boolean preferredForCoordinator() {
+ return memberData.preferredForCoordinator();
+ }
+
+ @Override
+ public int getVmKind() {
+ return memberData.getVmKind();
+ }
+
+ @Override
+ public int getMemberWeight() {
+ return memberData.getMemberWeight();
+ }
+
+ @Override
+ public List<String> getGroups() {
+ return Arrays.asList(memberData.getGroups());
+ }
+
+ @Override
+ public int getDSFID() {
+ return MEMBER_IDENTIFIER;
+ }
+
+ @Override
+ public void toData(DataOutput out, SerializationContext context) throws IOException {
+ memberData.writeEssentialData(out, context);
+ }
+
+ @Override
+ public void fromData(DataInput in, DeserializationContext context)
+ throws IOException, ClassNotFoundException {
+ memberData = new GMSMemberData();
+ memberData.readEssentialData(in, context);
+ }
+
+ @Override
+ public Version[] getSerializationVersions() {
+ return null;
+ }
+
+ @Override
+ public int compareTo(MemberIdentifierImpl o) {
+ return memberData.compareTo(o.memberData, true);
+ }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MembershipBuilderImpl.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MembershipBuilderImpl.java
index 530021f..7b506c2 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MembershipBuilderImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/MembershipBuilderImpl.java
@@ -14,6 +14,7 @@
*/
package org.apache.geode.distributed.internal.membership.gms;
+
import org.apache.geode.GemFireConfigException;
import org.apache.geode.SystemConnectException;
import org.apache.geode.distributed.internal.ClusterDistributionManager;
@@ -21,6 +22,7 @@ import org.apache.geode.distributed.internal.DistributionException;
import org.apache.geode.distributed.internal.membership.MembershipManager;
import org.apache.geode.distributed.internal.membership.adapter.GMSMembershipManager;
import org.apache.geode.distributed.internal.membership.gms.api.Authenticator;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifierFactory;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipBuilder;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipListener;
@@ -38,6 +40,7 @@ public class MembershipBuilderImpl implements MembershipBuilder {
private ClusterDistributionManager dm;
private MembershipConfig membershipConfig;
private DSFIDSerializer serializer;
+ private MemberIdentifierFactory memberFactory = new MemberIdentifierFactoryImpl();
public MembershipBuilderImpl(ClusterDistributionManager dm) {
this.dm = dm;
@@ -80,12 +83,18 @@ public class MembershipBuilderImpl implements MembershipBuilder {
}
@Override
+ public MembershipBuilder setMemberIDFactory(MemberIdentifierFactory memberFactory) {
+ this.memberFactory = memberFactory;
+ return this;
+ }
+
+ @Override
public MembershipManager create() {
GMSMembershipManager gmsMembershipManager =
new GMSMembershipManager(membershipListener, messageListener, dm);
Services services =
new Services(gmsMembershipManager.getGMSManager(), statistics, authenticator,
- membershipConfig, serializer);
+ membershipConfig, serializer, memberFactory);
try {
services.init();
services.start();
@@ -102,4 +111,6 @@ public class MembershipBuilderImpl implements MembershipBuilder {
return gmsMembershipManager;
}
+
+
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/Services.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/Services.java
index 86dc9af..9db57bd 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/Services.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/Services.java
@@ -19,13 +19,13 @@ import static org.apache.geode.internal.serialization.DataSerializableFixedID.FI
import static org.apache.geode.internal.serialization.DataSerializableFixedID.FIND_COORDINATOR_RESP;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.GET_VIEW_REQ;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.GET_VIEW_RESP;
-import static org.apache.geode.internal.serialization.DataSerializableFixedID.GMSMEMBER;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.HEARTBEAT_REQUEST;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.HEARTBEAT_RESPONSE;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.INSTALL_VIEW_MESSAGE;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.JOIN_REQUEST;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.JOIN_RESPONSE;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.LEAVE_REQUEST_MESSAGE;
+import static org.apache.geode.internal.serialization.DataSerializableFixedID.MEMBER_IDENTIFIER;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.NETVIEW;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.NETWORK_PARTITION_MESSAGE;
import static org.apache.geode.internal.serialization.DataSerializableFixedID.REMOVE_MEMBER_REQUEST;
@@ -41,6 +41,8 @@ import org.apache.geode.ForcedDisconnectException;
import org.apache.geode.annotations.VisibleForTesting;
import org.apache.geode.distributed.DistributedSystemDisconnectedException;
import org.apache.geode.distributed.internal.membership.gms.api.Authenticator;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifierFactory;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipStatistics;
import org.apache.geode.distributed.internal.membership.gms.fd.GMSHealthMonitor;
@@ -84,6 +86,8 @@ public class Services {
private final Stopper cancelCriterion;
private final DSFIDSerializer serializer;
+ private final MemberIdentifierFactory memberFactory;
+
private volatile boolean stopping;
private volatile boolean stopped;
private volatile Exception shutdownCause;
@@ -123,11 +127,12 @@ public class Services {
this.messenger = null;
this.auth = null;
this.serializer = null;
+ this.memberFactory = null;
}
public Services(Manager membershipManager, MembershipStatistics stats,
final Authenticator authenticator, MembershipConfig membershipConfig,
- DSFIDSerializer serializer) {
+ DSFIDSerializer serializer, MemberIdentifierFactory memberFactory) {
this.cancelCriterion = new Stopper();
this.stats = stats;
this.config = membershipConfig;
@@ -137,6 +142,7 @@ public class Services {
this.messenger = new JGroupsMessenger();
this.auth = authenticator;
this.serializer = serializer;
+ this.memberFactory = memberFactory;
registerSerializables(serializer);
}
@@ -151,7 +157,7 @@ public class Services {
serializer.registerDSFID(LEAVE_REQUEST_MESSAGE, LeaveRequestMessage.class);
serializer.registerDSFID(VIEW_ACK_MESSAGE, ViewAckMessage.class);
serializer.registerDSFID(INSTALL_VIEW_MESSAGE, InstallViewMessage.class);
- serializer.registerDSFID(GMSMEMBER, GMSMember.class);
+ serializer.registerDSFID(MEMBER_IDENTIFIER, MemberIdentifier.class);
serializer.registerDSFID(NETVIEW, GMSMembershipView.class);
serializer.registerDSFID(GET_VIEW_REQ, GetViewRequest.class);
serializer.registerDSFID(GET_VIEW_RESP, GetViewResponse.class);
@@ -214,7 +220,7 @@ public class Services {
}
}
- public void setLocalAddress(GMSMember address) {
+ public void setLocalAddress(MemberIdentifier address) {
this.messenger.setLocalAddress(address);
this.joinLeave.setLocalAddress(address);
this.healthMon.setLocalAddress(address);
@@ -291,8 +297,8 @@ public class Services {
this.manager.installView(v);
}
- public void memberSuspected(GMSMember initiator,
- GMSMember suspect, String reason) {
+ public void memberSuspected(MemberIdentifier initiator,
+ MemberIdentifier suspect, String reason) {
try {
this.joinLeave.memberSuspected(initiator, suspect, reason);
} finally {
@@ -340,6 +346,10 @@ public class Services {
return this.stats;
}
+ public MemberIdentifierFactory getMemberFactory() {
+ return memberFactory;
+ }
+
public Stopper getCancelCriterion() {
return this.cancelCriterion;
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java
index 304bcaa..17a141a 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java
@@ -14,20 +14,22 @@
*/
package org.apache.geode.distributed.internal.membership.gms;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
+
/** represents a suspicion raised about a member */
public class SuspectMember {
/** the source of suspicion */
- public GMSMember whoSuspected;
+ public MemberIdentifier whoSuspected;
/** suspected member */
- public GMSMember suspectedMember;
+ public MemberIdentifier suspectedMember;
/** the reason */
public String reason;
/** create a new SuspectMember */
- public SuspectMember(GMSMember whoSuspected,
- GMSMember suspectedMember, String reason) {
+ public SuspectMember(MemberIdentifier whoSuspected,
+ MemberIdentifier suspectedMember, String reason) {
this.whoSuspected = whoSuspected;
this.suspectedMember = suspectedMember;
this.reason = reason;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/Authenticator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/Authenticator.java
index 6e73b2e..a737f18 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/Authenticator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/Authenticator.java
@@ -16,8 +16,6 @@ package org.apache.geode.distributed.internal.membership.gms.api;
import java.util.Properties;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
-
public interface Authenticator {
/**
@@ -28,7 +26,7 @@ public interface Authenticator {
* @return null if authentication succeed (including no authenticator case), otherwise, return
* failure message
*/
- String authenticate(GMSMember member, Properties credentials);
+ String authenticate(MemberIdentifier member, Properties credentials);
/**
* Get credential object for the given GemFire distributed member.
@@ -36,5 +34,5 @@ public interface Authenticator {
* @param member the target distributed member
* @return the credentials
*/
- Properties getCredentials(GMSMember member);
+ Properties getCredentials(MemberIdentifier member);
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberData.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberData.java
new file mode 100644
index 0000000..f9aa90e
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberData.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.distributed.internal.membership.gms.api;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.InetAddress;
+
+import org.jgroups.util.UUID;
+
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.SerializationContext;
+
+/**
+ * MemberIdentifiers are created with a MemberData component. Use MemberDataBuilder to create
+ * one.
+ */
+public interface MemberData {
+ boolean isPartial();
+
+ int getPort();
+
+ boolean preferredForCoordinator();
+
+ void setPreferredForCoordinator(boolean preferred);
+
+ String getDurableId();
+
+ int getDurableTimeout();
+
+ InetAddress getInetAddress();
+
+ short getVersionOrdinal();
+
+ void setVersionOrdinal(short versionOrdinal);
+
+ void setUUID(UUID u);
+
+ UUID getUUID();
+
+ long getUuidMSBs();
+
+ long getUuidLSBs();
+
+ boolean isNetworkPartitionDetectionEnabled();
+
+ byte getMemberWeight();
+
+ InetAddress getInetAddr();
+
+ int getProcessId();
+
+ byte getVmKind();
+
+ int getVmViewId();
+
+ void setVmViewId(int id);
+
+ int getDirectChannelPort();
+
+ String getName();
+
+ String[] getRoles();
+
+ void setUdpPort(int udpPort);
+
+ void setNetworkPartitionDetectionEnabled(boolean networkPartitionDetectionEnabled);
+
+ void setMemberWeight(byte memberWeight);
+
+ void setInetAddr(InetAddress inetAddr);
+
+ void setProcessId(int processId);
+
+ void setVmKind(int vmKind);
+
+ void setVersion(org.apache.geode.internal.serialization.Version v);
+
+ void setDirectChannelPort(int directPort);
+
+ void setName(String name);
+
+ String[] getGroups();
+
+ void setGroups(String[] groups);
+
+ void setPort(int p);
+
+ /**
+ * checks to see if this address has UUID information needed to send messages via JGroups
+ */
+ boolean hasUUID();
+
+ String getHostName();
+
+ void setHostName(String hostName);
+
+ void setDurableTimeout(int newValue);
+
+ void setDurableId(String id);
+
+
+ void writeEssentialData(DataOutput out,
+ SerializationContext context) throws IOException;
+
+ void readEssentialData(java.io.DataInput in,
+ DeserializationContext context) throws IOException, ClassNotFoundException;
+
+
+ boolean hasAdditionalData();
+
+ void writeAdditionalData(DataOutput out) throws IOException;
+
+ void readAdditionalData(java.io.DataInput in) throws ClassNotFoundException, IOException;
+
+
+ int compareTo(MemberData o, boolean compareUUIDs);
+
+ int compareTo(MemberData o, boolean compareUUIDs, boolean compareViewIds);
+
+ int compareAdditionalData(MemberData his);
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberDataBuilder.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberDataBuilder.java
new file mode 100644
index 0000000..3bc7d6b
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberDataBuilder.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.distributed.internal.membership.gms.api;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.StringTokenizer;
+
+import org.apache.geode.distributed.internal.membership.gms.MemberDataBuilderImpl;
+
+/**
+ * Membership identifiers must hold a MemberData object. This builder let's you create
+ * one.<br>
+ * See {@link MemberIdentifierFactory}<br>
+ */
+public interface MemberDataBuilder {
+
+ /**
+ * Create a builder for the given host machine and host name
+ */
+ static MemberDataBuilder newBuilder(InetAddress hostAddress, String hostName) {
+ return MemberDataBuilderImpl.newBuilder(hostAddress, hostName);
+ }
+
+ /**
+ * Create a builder for the machine hosting this process
+ */
+ static MemberDataBuilder newBuilderForLocalHost(String hostName) {
+ return MemberDataBuilderImpl.newBuilderForLocalHost(hostName);
+ }
+
+ /** Parses comma-separated-roles/groups into array of groups (strings). */
+ static String[] parseGroups(String csvRoles, String csvGroups) {
+ List<String> groups = new ArrayList<String>();
+ parseCsv(groups, csvRoles);
+ parseCsv(groups, csvGroups);
+ return (String[]) groups.toArray(new String[groups.size()]);
+ }
+
+ static void parseCsv(List<String> groups, String csv) {
+ if (csv == null || csv.length() == 0) {
+ return;
+ }
+ StringTokenizer st = new StringTokenizer(csv, ",");
+ while (st.hasMoreTokens()) {
+ String groupName = st.nextToken().trim();
+ if (!groups.contains(groupName)) { // only add each group once
+ groups.add(groupName);
+ }
+ }
+ }
+
+ MemberDataBuilder setMembershipPort(int membershipPort);
+
+ MemberDataBuilder setDirectPort(int directChannelPort);
+
+ MemberDataBuilder setVmPid(int vmPid);
+
+ MemberDataBuilder setVmKind(int vmKind);
+
+ MemberDataBuilder setVmViewId(int vmViewId);
+
+ MemberDataBuilder setName(String name);
+
+ MemberDataBuilder setGroups(String[] groups);
+
+ MemberDataBuilder setDurableId(String durableId);
+
+ MemberDataBuilder setDurableTimeout(int durableTimeout);
+
+ MemberDataBuilder setPreferredForCoordinator(boolean preferredForCoordinator);
+
+ MemberDataBuilder setNetworkPartitionDetectionEnabled(boolean networkPartitionDetectionEnabled);
+
+ MemberDataBuilder setVersionOrdinal(short versionOrdinal);
+
+ MemberDataBuilder setUuidMostSignificantBits(long uuidMostSignificantBits);
+
+ MemberDataBuilder setUuidLeastSignificantBits(long uuidLeastSignificantBits);
+
+ MemberData build();
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberIdentifier.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberIdentifier.java
new file mode 100644
index 0000000..b437a9e
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberIdentifier.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.distributed.internal.membership.gms.api;
+
+import java.net.InetAddress;
+import java.util.List;
+
+import org.apache.geode.internal.serialization.DataSerializableFixedID;
+
+/**
+ * MemberIdentifier should be implemented by a user of GMS if the default member identifier
+ * is insufficient. Geode implements InternalDistributedMember.
+ *
+ * @see MemberIdentifierFactory - a factory to create identifiers you can inject into GMS
+ * @see MembershipBuilder - where you inject the factory
+ * @see MemberDataBuilder - used to build the MemberData objects held by identifiers
+ */
+public interface MemberIdentifier extends DataSerializableFixedID {
+ /**
+ * The type for regular members
+ */
+ int NORMAL_DM_TYPE = 10;
+ /**
+ * The DM type for locator members
+ */
+ int LOCATOR_DM_TYPE = 11;
+ /**
+ * The DM type for deprecated admin-only members
+ */
+ int ADMIN_ONLY_DM_TYPE = 12;
+ /**
+ * The DM type for stand-alone members (usually clients)
+ */
+ int LONER_DM_TYPE = 13;
+
+ /**
+ * Return the GMSMemberData associated with this identifier
+ */
+ MemberData getMemberData();
+
+ /**
+ * Return the hostname, if any, associated with this identifier (may be null)
+ */
+ String getHostName();
+
+ /**
+ * Return the InetAddress associated with this identifier
+ */
+ InetAddress getInetAddress();
+
+ /**
+ * Return the membership port associated with this identifier
+ */
+ int getPort();
+
+ /**
+ * Return the serialization version ordinal associated with this identifier
+ */
+ short getVersionOrdinal();
+
+ /**
+ * Return the view identifier in which this identifier was used to join the cluster
+ */
+ int getVmViewId();
+
+ /**
+ * Return whether this identifier is preferred as a membership coordinator over nodes that are not
+ * preferred
+ */
+ boolean preferredForCoordinator();
+
+ /**
+ * Return the type of identifier (normal, locator, admin, loner)
+ */
+ int getVmKind();
+
+ /**
+ * Returns the additional member weight assigned to this identifier
+ */
+ int getMemberWeight();
+
+ /**
+ * Returns the server group names associated with this identifier
+ */
+ List<String> getGroups();
+}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/Authenticator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberIdentifierFactory.java
old mode 100755
new mode 100644
similarity index 56%
copy from geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/Authenticator.java
copy to geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberIdentifierFactory.java
index 6e73b2e..4326e9a
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/Authenticator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MemberIdentifierFactory.java
@@ -14,27 +14,23 @@
*/
package org.apache.geode.distributed.internal.membership.gms.api;
-import java.util.Properties;
+import java.util.Comparator;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
-
-public interface Authenticator {
+/**
+ * A MemberIdentifierFactory is provided when building a membership service. It must provide
+ * implementations of the MemberIdentifier interface for use as identifiers in the membership
+ * service. For Geode this implementation is InternalDistributedMember.<br>
+ * See {@link MembershipBuilder} - where you inject your factory into GMS
+ */
+public interface MemberIdentifierFactory {
/**
- * Authenticate peer member
- *
- * @param member the member to be authenticated
- * @param credentials the credentials used in authentication
- * @return null if authentication succeed (including no authenticator case), otherwise, return
- * failure message
+ * Create a new identifier instance
*/
- String authenticate(GMSMember member, Properties credentials);
+ MemberIdentifier create(MemberData memberInfo);
/**
- * Get credential object for the given GemFire distributed member.
- *
- * @param member the target distributed member
- * @return the credentials
+ * Create a Comparator for the implementation of identifiers provided by this factory
*/
- Properties getCredentials(GMSMember member);
+ Comparator<MemberIdentifier> getComparator();
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MembershipBuilder.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MembershipBuilder.java
index 5bcbe1b..27665a1 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MembershipBuilder.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/api/MembershipBuilder.java
@@ -37,6 +37,10 @@ public interface MembershipBuilder {
MembershipBuilder setSerializer(DSFIDSerializer serializer);
+ MembershipBuilder setMemberIDFactory(MemberIdentifierFactory memberFactory);
+
+
+
MembershipManager create();
static MembershipBuilder newMembershipBuilder(ClusterDistributionManager dm) {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
index 6236d6f..9d55d5f 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitor.java
@@ -55,9 +55,9 @@ import org.apache.geode.CancelException;
import org.apache.geode.GemFireConfigException;
import org.apache.geode.SystemConnectException;
import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipStatistics;
import org.apache.geode.distributed.internal.membership.gms.interfaces.HealthMonitor;
@@ -83,11 +83,11 @@ import org.apache.geode.internal.serialization.Version;
* we check whether this member is still alive or not. Based on that we informed probable
* coordinators to remove that member from view.
* <p>
- * It has {@link #suspect(GMSMember, String)} api, which can be used to initiate
+ * It has {@link #suspect(MemberIdentifier, String)} api, which can be used to initiate
* suspect processing for any member. First is checks whether the member is responding or not. Then
* it informs probable coordinators to remove that member from view.
* <p>
- * It has {@link HealthMonitor#checkIfAvailable(GMSMember, String, boolean)} api to
+ * It has {@link HealthMonitor#checkIfAvailable(MemberIdentifier, String, boolean)} api to
* see if that member is
* alive. Then based on removal flag it initiates the suspect processing for that member.
*/
@@ -96,7 +96,7 @@ public class GMSHealthMonitor implements HealthMonitor {
private Services services;
private volatile GMSMembershipView currentView;
- private volatile GMSMember nextNeighbor;
+ private volatile MemberIdentifier nextNeighbor;
long memberTimeout;
private volatile boolean isStopping = false;
@@ -132,24 +132,24 @@ public class GMSHealthMonitor implements HealthMonitor {
/**
* this member's ID
*/
- private GMSMember localAddress;
+ private MemberIdentifier localAddress;
/**
* Timestamp at which we last had contact from a member
*/
- final ConcurrentMap<GMSMember, TimeStamp> memberTimeStamps =
+ final ConcurrentMap<MemberIdentifier, TimeStamp> memberTimeStamps =
new ConcurrentHashMap<>();
/**
* Members currently being suspected and the view they were suspected in
*/
- private final ConcurrentHashMap<GMSMember, GMSMembershipView> suspectedMemberIds =
+ private final ConcurrentHashMap<MemberIdentifier, GMSMembershipView> suspectedMemberIds =
new ConcurrentHashMap<>();
/**
* Members undergoing final checks
*/
- private final List<GMSMember> membersInFinalCheck =
+ private final List<MemberIdentifier> membersInFinalCheck =
Collections.synchronizedList(new ArrayList<>(30));
/**
@@ -221,7 +221,7 @@ public class GMSHealthMonitor implements HealthMonitor {
/***
* This class sets start interval timestamp to record the activity of all members. That is used by
- * {@link GMSHealthMonitor#contactedBy(GMSMember)} to record the activity of
+ * {@link GMSHealthMonitor#contactedBy(MemberIdentifier)} to record the activity of
* member.
*
* It initiates the suspect processing for next neighbour if it doesn't see any activity from that
@@ -244,7 +244,7 @@ public class GMSHealthMonitor implements HealthMonitor {
@Override
public void run() {
- GMSMember neighbor = nextNeighbor;
+ MemberIdentifier neighbor = nextNeighbor;
if (logger.isDebugEnabled()) {
logger.debug("cluster health monitor invoked with {}", neighbor);
}
@@ -270,7 +270,7 @@ public class GMSHealthMonitor implements HealthMonitor {
logger.info(
"Failure detector has noticed a JVM pause and is giving all members a heartbeat in view {}",
currentView);
- for (GMSMember member : myView.getMembers()) {
+ for (MemberIdentifier member : myView.getMembers()) {
contactedBy(member);
}
return;
@@ -346,11 +346,11 @@ public class GMSHealthMonitor implements HealthMonitor {
long uuidMSBs = in.readLong();
GMSHealthMonitor.this.stats.incFinalCheckRequestsReceived();
GMSHealthMonitor.this.stats.incTcpFinalCheckRequestsReceived();
- GMSMember gmbr = localAddress;
- UUID myUUID = gmbr.getUUID();
+ MemberIdentifier gmbr = localAddress;
+ UUID myUUID = gmbr.getMemberData().getUUID();
// during reconnect or rapid restart we will have a zero viewId but there may still
// be an old ID in the membership view that we do not want to respond to
- int myVmViewId = gmbr.getVmViewId();
+ int myVmViewId = gmbr.getMemberData().getVmViewId();
if (playingDead) {
logger.debug("HealthMonitor: simulating sick member in health check");
} else if (uuidLSBs == myUUID.getLeastSignificantBits()
@@ -410,7 +410,7 @@ public class GMSHealthMonitor implements HealthMonitor {
* Record the member activity for current time interval.
*/
@Override
- public void contactedBy(GMSMember sender) {
+ public void contactedBy(MemberIdentifier sender) {
contactedBy(sender, currentTimeStamp);
}
@@ -418,7 +418,7 @@ public class GMSHealthMonitor implements HealthMonitor {
/**
* Record member activity at a specified time
*/
- private void contactedBy(GMSMember sender, long timeStamp) {
+ private void contactedBy(MemberIdentifier sender, long timeStamp) {
TimeStamp cTS = new TimeStamp(timeStamp);
cTS = memberTimeStamps.putIfAbsent(sender, cTS);
if (cTS != null && cTS.getTime() < timeStamp) {
@@ -432,7 +432,7 @@ public class GMSHealthMonitor implements HealthMonitor {
private HeartbeatRequestMessage constructHeartbeatRequestMessage(
- final GMSMember mbr) {
+ final MemberIdentifier mbr) {
final int reqId = requestId.getAndIncrement();
final HeartbeatRequestMessage hrm = new HeartbeatRequestMessage(mbr, reqId);
hrm.setRecipient(mbr);
@@ -440,7 +440,7 @@ public class GMSHealthMonitor implements HealthMonitor {
return hrm;
}
- private void checkMember(final GMSMember mbr) {
+ private void checkMember(final MemberIdentifier mbr) {
final GMSMembershipView cv = GMSHealthMonitor.this.currentView;
// as check may take time
@@ -470,7 +470,7 @@ public class GMSHealthMonitor implements HealthMonitor {
}
- private void initiateSuspicion(GMSMember mbr, String reason) {
+ private void initiateSuspicion(MemberIdentifier mbr, String reason) {
if (services.getJoinLeave().isMemberLeaving(mbr)) {
return;
}
@@ -481,7 +481,7 @@ public class GMSHealthMonitor implements HealthMonitor {
* This method sends heartbeat request to other member and waits for member-timeout time for
* response. If it doesn't see response then it returns false.
*/
- private boolean doCheckMember(GMSMember member, boolean waitForResponse) {
+ private boolean doCheckMember(MemberIdentifier member, boolean waitForResponse) {
if (playingDead || beingSick) {
// a member playingDead should not be sending messages to other
// members, so we avoid sending heartbeat requests or suspect
@@ -499,7 +499,7 @@ public class GMSHealthMonitor implements HealthMonitor {
hrm.clearRequestId();
}
try {
- Set<GMSMember> membersNotReceivedMsg = this.services.getMessenger().send(hrm);
+ Set<MemberIdentifier> membersNotReceivedMsg = this.services.getMessenger().send(hrm);
this.stats.incHeartbeatRequestsSent();
if (membersNotReceivedMsg != null && membersNotReceivedMsg.contains(member)) {
// member is not part of current view.
@@ -548,7 +548,7 @@ public class GMSHealthMonitor implements HealthMonitor {
* @param suspectMember member that does not respond to HeartbeatRequestMessage
* @return true if successfully exchanged PING/PONG with TCP connection, otherwise false.
*/
- boolean doTCPCheckMember(GMSMember suspectMember, int port,
+ boolean doTCPCheckMember(MemberIdentifier suspectMember, int port,
boolean retryIfConnectFails) {
Socket clientSocket = null;
// make sure we try to check on the member for the contracted memberTimeout period
@@ -599,13 +599,13 @@ public class GMSHealthMonitor implements HealthMonitor {
}
// Package protected for testing purposes
- boolean doTCPCheckMember(GMSMember suspectMember, Socket clientSocket) {
+ boolean doTCPCheckMember(MemberIdentifier suspectMember, Socket clientSocket) {
try {
if (clientSocket.isConnected()) {
clientSocket.setSoTimeout((int) services.getConfig().getMemberTimeout());
InputStream in = clientSocket.getInputStream();
DataOutputStream out = new DataOutputStream(clientSocket.getOutputStream());
- GMSMember gmbr = suspectMember;
+ MemberIdentifier gmbr = suspectMember;
writeMemberToStream(gmbr, out);
this.stats.incFinalCheckRequestsSent();
this.stats.incTcpFinalCheckRequestsSent();
@@ -642,21 +642,21 @@ public class GMSHealthMonitor implements HealthMonitor {
return false;
}
- void writeMemberToStream(GMSMember gmbr, DataOutputStream out) throws IOException {
+ void writeMemberToStream(MemberIdentifier gmbr, DataOutputStream out) throws IOException {
out.writeShort(Version.getCurrentVersion().ordinal());
out.writeInt(gmbr.getVmViewId());
- out.writeLong(gmbr.getUuidLSBs());
- out.writeLong(gmbr.getUuidMSBs());
+ out.writeLong(gmbr.getMemberData().getUuidLSBs());
+ out.writeLong(gmbr.getMemberData().getUuidMSBs());
out.flush();
}
@Override
- public void suspect(GMSMember mbr, String reason) {
+ public void suspect(MemberIdentifier mbr, String reason) {
initiateSuspicion(mbr, reason);
}
@Override
- public boolean checkIfAvailable(GMSMember mbr, String reason,
+ public boolean checkIfAvailable(MemberIdentifier mbr, String reason,
boolean initiateRemoval) {
if (membersInFinalCheck.contains(mbr)) {
return true; // status unknown for now but someone is checking
@@ -763,7 +763,7 @@ public class GMSHealthMonitor implements HealthMonitor {
}
GMSMembershipView v = currentView;
if (v != null) {
- List<GMSMember> mbrs = v.getMembers();
+ List<MemberIdentifier> mbrs = v.getMembers();
int index = mbrs.indexOf(localAddress);
if (index < 0 || mbrs.size() < 2) {
continue;
@@ -775,8 +775,8 @@ public class GMSHealthMonitor implements HealthMonitor {
}
}
- private void sendHeartbeats(List<GMSMember> mbrs, int startIndex) {
- GMSMember coordinator = currentView.getCoordinator();
+ private void sendHeartbeats(List<MemberIdentifier> mbrs, int startIndex) {
+ MemberIdentifier coordinator = currentView.getCoordinator();
if (coordinator != null && !coordinator.equals(localAddress)) {
HeartbeatMessage message = new HeartbeatMessage(-1);
message.setRecipient(coordinator);
@@ -798,7 +798,7 @@ public class GMSHealthMonitor implements HealthMonitor {
if (index < 0) {
index = mbrs.size() - 1;
}
- GMSMember mbr = mbrs.get(index);
+ MemberIdentifier mbr = mbrs.get(index);
if (mbr.equals(localAddress)) {
break;
}
@@ -830,13 +830,13 @@ public class GMSHealthMonitor implements HealthMonitor {
synchronized (suspectRequestsInView) {
suspectRequestsInView.clear();
}
- for (Iterator<GMSMember> it = memberTimeStamps.keySet().iterator(); it
+ for (Iterator<MemberIdentifier> it = memberTimeStamps.keySet().iterator(); it
.hasNext();) {
if (!newView.contains(it.next())) {
it.remove();
}
}
- for (Iterator<GMSMember> it = suspectedMemberIds.keySet().iterator(); it
+ for (Iterator<MemberIdentifier> it = suspectedMemberIds.keySet().iterator(); it
.hasNext();) {
if (!newView.contains(it.next())) {
it.remove();
@@ -862,7 +862,7 @@ public class GMSHealthMonitor implements HealthMonitor {
* It becomes null when we suspect current neighbour, during that time it watches member next to
* suspect member.
*/
- protected synchronized void setNextNeighbor(GMSMembershipView newView, GMSMember nextTo) {
+ protected synchronized void setNextNeighbor(GMSMembershipView newView, MemberIdentifier nextTo) {
if (newView == null) {
return;
}
@@ -870,11 +870,11 @@ public class GMSHealthMonitor implements HealthMonitor {
nextTo = localAddress;
}
- List<GMSMember> allMembers = newView.getMembers();
+ List<MemberIdentifier> allMembers = newView.getMembers();
if (allMembers.size() > 1 && suspectedMemberIds.size() >= allMembers.size() - 1) {
boolean nonSuspectFound = false;
- for (GMSMember member : allMembers) {
+ for (MemberIdentifier member : allMembers) {
if (member.equals(localAddress)) {
continue;
}
@@ -893,12 +893,12 @@ public class GMSHealthMonitor implements HealthMonitor {
int index = allMembers.indexOf(nextTo);
if (index != -1) {
int nextNeighborIndex = (index + 1) % allMembers.size();
- GMSMember newNeighbor = allMembers.get(nextNeighborIndex);
+ MemberIdentifier newNeighbor = allMembers.get(nextNeighborIndex);
if (suspectedMemberIds.containsKey(newNeighbor)) {
setNextNeighbor(newView, newNeighbor);
return;
}
- GMSMember oldNeighbor = nextNeighbor;
+ MemberIdentifier oldNeighbor = nextNeighbor;
if (oldNeighbor != newNeighbor) {
logger.debug("Failure detection is now watching " + newNeighbor);
nextNeighbor = newNeighbor;
@@ -916,7 +916,7 @@ public class GMSHealthMonitor implements HealthMonitor {
}
/** test method */
- public GMSMember getNextNeighbor() {
+ public MemberIdentifier getNextNeighbor() {
return nextNeighbor;
}
@@ -1004,7 +1004,7 @@ public class GMSHealthMonitor implements HealthMonitor {
/**
* Test method - check to see if a member is under suspicion
*/
- public boolean isSuspectMember(GMSMember m) {
+ public boolean isSuspectMember(MemberIdentifier m) {
return this.suspectedMemberIds.containsKey(m);
}
@@ -1014,8 +1014,8 @@ public class GMSHealthMonitor implements HealthMonitor {
}
@Override
- public void memberSuspected(GMSMember initiator,
- GMSMember suspect, String reason) {
+ public void memberSuspected(MemberIdentifier initiator,
+ MemberIdentifier suspect, String reason) {
synchronized (suspectRequestsInView) {
suspectedMemberIds.put(suspect, currentView);
Collection<SuspectRequest> requests = suspectRequestsInView.get(currentView);
@@ -1036,7 +1036,7 @@ public class GMSHealthMonitor implements HealthMonitor {
}
}
- private void memberUnsuspected(GMSMember mbr) {
+ private void memberUnsuspected(MemberIdentifier mbr) {
synchronized (suspectRequestsInView) {
if (suspectedMemberIds.remove(mbr) != null) {
logger.info("No longer suspecting {}", mbr);
@@ -1076,7 +1076,7 @@ public class GMSHealthMonitor implements HealthMonitor {
}
@Override
- public void setLocalAddress(GMSMember idm) {
+ public void setLocalAddress(MemberIdentifier idm) {
this.localAddress = idm;
}
@@ -1096,12 +1096,12 @@ public class GMSHealthMonitor implements HealthMonitor {
}
// only respond if the intended recipient is this member
- GMSMember me = localAddress;
+ MemberIdentifier me = localAddress;
if (me == null || me.getVmViewId() >= 0 && m.getTarget().equals(me)) {
HeartbeatMessage hm = new HeartbeatMessage(m.getRequestId());
hm.setRecipient(m.getSender());
- Set<GMSMember> membersNotReceivedMsg = services.getMessenger().send(hm);
+ Set<MemberIdentifier> membersNotReceivedMsg = services.getMessenger().send(hm);
this.stats.incHeartbeatsSent();
if (membersNotReceivedMsg != null && membersNotReceivedMsg.contains(m.getSender())) {
logger.debug("Unable to send heartbeat to member: {}", m.getSender());
@@ -1165,7 +1165,7 @@ public class GMSHealthMonitor implements HealthMonitor {
List<SuspectRequest> suspectRequests = incomingRequest.getMembers();
- GMSMember sender = incomingRequest.getSender();
+ MemberIdentifier sender = incomingRequest.getSender();
int viewId = sender.getVmViewId();
if (cv.getViewId() >= viewId && !cv.contains(incomingRequest.getSender())) {
logger.info("Membership ignoring suspect request for " + incomingRequest + " from non-member "
@@ -1212,7 +1212,7 @@ public class GMSHealthMonitor implements HealthMonitor {
}
}
List membersLeaving = new ArrayList();
- for (GMSMember member : cv.getMembers()) {
+ for (MemberIdentifier member : cv.getMembers()) {
if (services.getJoinLeave().isMemberLeaving(member)) {
membersLeaving.add(member);
}
@@ -1226,7 +1226,7 @@ public class GMSHealthMonitor implements HealthMonitor {
check,
check.getCoordinator(), localAddress);
- GMSMember coordinator = check.getCoordinator();
+ MemberIdentifier coordinator = check.getCoordinator();
if (coordinator != null && coordinator.equals(localAddress)) {
// new coordinator
checkIfAvailable(sender, membersToCheck, cv);
@@ -1245,7 +1245,7 @@ public class GMSHealthMonitor implements HealthMonitor {
private void logSuspectRequests(SuspectMembersMessage incomingRequest,
- GMSMember sender) {
+ MemberIdentifier sender) {
for (SuspectRequest req : incomingRequest.getMembers()) {
String who = sender.equals(localAddress) ? "myself" : sender.toString();
logger.info("received suspect message from {} for {}: {}", who, req.getSuspectMember(),
@@ -1275,11 +1275,11 @@ public class GMSHealthMonitor implements HealthMonitor {
* available for the member (in the view) then we attempt to connect to its socket and ask if it's
* the expected member. Otherwise we send a heartbeat request and wait for a reply.
*/
- private void checkIfAvailable(final GMSMember initiator,
+ private void checkIfAvailable(final MemberIdentifier initiator,
List<SuspectRequest> sMembers, final GMSMembershipView cv) {
for (final SuspectRequest sr : sMembers) {
- final GMSMember mbr = sr.getSuspectMember();
+ final MemberIdentifier mbr = sr.getSuspectMember();
if (!cv.contains(mbr) || membersInFinalCheck.contains(mbr)) {
continue;
@@ -1314,8 +1314,8 @@ public class GMSHealthMonitor implements HealthMonitor {
* @param reason why we're doing this check
* @return true if the check passes
*/
- protected boolean inlineCheckIfAvailable(final GMSMember initiator,
- final GMSMembershipView cv, boolean isFinalCheck, final GMSMember mbr,
+ protected boolean inlineCheckIfAvailable(final MemberIdentifier initiator,
+ final GMSMembershipView cv, boolean isFinalCheck, final MemberIdentifier mbr,
final String reason) {
if (services.getJoinLeave().isMemberLeaving(mbr)) {
@@ -1408,9 +1408,9 @@ public class GMSHealthMonitor implements HealthMonitor {
&& initiator.getVersionOrdinal() >= Version.GEODE_1_4_0.ordinal()) {
// let others know that this member is no longer suspect
FinalCheckPassedMessage message = new FinalCheckPassedMessage(initiator, mbr);
- List<GMSMember> members = cv.getMembers();
- List<GMSMember> recipients = new ArrayList<>(members.size());
- for (GMSMember member : members) {
+ List<MemberIdentifier> members = cv.getMembers();
+ List<MemberIdentifier> recipients = new ArrayList<>(members.size());
+ for (MemberIdentifier member : members) {
if (!isSuspectMember(member) && !membersInFinalCheck.contains(member) &&
!member.equals(localAddress)) {
recipients.add(member);
@@ -1435,7 +1435,7 @@ public class GMSHealthMonitor implements HealthMonitor {
}
@Override
- public void memberShutdown(GMSMember mbr, String reason) {}
+ public void memberShutdown(MemberIdentifier mbr, String reason) {}
@Override
public int getFailureDetectionPort() {
@@ -1444,10 +1444,10 @@ public class GMSHealthMonitor implements HealthMonitor {
private void sendSuspectRequest(final List<SuspectRequest> requests) {
logger.debug("Sending suspect request for members {}", requests);
- List<GMSMember> recipients;
+ List<MemberIdentifier> recipients;
if (currentView.size() > MembershipConfig.SMALL_CLUSTER_SIZE) {
- HashSet<GMSMember> filter = new HashSet<>();
- for (Enumeration<GMSMember> e = suspectedMemberIds.keys(); e
+ HashSet<MemberIdentifier> filter = new HashSet<>();
+ for (Enumeration<MemberIdentifier> e = suspectedMemberIds.keys(); e
.hasMoreElements();) {
filter.add(e.nextElement());
}
@@ -1463,7 +1463,7 @@ public class GMSHealthMonitor implements HealthMonitor {
logger.trace("Sending suspect messages to {}", recipients);
SuspectMembersMessage smm = new SuspectMembersMessage(recipients, requests);
smm.setSender(localAddress);
- Set<GMSMember> failedRecipients;
+ Set<MemberIdentifier> failedRecipients;
try {
failedRecipients = services.getMessenger().send(smm);
this.stats.incSuspectsSent();
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/GMSMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/GMSMessage.java
index 074bf0e..8dae914 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/GMSMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/GMSMessage.java
@@ -16,13 +16,13 @@ package org.apache.geode.distributed.internal.membership.gms.interfaces;
import java.util.List;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
public interface GMSMessage {
- void setRecipient(GMSMember member);
+ void setRecipient(MemberIdentifier member);
- void setRecipients(List<GMSMember> recipients);
+ void setRecipients(List<MemberIdentifier> recipients);
/** is this a high priority message that should be sent out-of-band? */
boolean isHighPriority();
@@ -31,7 +31,7 @@ public interface GMSMessage {
/** register any reply processor prior to transmission, if necessary */
void registerProcessor();
- List<GMSMember> getRecipients();
+ List<MemberIdentifier> getRecipients();
/** from DataSerializableFixedID */
int getDSFID();
@@ -47,9 +47,9 @@ public interface GMSMessage {
}
/** establishes the sender of a message on the receiving side of a communications channel */
- void setSender(GMSMember sender);
+ void setSender(MemberIdentifier sender);
- GMSMember getSender();
+ MemberIdentifier getSender();
void resetTimestamp();
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/HealthMonitor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/HealthMonitor.java
index d915914..7cdec2f 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/HealthMonitor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/HealthMonitor.java
@@ -15,7 +15,7 @@
package org.apache.geode.distributed.internal.membership.gms.interfaces;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
public interface HealthMonitor extends Service {
@@ -23,13 +23,13 @@ public interface HealthMonitor extends Service {
* Note that this member has been contacted by the given member
*
*/
- void contactedBy(GMSMember sender);
+ void contactedBy(MemberIdentifier sender);
/**
* initiate, asynchronously, suspicion that the member is no longer available
*
*/
- void suspect(GMSMember mbr, String reason);
+ void suspect(MemberIdentifier mbr, String reason);
/**
* Check on the health of the given member, initiating suspicion if it fails. Return true if the
@@ -38,13 +38,13 @@ public interface HealthMonitor extends Service {
* @param reason the reason this check is being performed
* @param initiateRemoval if the member should be removed if it is not available
*/
- boolean checkIfAvailable(GMSMember mbr, String reason, boolean initiateRemoval);
+ boolean checkIfAvailable(MemberIdentifier mbr, String reason, boolean initiateRemoval);
/**
* Invoked by the Manager, this notifies the HealthMonitor that a ShutdownMessage has been
* received from the given member
*/
- void memberShutdown(GMSMember mbr, String reason);
+ void memberShutdown(MemberIdentifier mbr, String reason);
/**
* Returns the failure detection port for this member, or -1 if there is no such port
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/JoinLeave.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/JoinLeave.java
index 6954515..1b6f193 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/JoinLeave.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/JoinLeave.java
@@ -14,8 +14,8 @@
*/
package org.apache.geode.distributed.internal.membership.gms.interfaces;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
public interface JoinLeave extends Service {
@@ -33,23 +33,23 @@ public interface JoinLeave extends Service {
/**
* force another member out of the system
*/
- void remove(GMSMember m, String reason);
+ void remove(MemberIdentifier m, String reason);
/**
* Invoked by the Manager, this notifies the HealthMonitor that a ShutdownMessage has been
* received from the given member
*/
- void memberShutdown(GMSMember mbr, String reason);
+ void memberShutdown(MemberIdentifier mbr, String reason);
/**
* returns the local address
*/
- GMSMember getMemberID();
+ MemberIdentifier getMemberID();
/**
* Get canonical "GMSMember" from current view or prepared view.
*/
- GMSMember getMemberID(GMSMember m);
+ MemberIdentifier getMemberID(MemberIdentifier m);
/**
* returns the current membership view
@@ -66,7 +66,7 @@ public interface JoinLeave extends Service {
* check to see if a member is already in the process of leaving or being removed (in the next
* view)
*/
- boolean isMemberLeaving(GMSMember mbr);
+ boolean isMemberLeaving(MemberIdentifier mbr);
/**
* test hook
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Manager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Manager.java
index dc244f2..45f1b48 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Manager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Manager.java
@@ -16,9 +16,9 @@ package org.apache.geode.distributed.internal.membership.gms.interfaces;
import java.util.Collection;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.serialization.DataSerializableFixedID;
/**
@@ -41,7 +41,7 @@ public interface Manager extends Service, MessageHandler<GMSMessage> {
/**
* notifies the manager that membership quorum has been lost
*/
- void quorumLost(Collection<GMSMember> failures, GMSMembershipView view);
+ void quorumLost(Collection<MemberIdentifier> failures, GMSMembershipView view);
/**
* sometimes we cannot perform multicast messaging, such as during a rolling upgrade.
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Messenger.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Messenger.java
index 0a45aa1..89e3700 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Messenger.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Messenger.java
@@ -17,8 +17,8 @@ package org.apache.geode.distributed.internal.membership.gms.interfaces;
import java.util.Map;
import java.util.Set;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.messenger.GMSQuorumChecker;
public interface Messenger extends Service {
@@ -31,29 +31,29 @@ public interface Messenger extends Service {
* sends an asynchronous message when the membership view may not have been established. Returns
* destinations that did not receive the message due to no longer being in the view
*/
- Set<GMSMember> send(GMSMessage m, GMSMembershipView alternateView);
+ Set<MemberIdentifier> send(GMSMessage m, GMSMembershipView alternateView);
/**
* sends an asynchronous message. Returns destinations that did not receive the message due to no
* longer being in the view
*/
- Set<GMSMember> send(GMSMessage m);
+ Set<MemberIdentifier> send(GMSMessage m);
/**
* sends an asynchronous message. Returns destinations that did not receive the message due to no
* longer being in the view. Does not guarantee delivery of the message (no retransmissions)
*/
- Set<GMSMember> sendUnreliably(GMSMessage m);
+ Set<MemberIdentifier> sendUnreliably(GMSMessage m);
/**
* returns the endpoint ID for this member
*/
- GMSMember getMemberID();
+ MemberIdentifier getMemberID();
/**
* check to see if a member ID has already been used
*/
- boolean isOldMembershipIdentifier(GMSMember id);
+ boolean isOldMembershipIdentifier(MemberIdentifier id);
/**
* retrieves the quorum checker that is used during auto-reconnect attempts
@@ -76,7 +76,7 @@ public interface Messenger extends Service {
* @param state messaging state is stored in this map
* @param includeMulticast whether to record multicast state
*/
- void getMessageState(GMSMember member, Map<String, Long> state,
+ void getMessageState(MemberIdentifier member, Map<String, Long> state,
boolean includeMulticast);
/**
@@ -86,7 +86,7 @@ public interface Messenger extends Service {
* @param member the member flushing operations to this member
* @param state the state of that member's outgoing messaging to this member
*/
- void waitForMessageState(GMSMember member, Map<String, Long> state)
+ void waitForMessageState(MemberIdentifier member, Map<String, Long> state)
throws InterruptedException;
/**
@@ -94,14 +94,14 @@ public interface Messenger extends Service {
*
* @return byte[] public key for member
*/
- byte[] getPublicKey(GMSMember mbr);
+ byte[] getPublicKey(MemberIdentifier mbr);
/**
* Set public key of member.
*
*/
- void setPublicKey(byte[] publickey, GMSMember mbr);
+ void setPublicKey(byte[] publickey, MemberIdentifier mbr);
/**
* Set cluster key in local member.Memebr calls when it gets cluster key in join response
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Service.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Service.java
index 956d4f7..a643df1 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Service.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/interfaces/Service.java
@@ -14,9 +14,9 @@
*/
package org.apache.geode.distributed.internal.membership.gms.interfaces;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
/**
* Services in GMS all implement this interface
@@ -74,9 +74,9 @@ public interface Service {
/**
* a member is suspected of having crashed
*/
- void memberSuspected(GMSMember initiator, GMSMember suspect,
+ void memberSuspected(MemberIdentifier initiator, MemberIdentifier suspect,
String reason);
- default void setLocalAddress(GMSMember address) {}
+ default void setLocalAddress(MemberIdentifier address) {}
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java
index 2b4f3c1..dce7a59 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorRequest.java
@@ -20,8 +20,8 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.messages.AbstractGMSMessage;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
@@ -31,20 +31,20 @@ import org.apache.geode.internal.serialization.Version;
public class FindCoordinatorRequest extends AbstractGMSMessage
implements PeerLocatorRequest {
- private GMSMember memberID;
- private Collection<GMSMember> rejectedCoordinators;
+ private MemberIdentifier memberID;
+ private Collection<MemberIdentifier> rejectedCoordinators;
private int lastViewId;
private byte[] myPublicKey;
private int requestId;
private String dhalgo;
- public FindCoordinatorRequest(GMSMember myId) {
+ public FindCoordinatorRequest(MemberIdentifier myId) {
this.memberID = myId;
this.dhalgo = "";
}
- public FindCoordinatorRequest(GMSMember myId,
- Collection<GMSMember> rejectedCoordinators, int lastViewId, byte[] pk,
+ public FindCoordinatorRequest(MemberIdentifier myId,
+ Collection<MemberIdentifier> rejectedCoordinators, int lastViewId, byte[] pk,
int requestId, String dhalgo) {
this.memberID = myId;
this.rejectedCoordinators = rejectedCoordinators;
@@ -58,7 +58,7 @@ public class FindCoordinatorRequest extends AbstractGMSMessage
// no-arg constructor for serialization
}
- public GMSMember getMemberID() {
+ public MemberIdentifier getMemberID() {
return memberID;
}
@@ -70,7 +70,7 @@ public class FindCoordinatorRequest extends AbstractGMSMessage
return dhalgo;
}
- public Collection<GMSMember> getRejectedCoordinators() {
+ public Collection<MemberIdentifier> getRejectedCoordinators() {
return rejectedCoordinators;
}
@@ -110,7 +110,7 @@ public class FindCoordinatorRequest extends AbstractGMSMessage
GMSUtil.writeMemberID(memberID, out, context);
if (this.rejectedCoordinators != null) {
out.writeInt(this.rejectedCoordinators.size());
- for (GMSMember mbr : this.rejectedCoordinators) {
+ for (MemberIdentifier mbr : this.rejectedCoordinators) {
GMSUtil.writeMemberID(mbr, out, context);
}
} else {
@@ -127,7 +127,7 @@ public class FindCoordinatorRequest extends AbstractGMSMessage
DeserializationContext context) throws IOException, ClassNotFoundException {
this.memberID = GMSUtil.readMemberID(in, context);
int size = in.readInt();
- this.rejectedCoordinators = new ArrayList<GMSMember>(size);
+ this.rejectedCoordinators = new ArrayList<MemberIdentifier>(size);
for (int i = 0; i < size; i++) {
this.rejectedCoordinators.add(GMSUtil.readMemberID(in, context));
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorResponse.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorResponse.java
index eb036c7..bbdfb3a 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/FindCoordinatorResponse.java
@@ -22,9 +22,9 @@ import java.util.HashSet;
import java.util.Objects;
import java.util.Set;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.messages.AbstractGMSMessage;
import org.apache.geode.internal.serialization.DataSerializableFixedID;
import org.apache.geode.internal.serialization.DeserializationContext;
@@ -35,11 +35,11 @@ import org.apache.geode.internal.serialization.Version;
public class FindCoordinatorResponse extends AbstractGMSMessage
implements DataSerializableFixedID {
- private GMSMember coordinator;
- private GMSMember senderId;
+ private MemberIdentifier coordinator;
+ private MemberIdentifier senderId;
private boolean fromView;
private GMSMembershipView view;
- private Set<GMSMember> registrants;
+ private Set<MemberIdentifier> registrants;
private boolean networkPartitionDetectionEnabled;
private boolean usePreferredCoordinators;
private boolean isShortForm;
@@ -48,9 +48,9 @@ public class FindCoordinatorResponse extends AbstractGMSMessage
private int requestId;
- public FindCoordinatorResponse(GMSMember coordinator,
- GMSMember senderId, boolean fromView, GMSMembershipView view,
- HashSet<GMSMember> registrants, boolean networkPartitionDectionEnabled,
+ public FindCoordinatorResponse(MemberIdentifier coordinator,
+ MemberIdentifier senderId, boolean fromView, GMSMembershipView view,
+ HashSet<MemberIdentifier> registrants, boolean networkPartitionDectionEnabled,
boolean usePreferredCoordinators, byte[] pk) {
this.coordinator = coordinator;
this.senderId = senderId;
@@ -63,8 +63,8 @@ public class FindCoordinatorResponse extends AbstractGMSMessage
this.coordinatorPublicKey = pk;
}
- public FindCoordinatorResponse(GMSMember coordinator,
- GMSMember senderId, byte[] pk, int requestId) {
+ public FindCoordinatorResponse(MemberIdentifier coordinator,
+ MemberIdentifier senderId, byte[] pk, int requestId) {
this.coordinator = coordinator;
this.senderId = senderId;
this.isShortForm = true;
@@ -100,7 +100,7 @@ public class FindCoordinatorResponse extends AbstractGMSMessage
return usePreferredCoordinators;
}
- public GMSMember getCoordinator() {
+ public MemberIdentifier getCoordinator() {
return coordinator;
}
@@ -108,7 +108,7 @@ public class FindCoordinatorResponse extends AbstractGMSMessage
* When the response comes from a locator via TcpClient this will return the locators member ID.
* If the locator hasn't yet joined this may be null.
*/
- public GMSMember getSenderId() {
+ public MemberIdentifier getSenderId() {
return senderId;
}
@@ -120,7 +120,7 @@ public class FindCoordinatorResponse extends AbstractGMSMessage
return view;
}
- public Set<GMSMember> getRegistrants() {
+ public Set<MemberIdentifier> getRegistrants() {
return registrants;
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java
index edb0f3d..d7fb123 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/locator/GMSLocator.java
@@ -40,12 +40,13 @@ import org.apache.logging.log4j.Logger;
import org.apache.geode.InternalGemFireException;
import org.apache.geode.annotations.VisibleForTesting;
import org.apache.geode.distributed.internal.LocatorStats;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.interfaces.Locator;
import org.apache.geode.distributed.internal.membership.gms.membership.HostAddress;
+import org.apache.geode.distributed.internal.membership.gms.messenger.GMSMemberWrapper;
import org.apache.geode.distributed.internal.tcpserver.TcpClient;
import org.apache.geode.internal.InternalDataSerializer;
import org.apache.geode.internal.logging.LogService;
@@ -64,15 +65,15 @@ public class GMSLocator implements Locator {
private final String locatorString;
private final List<HostAddress> locators;
private final LocatorStats locatorStats;
- private final Set<GMSMember> registrants = new HashSet<>();
- private final Map<GMSMember.GMSMemberWrapper, byte[]> publicKeys =
+ private final Set<MemberIdentifier> registrants = new HashSet<>();
+ private final Map<GMSMemberWrapper, byte[]> publicKeys =
new ConcurrentHashMap<>();
private final Path workingDirectory;
private volatile boolean isCoordinator;
private Services services;
- private GMSMember localAddress;
+ private MemberIdentifier localAddress;
/**
* The current membership view, or one recovered from disk. This is a copy-on-write variable.
@@ -219,7 +220,7 @@ public class GMSLocator implements Locator {
if (services == null) {
if (findRequest.getMyPublicKey() != null) {
- publicKeys.put(new GMSMember.GMSMemberWrapper(findRequest.getMemberID()),
+ publicKeys.put(new GMSMemberWrapper(findRequest.getMemberID()),
findRequest.getMyPublicKey());
}
logger.debug(
@@ -255,14 +256,14 @@ public class GMSLocator implements Locator {
}
}
- GMSMember coordinator = null;
+ MemberIdentifier coordinator = null;
boolean fromView = false;
if (responseView != null) {
// if the ID of the requester matches an entry in the membership view then remove
// that entry - it's obviously an old member since the ID has been reused
- GMSMember requestingMemberID = findRequest.getMemberID();
- for (GMSMember id : responseView.getMembers()) {
- if (requestingMemberID.compareTo(id, false) == 0) {
+ MemberIdentifier requestingMemberID = findRequest.getMemberID();
+ for (MemberIdentifier id : responseView.getMembers()) {
+ if (requestingMemberID.getMemberData().compareTo(id.getMemberData(), false) == 0) {
GMSMembershipView newView = new GMSMembershipView(responseView, responseView.getViewId());
newView.remove(id);
responseView = newView;
@@ -282,17 +283,18 @@ public class GMSLocator implements Locator {
if (coordinator == null) {
// find the "oldest" registrant
- Collection<GMSMember> rejections = findRequest.getRejectedCoordinators();
+ Collection<MemberIdentifier> rejections = findRequest.getRejectedCoordinators();
if (rejections == null) {
rejections = Collections.emptyList();
}
synchronized (registrants) {
coordinator = services.getJoinLeave().getMemberID();
- for (GMSMember mbr : registrants) {
- if (mbr != coordinator && (coordinator == null || mbr.compareTo(coordinator) < 0)) {
+ for (MemberIdentifier mbr : registrants) {
+ if (mbr != coordinator && (coordinator == null || Objects.compare(mbr, coordinator,
+ services.getMemberFactory().getComparator()) < 0)) {
if (!rejections.contains(mbr) && (mbr.preferredForCoordinator()
- || !mbr.isNetworkPartitionDetectionEnabled())) {
+ || !mbr.getMemberData().isNetworkPartitionDetectionEnabled())) {
coordinator = mbr;
}
}
@@ -356,8 +358,8 @@ public class GMSLocator implements Locator {
locatorStats.endLocatorResponse(startTime);
}
- public byte[] getPublicKey(GMSMember member) {
- return publicKeys.get(new GMSMember.GMSMemberWrapper(member));
+ public byte[] getPublicKey(MemberIdentifier member) {
+ return publicKeys.get(new GMSMemberWrapper(member));
}
public void shutDown() {
@@ -366,7 +368,7 @@ public class GMSLocator implements Locator {
}
@VisibleForTesting
- public List<GMSMember> getMembers() {
+ public List<MemberIdentifier> getMembers() {
if (view != null) {
return new ArrayList<>(view.getMembers());
}
@@ -443,11 +445,11 @@ public class GMSLocator implements Locator {
// this is not a valid view so it shouldn't have a usable Id
recoveredView.setViewId(-1);
- List<GMSMember> members = new ArrayList<>(recoveredView.getMembers());
+ List<MemberIdentifier> members = new ArrayList<>(recoveredView.getMembers());
// Remove locators from the view. Since we couldn't recover from an existing
// locator we know that all of the locators in the view are defunct
- for (GMSMember member : members) {
- if (member.getVmKind() == GMSMember.LOCATOR_DM_TYPE) {
+ for (MemberIdentifier member : members) {
+ if (member.getVmKind() == MemberIdentifier.LOCATOR_DM_TYPE) {
recoveredView.remove(member);
}
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeave.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeave.java
index 8bc6bbe..b964b62 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeave.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeave.java
@@ -50,10 +50,10 @@ import org.apache.geode.distributed.DistributedSystemDisconnectedException;
import org.apache.geode.distributed.Locator;
import org.apache.geode.distributed.internal.ClusterDistributionManager;
import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.interfaces.JoinLeave;
import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorRequest;
@@ -129,7 +129,7 @@ public class GMSJoinLeave implements JoinLeave {
/**
* my address
*/
- private GMSMember localAddress;
+ private MemberIdentifier localAddress;
private Services services;
@@ -161,12 +161,12 @@ public class GMSJoinLeave implements JoinLeave {
/**
* members who we have been declared dead in the current view
*/
- private final Set<GMSMember> removedMembers = new HashSet<>();
+ private final Set<MemberIdentifier> removedMembers = new HashSet<>();
/**
* members who we've received a leave message from
**/
- private final Set<GMSMember> leftMembers = new HashSet<>();
+ private final Set<MemberIdentifier> leftMembers = new HashSet<>();
/**
* a new view being installed
@@ -248,9 +248,9 @@ public class GMSJoinLeave implements JoinLeave {
static class SearchState {
public int joinedMembersContacted;
- Set<GMSMember> alreadyTried = new HashSet<>();
- Set<GMSMember> registrants = new HashSet<>();
- GMSMember possibleCoordinator;
+ Set<MemberIdentifier> alreadyTried = new HashSet<>();
+ Set<MemberIdentifier> registrants = new HashSet<>();
+ MemberIdentifier possibleCoordinator;
int viewId = -100;
int locatorsContacted = 0;
boolean hasContactedAJoinedLocator;
@@ -418,7 +418,7 @@ public class GMSJoinLeave implements JoinLeave {
SearchState state = searchState;
// send a join request to the coordinator and wait for a response
- GMSMember coord = state.possibleCoordinator;
+ MemberIdentifier coord = state.possibleCoordinator;
if (state.alreadyTried.contains(coord)) {
logger.info("Probable coordinator is still {} - waiting for a join-response", coord);
} else {
@@ -482,7 +482,7 @@ public class GMSJoinLeave implements JoinLeave {
joinResponse[0] = null;
// we got view here that means either we have to wait for
GMSMembershipView v = response.getCurrentView();
- GMSMember coord = v.getCoordinator();
+ MemberIdentifier coord = v.getCoordinator();
if (searchState.alreadyTried.contains(coord)) {
searchState.view = response.getCurrentView();
// we already sent join request to it..so lets wait some more time here
@@ -507,7 +507,7 @@ public class GMSJoinLeave implements JoinLeave {
}
@Override
- public boolean isMemberLeaving(GMSMember mbr) {
+ public boolean isMemberLeaving(MemberIdentifier mbr) {
if (getPendingRequestIDs(LEAVE_REQUEST_MESSAGE).contains(mbr)
|| getPendingRequestIDs(REMOVE_MEMBER_REQUEST).contains(mbr)
|| !currentView.contains(mbr)) {
@@ -591,7 +591,7 @@ public class GMSJoinLeave implements JoinLeave {
return;
}
- GMSMember mbr = incomingRequest.getMemberID();
+ MemberIdentifier mbr = incomingRequest.getMemberID();
logger.info(() -> "JoinLeave.processMessage(LeaveRequestMessage) invoked. isCoordinator="
+ isCoordinator
@@ -623,7 +623,7 @@ public class GMSJoinLeave implements JoinLeave {
leftMembers.add(mbr);
check.removeAll(leftMembers);
}
- GMSMember coordinator = check.getCoordinator();
+ MemberIdentifier coordinator = check.getCoordinator();
logger.info("View with removed and left members removed is {} and coordinator would be {}",
check, coordinator);
if (coordinator.equals(localAddress)) {
@@ -655,7 +655,7 @@ public class GMSJoinLeave implements JoinLeave {
boolean fromMe =
incomingRequest.getSender() == null || incomingRequest.getSender().equals(localAddress);
- GMSMember mbr = incomingRequest.getMemberID();
+ MemberIdentifier mbr = incomingRequest.getMemberID();
if (v != null && !v.contains(incomingRequest.getSender())) {
logger.info("Membership ignoring removal request for " + mbr + " from non-member "
@@ -771,7 +771,7 @@ public class GMSJoinLeave implements JoinLeave {
*
* @param oldCoordinator may be null
*/
- private void becomeCoordinator(GMSMember oldCoordinator) {
+ private void becomeCoordinator(MemberIdentifier oldCoordinator) {
assert Thread.holdsLock(viewInstallationLock);
@@ -791,7 +791,7 @@ public class GMSJoinLeave implements JoinLeave {
GMSMembershipView newView = new GMSMembershipView(this.localAddress);
newView.setFailureDetectionPort(localAddress,
services.getHealthMonitor().getFailureDetectionPort());
- this.localAddress.setVmViewId(0);
+ this.localAddress.getMemberData().setVmViewId(0);
installView(newView);
isJoined = true;
createAndStartViewCreator(newView);
@@ -818,18 +818,18 @@ public class GMSJoinLeave implements JoinLeave {
}
}
- private GMSMembershipView copyCurrentViewAndAddMyAddress(GMSMember oldCoordinator) {
+ private GMSMembershipView copyCurrentViewAndAddMyAddress(MemberIdentifier oldCoordinator) {
boolean testing = unitTesting.contains("noRandomViewChange");
GMSMembershipView newView;
- Set<GMSMember> leaving = new HashSet<>();
- Set<GMSMember> removals;
+ Set<MemberIdentifier> leaving = new HashSet<>();
+ Set<MemberIdentifier> removals;
synchronized (viewInstallationLock) {
int rand = testing ? 0 : GMSMembershipView.RANDOM.nextInt(10);
int viewNumber = currentView.getViewId() + 5 + rand;
if (this.localAddress.getVmViewId() < 0) {
- this.localAddress.setVmViewId(viewNumber);
+ this.localAddress.getMemberData().setVmViewId(viewNumber);
}
- List<GMSMember> mbrs = new ArrayList<>(currentView.getMembers());
+ List<MemberIdentifier> mbrs = new ArrayList<>(currentView.getMembers());
if (!mbrs.contains(localAddress)) {
mbrs.add(localAddress);
}
@@ -853,10 +853,10 @@ public class GMSJoinLeave implements JoinLeave {
return newView;
}
- private void sendRemoveMessages(List<GMSMember> removals, List<String> reasons,
- Set<GMSMember> oldIds) {
+ private void sendRemoveMessages(List<MemberIdentifier> removals, List<String> reasons,
+ Set<MemberIdentifier> oldIds) {
Iterator<String> reason = reasons.iterator();
- for (GMSMember mbr : removals) {
+ for (MemberIdentifier mbr : removals) {
// if olds not contains mbr then send remove request
if (!oldIds.contains(mbr)) {
RemoveMemberMessage response = new RemoveMemberMessage(mbr, mbr, reason.next());
@@ -872,7 +872,7 @@ public class GMSJoinLeave implements JoinLeave {
|| services.getManager().shutdownInProgress() || services.getManager().isShutdownStarted();
}
- boolean prepareView(GMSMembershipView view, List<GMSMember> newMembers)
+ boolean prepareView(GMSMembershipView view, List<MemberIdentifier> newMembers)
throws InterruptedException {
// GEODE-2193 - don't send a view with new members if we're shutting down
if (isShuttingDown()) {
@@ -881,7 +881,7 @@ public class GMSJoinLeave implements JoinLeave {
return sendView(view, true, this.prepareProcessor);
}
- void sendView(GMSMembershipView view, List<GMSMember> newMembers)
+ void sendView(GMSMembershipView view, List<MemberIdentifier> newMembers)
throws InterruptedException {
if (isShuttingDown()) {
throw new InterruptedException("shutting down");
@@ -896,13 +896,13 @@ public class GMSJoinLeave implements JoinLeave {
int id = view.getViewId();
InstallViewMessage msg = new InstallViewMessage(view,
services.getAuthenticator().getCredentials(this.localAddress), preparing);
- List<GMSMember> recips = new ArrayList<>(view.getMembers());
+ List<MemberIdentifier> recips = new ArrayList<>(view.getMembers());
// a recent member was seen not to receive a new view - I think this is why
// recips.removeAll(newMembers); // new members get the view in a JoinResponseMessage
recips.remove(this.localAddress); // no need to send it to ourselves
- List<GMSMember> responders = recips;
+ List<MemberIdentifier> responders = recips;
if (!view.getCrashedMembers().isEmpty()) {
recips = new ArrayList<>(recips);
recips.addAll(view.getCrashedMembers());
@@ -934,8 +934,8 @@ public class GMSJoinLeave implements JoinLeave {
msg.setRecipients(recips);
- Set<GMSMember> pendingLeaves = getPendingRequestIDs(LEAVE_REQUEST_MESSAGE);
- Set<GMSMember> pendingRemovals = getPendingRequestIDs(REMOVE_MEMBER_REQUEST);
+ Set<MemberIdentifier> pendingLeaves = getPendingRequestIDs(LEAVE_REQUEST_MESSAGE);
+ Set<MemberIdentifier> pendingRemovals = getPendingRequestIDs(REMOVE_MEMBER_REQUEST);
pendingRemovals.removeAll(view.getCrashedMembers());
viewReplyProcessor.initialize(id, new HashSet(responders));
viewReplyProcessor.processPendingRequests(pendingLeaves, pendingRemovals);
@@ -946,11 +946,11 @@ public class GMSJoinLeave implements JoinLeave {
if (preparing) {
logger.debug("waiting for view responses");
- Set<GMSMember> failedToRespond = viewReplyProcessor.waitForResponses();
+ Set<MemberIdentifier> failedToRespond = viewReplyProcessor.waitForResponses();
logger.info("finished waiting for responses to view preparation");
- GMSMember conflictingViewSender =
+ MemberIdentifier conflictingViewSender =
viewReplyProcessor.getConflictingViewSender();
GMSMembershipView conflictingView = viewReplyProcessor.getConflictingView();
if (conflictingView != null) {
@@ -971,7 +971,7 @@ public class GMSJoinLeave implements JoinLeave {
private void addPublicKeysToView(GMSMembershipView view) {
String sDHAlgo = services.getConfig().getSecurityUDPDHAlgo();
if (sDHAlgo != null && !sDHAlgo.isEmpty()) {
- for (GMSMember mbr : view.getMembers()) {
+ for (MemberIdentifier mbr : view.getMembers()) {
if (Objects.isNull(view.getPublicKey(mbr))) {
byte[] pk = services.getMessenger().getPublicKey(mbr);
view.setPublicKey(mbr, pk);
@@ -1010,7 +1010,7 @@ public class GMSJoinLeave implements JoinLeave {
if (!this.isJoined && !m.isPreparing()) {
// if we're still waiting for a join response and we're in this view we
// should install the view so join() can finish its work
- for (GMSMember mbr : view.getMembers()) {
+ for (MemberIdentifier mbr : view.getMembers()) {
if (localAddress.equals(mbr)) {
viewContainsMyNewAddress = true;
break;
@@ -1031,12 +1031,12 @@ public class GMSJoinLeave implements JoinLeave {
} else {
this.preparedView = view;
// complete filling in the member ID of this node, if possible
- for (GMSMember mbr : view.getMembers()) {
+ for (MemberIdentifier mbr : view.getMembers()) {
if (this.localAddress.equals(mbr)) {
this.birthViewId = mbr.getVmViewId();
- this.localAddress.setVmViewId(this.birthViewId);
- GMSMember me = this.localAddress;
- me.setBirthViewId(birthViewId);
+ this.localAddress.getMemberData().setVmViewId(this.birthViewId);
+ MemberIdentifier me = this.localAddress;
+ me.getMemberData().setVmViewId(birthViewId);
break;
}
}
@@ -1124,8 +1124,8 @@ public class GMSJoinLeave implements JoinLeave {
FindCoordinatorRequest request = new FindCoordinatorRequest(this.localAddress,
state.alreadyTried, state.viewId, services.getMessenger().getPublicKey(localAddress),
services.getMessenger().getRequestId(), dhalgo);
- Set<GMSMember> possibleCoordinators = new HashSet<GMSMember>();
- Set<GMSMember> coordinatorsWithView = new HashSet<GMSMember>();
+ Set<MemberIdentifier> possibleCoordinators = new HashSet<MemberIdentifier>();
+ Set<MemberIdentifier> coordinatorsWithView = new HashSet<MemberIdentifier>();
long giveUpTime =
System.currentTimeMillis() + ((long) services.getConfig().getLocatorWaitTime() * 1000L);
@@ -1161,7 +1161,7 @@ public class GMSJoinLeave implements JoinLeave {
+ "so I will not become membership coordinator on this attempt to join");
state.hasContactedAJoinedLocator = true;
}
- GMSMember responseCoordinator = response.getCoordinator();
+ MemberIdentifier responseCoordinator = response.getCoordinator();
if (responseCoordinator != null) {
anyResponses = true;
GMSMembershipView v = response.getView();
@@ -1207,28 +1207,29 @@ public class GMSJoinLeave implements JoinLeave {
possibleCoordinators = coordinatorsWithView;// lets check current coordinators in view only
}
- Iterator<GMSMember> it = possibleCoordinators.iterator();
+ Iterator<MemberIdentifier> it = possibleCoordinators.iterator();
if (possibleCoordinators.size() == 1) {
state.possibleCoordinator = it.next();
} else {
- GMSMember oldest = it.next();
+ MemberIdentifier oldest = it.next();
while (it.hasNext()) {
- GMSMember candidate = it.next();
- if (oldest.compareTo(candidate) > 0) {
+ MemberIdentifier candidate = it.next();
+ if (services.getMemberFactory().getComparator().compare(oldest, candidate) > 0) {
oldest = candidate;
}
}
state.possibleCoordinator = oldest;
}
- GMSMember coord = null;
+ MemberIdentifier coord = null;
boolean coordIsNoob = true;
for (; it.hasNext();) {
- GMSMember mbr = it.next();
+ MemberIdentifier mbr = it.next();
if (!state.alreadyTried.contains(mbr)) {
boolean mbrIsNoob = (mbr.getVmViewId() < 0);
if (mbrIsNoob) {
// member has not yet joined
- if (coordIsNoob && (coord == null || coord.compareTo(mbr) > 0)) {
+ if (coordIsNoob && (coord == null
+ || services.getMemberFactory().getComparator().compare(coord, mbr) > 0)) {
coord = mbr;
}
} else {
@@ -1259,7 +1260,7 @@ public class GMSJoinLeave implements JoinLeave {
ArrayList<FindCoordinatorResponse> result;
SearchState state = searchState;
GMSMembershipView v = state.view;
- List<GMSMember> recipients = new ArrayList(v.getMembers());
+ List<MemberIdentifier> recipients = new ArrayList(v.getMembers());
logger.info("searching for coordinator in findCoordinatorFromView");
@@ -1284,8 +1285,8 @@ public class GMSJoinLeave implements JoinLeave {
// Here we are sending message one-by-one to all recipients as we don't have cluster secret
// key yet.
// Usually this happens when locator re-joins the cluster and it has saved view.
- for (GMSMember mbr : recipients) {
- List<GMSMember> r = new ArrayList<>();
+ for (MemberIdentifier mbr : recipients) {
+ List<MemberIdentifier> r = new ArrayList<>();
r.add(mbr);
FindCoordinatorRequest req = new FindCoordinatorRequest(localAddress, state.alreadyTried,
state.viewId, services.getMessenger().getPublicKey(localAddress),
@@ -1315,7 +1316,7 @@ public class GMSJoinLeave implements JoinLeave {
state.responses.clear();
}
- GMSMember bestGuessCoordinator = null;
+ MemberIdentifier bestGuessCoordinator = null;
if (localAddress.preferredForCoordinator()) {
// it's possible that all other potential coordinators are gone
// and this new member must become the coordinator
@@ -1325,7 +1326,7 @@ public class GMSJoinLeave implements JoinLeave {
boolean bestGuessIsNotMember = true;
for (FindCoordinatorResponse resp : result) {
logger.info("findCoordinatorFromView processing {}", resp);
- GMSMember suggestedCoordinator = resp.getCoordinator();
+ MemberIdentifier suggestedCoordinator = resp.getCoordinator();
if (resp.getSenderId().getVmViewId() >= 0) {
state.joinedMembersContacted++;
}
@@ -1335,7 +1336,8 @@ public class GMSJoinLeave implements JoinLeave {
if (suggestedIsNotMember) {
// member has not yet joined
if (bestGuessIsNotMember && (bestGuessCoordinator == null
- || bestGuessCoordinator.compareTo(suggestedCoordinator, false) > 0)) {
+ || bestGuessCoordinator.getMemberData()
+ .compareTo(suggestedCoordinator.getMemberData(), false) > 0)) {
bestGuessCoordinator = suggestedCoordinator;
}
} else {
@@ -1459,7 +1461,7 @@ public class GMSJoinLeave implements JoinLeave {
}
@Override
- public GMSMember getMemberID() {
+ public MemberIdentifier getMemberID() {
return this.localAddress;
}
@@ -1476,13 +1478,13 @@ public class GMSJoinLeave implements JoinLeave {
if (currentView == null && !this.isJoined) {
boolean found = false;
- for (GMSMember mbr : newView.getMembers()) {
+ for (MemberIdentifier mbr : newView.getMembers()) {
if (this.localAddress.equals(mbr)) {
found = true;
this.birthViewId = mbr.getVmViewId();
- this.localAddress.setVmViewId(this.birthViewId);
- GMSMember me = (GMSMember) this.localAddress;
- me.setBirthViewId(birthViewId);
+ this.localAddress.getMemberData().setVmViewId(this.birthViewId);
+ MemberIdentifier me = this.localAddress;
+ me.getMemberData().setVmViewId(birthViewId);
break;
}
}
@@ -1494,7 +1496,7 @@ public class GMSJoinLeave implements JoinLeave {
if (isJoined && isNetworkPartition(newView, true)) {
if (quorumRequired) {
- Set<GMSMember> crashes = newView.getActualCrashedMembers(currentView);
+ Set<MemberIdentifier> crashes = newView.getActualCrashedMembers(currentView);
forceDisconnect(String.format(
"Exiting due to possible network partition event due to loss of %s cache processes: %s",
crashes.size(), crashes));
@@ -1567,9 +1569,9 @@ public class GMSJoinLeave implements JoinLeave {
}
}
- private void removeMembersFromCollectionIfNotInView(Collection<GMSMember> members,
+ private void removeMembersFromCollectionIfNotInView(Collection<MemberIdentifier> members,
GMSMembershipView currentView) {
- Iterator<GMSMember> iterator = members.iterator();
+ Iterator<MemberIdentifier> iterator = members.iterator();
while (iterator.hasNext()) {
if (!currentView.contains(iterator.next())) {
iterator.remove();
@@ -1582,7 +1584,7 @@ public class GMSJoinLeave implements JoinLeave {
*
*/
void sendNetworkPartitionMessage(GMSMembershipView view) {
- List<GMSMember> recipients = new ArrayList<>(view.getMembers());
+ List<MemberIdentifier> recipients = new ArrayList<>(view.getMembers());
recipients.remove(localAddress);
NetworkPartitionMessage msg = new NetworkPartitionMessage(recipients);
try {
@@ -1685,22 +1687,22 @@ public class GMSJoinLeave implements JoinLeave {
@Override
public void started() {}
- public void setLocalAddress(GMSMember address) {
+ public void setLocalAddress(MemberIdentifier address) {
this.localAddress = address;
- GMSMember mbr = (GMSMember) this.localAddress;
+ MemberIdentifier mbr = (MemberIdentifier) this.localAddress;
if (services.getConfig().areLocatorsPreferredAsCoordinators()) {
boolean preferred = false;
if (services.getLocator() != null || Locator.hasLocator()
|| !services.getConfig().getStartLocator().isEmpty()
- || localAddress.getVmKind() == GMSMember.LOCATOR_DM_TYPE) {
+ || localAddress.getMemberData().getVmKind() == MemberIdentifier.LOCATOR_DM_TYPE) {
logger
.info("This member is hosting a locator will be preferred as a membership coordinator");
preferred = true;
}
- mbr.setPreferredForCoordinator(preferred);
+ mbr.getMemberData().setPreferredForCoordinator(preferred);
} else {
- mbr.setPreferredForCoordinator(true);
+ mbr.getMemberData().setPreferredForCoordinator(true);
}
}
@@ -1714,8 +1716,8 @@ public class GMSJoinLeave implements JoinLeave {
public void stopped() {}
@Override
- public void memberSuspected(GMSMember initiator,
- GMSMember suspect, String reason) {
+ public void memberSuspected(MemberIdentifier initiator,
+ MemberIdentifier suspect, String reason) {
prepareProcessor.memberSuspected(suspect);
viewProcessor.memberSuspected(suspect);
}
@@ -1728,7 +1730,7 @@ public class GMSJoinLeave implements JoinLeave {
stopCoordinatorServices();
if (view != null) {
if (view.size() > 1) {
- List<GMSMember> coords =
+ List<MemberIdentifier> coords =
view.getPreferredCoordinators(Collections.emptySet(), localAddress, 5);
logger.debug("Sending my leave request to {}", coords);
LeaveRequestMessage m =
@@ -1740,13 +1742,13 @@ public class GMSJoinLeave implements JoinLeave {
}
@Override
- public void remove(GMSMember m, String reason) {
+ public void remove(MemberIdentifier m, String reason) {
GMSMembershipView v = this.currentView;
services.getCancelCriterion().checkCancelInProgress(null);
if (v != null && v.contains(m)) {
- Set<GMSMember> filter = new HashSet<>();
+ Set<MemberIdentifier> filter = new HashSet<>();
filter.add(m);
RemoveMemberMessage msg =
new RemoveMemberMessage(v.getPreferredCoordinators(filter, getMemberID(), 5), m, reason);
@@ -1764,14 +1766,14 @@ public class GMSJoinLeave implements JoinLeave {
}
@Override
- public void memberShutdown(GMSMember mbr, String reason) {
+ public void memberShutdown(MemberIdentifier mbr, String reason) {
LeaveRequestMessage msg = new LeaveRequestMessage(Collections.singletonList(this.localAddress),
mbr, reason);
msg.setSender(mbr);
processMessage(msg);
}
- boolean checkIfAvailable(GMSMember fmbr) {
+ boolean checkIfAvailable(MemberIdentifier fmbr) {
// return the member id if it fails health checks
logger.info("checking state of member " + fmbr);
if (services.getHealthMonitor().checkIfAvailable(fmbr,
@@ -1783,9 +1785,9 @@ public class GMSJoinLeave implements JoinLeave {
return false;
}
- private GMSMember getMemId(GMSMember jgId,
- List<GMSMember> members) {
- for (GMSMember m : members) {
+ private MemberIdentifier getMemId(MemberIdentifier jgId,
+ List<MemberIdentifier> members) {
+ for (MemberIdentifier m : members) {
if (m.equals(jgId)) {
return m;
}
@@ -1794,9 +1796,9 @@ public class GMSJoinLeave implements JoinLeave {
}
@Override
- public GMSMember getMemberID(GMSMember member) {
+ public MemberIdentifier getMemberID(MemberIdentifier member) {
GMSMembershipView v = currentView;
- GMSMember ret = null;
+ MemberIdentifier ret = null;
if (v != null) {
ret = getMemId(member, v.getMembers());
}
@@ -1869,8 +1871,8 @@ public class GMSJoinLeave implements JoinLeave {
/**
* returns the member IDs of the pending requests having the given DataSerializableFixedID
*/
- Set<GMSMember> getPendingRequestIDs(int theDSFID) {
- Set<GMSMember> result = new HashSet<>();
+ Set<MemberIdentifier> getPendingRequestIDs(int theDSFID) {
+ Set<MemberIdentifier> result = new HashSet<>();
synchronized (viewRequests) {
for (AbstractGMSMessage msg : viewRequests) {
if (msg.getDSFID() == theDSFID) {
@@ -1895,18 +1897,18 @@ public class GMSJoinLeave implements JoinLeave {
class ViewReplyProcessor {
volatile int viewId = -1;
- final Set<GMSMember> notRepliedYet = new HashSet<>();
+ final Set<MemberIdentifier> notRepliedYet = new HashSet<>();
GMSMembershipView conflictingView;
- GMSMember conflictingViewSender;
+ MemberIdentifier conflictingViewSender;
volatile boolean waiting;
final boolean isPrepareViewProcessor;
- final Set<GMSMember> pendingRemovals = new HashSet<>();
+ final Set<MemberIdentifier> pendingRemovals = new HashSet<>();
ViewReplyProcessor(boolean forPreparation) {
this.isPrepareViewProcessor = forPreparation;
}
- synchronized void initialize(int viewId, Set<GMSMember> recips) {
+ synchronized void initialize(int viewId, Set<MemberIdentifier> recips) {
waiting = true;
this.viewId = viewId;
notRepliedYet.clear();
@@ -1919,24 +1921,24 @@ public class GMSJoinLeave implements JoinLeave {
return waiting;
}
- synchronized void processPendingRequests(Set<GMSMember> pendingLeaves,
- Set<GMSMember> pendingRemovals) {
+ synchronized void processPendingRequests(Set<MemberIdentifier> pendingLeaves,
+ Set<MemberIdentifier> pendingRemovals) {
// there's no point in waiting for members who have already
// requested to leave or who have been declared crashed.
// We don't want to mix the two because pending removals
// aren't reflected as having crashed in the current view
// and need to cause a new view to be generated
- for (GMSMember mbr : pendingLeaves) {
+ for (MemberIdentifier mbr : pendingLeaves) {
notRepliedYet.remove(mbr);
}
- for (GMSMember mbr : pendingRemovals) {
+ for (MemberIdentifier mbr : pendingRemovals) {
if (this.notRepliedYet.contains(mbr)) {
this.pendingRemovals.add(mbr);
}
}
}
- synchronized void memberSuspected(GMSMember suspect) {
+ synchronized void memberSuspected(MemberIdentifier suspect) {
if (waiting) {
// we will do a final check on this member if it hasn't already
// been done, so stop waiting for it now
@@ -1948,14 +1950,14 @@ public class GMSJoinLeave implements JoinLeave {
}
}
- synchronized void processLeaveRequest(GMSMember mbr) {
+ synchronized void processLeaveRequest(MemberIdentifier mbr) {
if (waiting) {
logger.debug("view response processor recording leave request for {}", mbr);
stopWaitingFor(mbr);
}
}
- synchronized void processRemoveRequest(GMSMember mbr) {
+ synchronized void processRemoveRequest(MemberIdentifier mbr) {
if (waiting) {
logger.debug("view response processor recording remove request for {}", mbr);
pendingRemovals.add(mbr);
@@ -1963,7 +1965,7 @@ public class GMSJoinLeave implements JoinLeave {
}
}
- synchronized void processViewResponse(int viewId, GMSMember sender,
+ synchronized void processViewResponse(int viewId, MemberIdentifier sender,
GMSMembershipView conflictingView) {
if (!waiting) {
return;
@@ -1983,7 +1985,7 @@ public class GMSJoinLeave implements JoinLeave {
/**
* call with synchronized(this)
*/
- private void stopWaitingFor(GMSMember mbr) {
+ private void stopWaitingFor(MemberIdentifier mbr) {
notRepliedYet.remove(mbr);
checkIfDone();
}
@@ -2002,8 +2004,8 @@ public class GMSJoinLeave implements JoinLeave {
}
}
- Set<GMSMember> waitForResponses() throws InterruptedException {
- Set<GMSMember> result;
+ Set<MemberIdentifier> waitForResponses() throws InterruptedException {
+ Set<MemberIdentifier> result;
long endOfWait = System.currentTimeMillis() + viewAckTimeout;
try {
while (System.currentTimeMillis() < endOfWait
@@ -2041,11 +2043,11 @@ public class GMSJoinLeave implements JoinLeave {
return this.conflictingView;
}
- GMSMember getConflictingViewSender() {
+ MemberIdentifier getConflictingViewSender() {
return this.conflictingViewSender;
}
- synchronized Set<GMSMember> getUnresponsiveMembers() {
+ synchronized Set<MemberIdentifier> getUnresponsiveMembers() {
return new HashSet<>(this.notRepliedYet);
}
}
@@ -2070,7 +2072,7 @@ public class GMSJoinLeave implements JoinLeave {
if (v != null) {
InstallViewMessage msg = new InstallViewMessage(v,
services.getAuthenticator().getCredentials(localAddress), false);
- List<GMSMember> recips =
+ List<MemberIdentifier> recips =
new ArrayList<>(v.size() + v.getCrashedMembers().size());
recips.addAll(v.getMembers());
recips.remove(localAddress);
@@ -2099,15 +2101,15 @@ public class GMSJoinLeave implements JoinLeave {
/**
* initial joining members. guarded by synch on ViewCreator
*/
- List<GMSMember> initialJoins = Collections.emptyList();
+ List<MemberIdentifier> initialJoins = Collections.emptyList();
/**
* initial leaving members guarded by synch on ViewCreator
*/
- Set<GMSMember> initialLeaving;
+ Set<MemberIdentifier> initialLeaving;
/**
* initial crashed members. guarded by synch on ViewCreator
*/
- Set<GMSMember> initialRemovals;
+ Set<MemberIdentifier> initialRemovals;
ViewCreator(String name) {
super(name);
@@ -2140,8 +2142,8 @@ public class GMSJoinLeave implements JoinLeave {
* @param leaving - members leaving in this view
* @param removals - members crashed in this view
*/
- synchronized void setInitialView(GMSMembershipView newView, List<GMSMember> newMembers,
- Set<GMSMember> leaving, Set<GMSMember> removals) {
+ synchronized void setInitialView(GMSMembershipView newView, List<MemberIdentifier> newMembers,
+ Set<MemberIdentifier> leaving, Set<MemberIdentifier> removals) {
this.initialView = newView;
this.initialJoins = newMembers;
this.initialLeaving = leaving;
@@ -2162,9 +2164,9 @@ public class GMSJoinLeave implements JoinLeave {
}
try {
GMSMembershipView iView;
- List<GMSMember> iJoins;
- Set<GMSMember> iLeaves;
- Set<GMSMember> iRemoves;
+ List<MemberIdentifier> iJoins;
+ Set<MemberIdentifier> iLeaves;
+ Set<MemberIdentifier> iRemoves;
synchronized (this) {
iView = initialView;
iJoins = initialJoins;
@@ -2206,7 +2208,7 @@ public class GMSJoinLeave implements JoinLeave {
* This allows GMSJoinLeave to tell the ViewCreator to shut down after finishing its current
* task. See GEODE-870.
*/
- private void markViewCreatorForShutdown(GMSMember viewCreator) {
+ private void markViewCreatorForShutdown(MemberIdentifier viewCreator) {
logger.info(
"Marking view creator for shutdown because {} is now the coordinator. My address is {}."
+ " Net member IDs are {} and {} respectively",
@@ -2231,13 +2233,13 @@ public class GMSJoinLeave implements JoinLeave {
// add the new members from the prepared view to the new view,
// preserving their failure-detection ports
- List<GMSMember> newMembers;
+ List<MemberIdentifier> newMembers;
if (currentView != null) {
newMembers = v.getNewMembers(currentView);
} else {
newMembers = v.getMembers();
}
- for (GMSMember newMember : newMembers) {
+ for (MemberIdentifier newMember : newMembers) {
newView.add(newMember);
newView.setFailureDetectionPort(newMember, v.getFailureDetectionPort(newMember));
newView.setPublicKey(newMember, v.getPublicKey(newMember));
@@ -2393,25 +2395,25 @@ public class GMSJoinLeave implements JoinLeave {
*/
void createAndSendView(List<AbstractGMSMessage> requests)
throws InterruptedException, ViewAbandonedException {
- List<GMSMember> joinReqs = new ArrayList<>(10);
- Map<GMSMember, Integer> joinPorts = new HashMap<>(10);
- Set<GMSMember> leaveReqs = new HashSet<>(10);
- List<GMSMember> removalReqs = new ArrayList<>(10);
+ List<MemberIdentifier> joinReqs = new ArrayList<>(10);
+ Map<MemberIdentifier, Integer> joinPorts = new HashMap<>(10);
+ Set<MemberIdentifier> leaveReqs = new HashSet<>(10);
+ List<MemberIdentifier> removalReqs = new ArrayList<>(10);
List<String> removalReasons = new ArrayList<String>(10);
GMSMembershipView oldView = currentView;
- List<GMSMember> oldMembers;
+ List<MemberIdentifier> oldMembers;
if (oldView != null) {
oldMembers = new ArrayList<>(oldView.getMembers());
} else {
oldMembers = Collections.emptyList();
}
- Set<GMSMember> oldIDs = new HashSet<>();
+ Set<MemberIdentifier> oldIDs = new HashSet<>();
for (AbstractGMSMessage msg : requests) {
logger.debug("processing request {}", msg);
- GMSMember mbr;
+ MemberIdentifier mbr;
switch (msg.getDSFID()) {
case JOIN_REQUEST:
JoinRequestMessage jmsg = (JoinRequestMessage) msg;
@@ -2454,7 +2456,7 @@ public class GMSJoinLeave implements JoinLeave {
for (AbstractGMSMessage msg : requests) {
switch (msg.getDSFID()) {
case REMOVE_MEMBER_REQUEST:
- GMSMember mbr = ((RemoveMemberMessage) msg).getMemberID();
+ MemberIdentifier mbr = ((RemoveMemberMessage) msg).getMemberID();
if (!leaveReqs.contains(mbr)) {
if (oldMembers.contains(mbr) && !removalReqs.contains(mbr)) {
removalReqs.add(mbr);
@@ -2472,7 +2474,7 @@ public class GMSJoinLeave implements JoinLeave {
}
}
- for (GMSMember mbr : oldIDs) {
+ for (MemberIdentifier mbr : oldIDs) {
if (!leaveReqs.contains(mbr) && !removalReqs.contains(mbr)) {
removalReqs.add(mbr);
removalReasons.add("Removal of old ID that has been reused");
@@ -2486,12 +2488,12 @@ public class GMSJoinLeave implements JoinLeave {
GMSMembershipView newView;
synchronized (viewInstallationLock) {
int viewNumber = 0;
- List<GMSMember> mbrs;
+ List<MemberIdentifier> mbrs;
if (currentView == null) {
- mbrs = new ArrayList<GMSMember>();
+ mbrs = new ArrayList<MemberIdentifier>();
} else {
viewNumber = currentView.getViewId() + 1;
- mbrs = new ArrayList<GMSMember>(oldMembers);
+ mbrs = new ArrayList<MemberIdentifier>(oldMembers);
}
mbrs.removeAll(leaveReqs);
mbrs.removeAll(removalReqs);
@@ -2499,8 +2501,8 @@ public class GMSJoinLeave implements JoinLeave {
// be reused in an auto-reconnect and get a new vmViewID
mbrs.addAll(joinReqs);
newView = new GMSMembershipView(localAddress, viewNumber, mbrs, leaveReqs,
- new HashSet<GMSMember>(removalReqs));
- for (GMSMember mbr : joinReqs) {
+ new HashSet<MemberIdentifier>(removalReqs));
+ for (MemberIdentifier mbr : joinReqs) {
if (mbrs.contains(mbr)) {
newView.setFailureDetectionPort(mbr, joinPorts.get(mbr));
}
@@ -2518,9 +2520,9 @@ public class GMSJoinLeave implements JoinLeave {
return;
}
- for (GMSMember mbr : joinReqs) {
+ for (MemberIdentifier mbr : joinReqs) {
if (mbr.getVmViewId() < 0) {
- mbr.setVmViewId(newView.getViewId());
+ mbr.getMemberData().setVmViewId(newView.getViewId());
}
}
@@ -2541,8 +2543,8 @@ public class GMSJoinLeave implements JoinLeave {
* This handles the 2-phase installation of the view
*
*/
- void prepareAndSendView(GMSMembershipView newView, List<GMSMember> joinReqs,
- Set<GMSMember> leaveReqs, Set<GMSMember> removalReqs)
+ void prepareAndSendView(GMSMembershipView newView, List<MemberIdentifier> joinReqs,
+ Set<MemberIdentifier> leaveReqs, Set<MemberIdentifier> removalReqs)
throws InterruptedException, ViewAbandonedException {
boolean prepared;
do {
@@ -2554,7 +2556,7 @@ public class GMSJoinLeave implements JoinLeave {
sendNetworkPartitionMessage(newView);
Thread.sleep(BROADCAST_MESSAGE_SLEEP_TIME);
- Set<GMSMember> crashes = newView.getActualCrashedMembers(currentView);
+ Set<MemberIdentifier> crashes = newView.getActualCrashedMembers(currentView);
forceDisconnect(String.format(
"Exiting due to possible network partition event due to loss of %s cache processes: %s",
crashes.size(), crashes));
@@ -2574,7 +2576,7 @@ public class GMSJoinLeave implements JoinLeave {
break;
}
- Set<GMSMember> unresponsive = prepareProcessor.getUnresponsiveMembers();
+ Set<MemberIdentifier> unresponsive = prepareProcessor.getUnresponsiveMembers();
unresponsive.removeAll(removalReqs);
unresponsive.removeAll(leaveReqs);
if (!unresponsive.isEmpty()) {
@@ -2586,7 +2588,7 @@ public class GMSJoinLeave implements JoinLeave {
}
logger.debug("unresponsive members that could not be reached: {}", unresponsive);
- List<GMSMember> failures =
+ List<MemberIdentifier> failures =
new ArrayList<>(currentView.getCrashedMembers().size() + unresponsive.size());
boolean conflictingViewNotFromMe =
@@ -2599,8 +2601,9 @@ public class GMSJoinLeave implements JoinLeave {
lastConflictingView = conflictingView;
// if I am not a locator and the conflicting view is from a locator I should
// let it take control and stop sending membership views
- if (localAddress.getVmKind() != ClusterDistributionManager.LOCATOR_DM_TYPE
- && conflictingView.getCreator()
+ if (localAddress.getMemberData()
+ .getVmKind() != ClusterDistributionManager.LOCATOR_DM_TYPE
+ && conflictingView.getCreator().getMemberData()
.getVmKind() == ClusterDistributionManager.LOCATOR_DM_TYPE) {
logger.info("View preparation interrupted - a locator is taking over as "
+ "membership coordinator in this view: {}", conflictingView);
@@ -2609,18 +2612,18 @@ public class GMSJoinLeave implements JoinLeave {
}
logger.info("adding these crashed members from a conflicting view to the crash-set "
+ "for the next view: {}\nconflicting view: {}", unresponsive, conflictingView);
- failures.addAll((Set<GMSMember>) (Set<?>) conflictingView.getCrashedMembers());
+ failures.addAll((Set<MemberIdentifier>) (Set<?>) conflictingView.getCrashedMembers());
// this member may have been kicked out of the conflicting view
if (failures.contains(localAddress)) {
forceDisconnect("I am no longer a member of the distributed system");
setShutdownFlag();
return;
}
- List<GMSMember> newMembers = conflictingView.getNewMembers();
+ List<MemberIdentifier> newMembers = conflictingView.getNewMembers();
if (!newMembers.isEmpty()) {
logger.info("adding these new members from a conflicting view to the new view: {}",
newMembers);
- for (GMSMember mbr : newMembers) {
+ for (MemberIdentifier mbr : newMembers) {
int port = conflictingView.getFailureDetectionPort(mbr);
newView.add(mbr);
newView.setFailureDetectionPort(mbr, port);
@@ -2646,12 +2649,12 @@ public class GMSJoinLeave implements JoinLeave {
if (!prepared) {
// abort the current view and try again
removalReqs.addAll(failures);
- List<GMSMember> newMembers = new ArrayList<>(newView.getMembers());
+ List<MemberIdentifier> newMembers = new ArrayList<>(newView.getMembers());
newMembers.removeAll(removalReqs);
GMSMembershipView tempView =
new GMSMembershipView(localAddress, newView.getViewId() + 1, newMembers,
leaveReqs, removalReqs);
- for (GMSMember mbr : newView.getMembers()) {
+ for (MemberIdentifier mbr : newView.getMembers()) {
if (tempView.contains(mbr)) {
tempView.setFailureDetectionPort(mbr, newView.getFailureDetectionPort(mbr));
}
@@ -2700,13 +2703,13 @@ public class GMSJoinLeave implements JoinLeave {
* healthy
*
*/
- private void removeHealthyMembers(final Set<GMSMember> suspects)
+ private void removeHealthyMembers(final Set<MemberIdentifier> suspects)
throws InterruptedException {
- List<Callable<GMSMember>> checkers =
+ List<Callable<MemberIdentifier>> checkers =
new ArrayList<>(suspects.size());
- Set<GMSMember> newRemovals = new HashSet<>();
- Set<GMSMember> newLeaves = new HashSet<>();
+ Set<MemberIdentifier> newRemovals = new HashSet<>();
+ Set<MemberIdentifier> newLeaves = new HashSet<>();
filterMembers(suspects, newRemovals, REMOVE_MEMBER_REQUEST);
filterMembers(suspects, newLeaves, LEAVE_REQUEST_MESSAGE);
@@ -2714,13 +2717,13 @@ public class GMSJoinLeave implements JoinLeave {
suspects.removeAll(newLeaves);
- for (GMSMember mbr : suspects) {
+ for (MemberIdentifier mbr : suspects) {
if (newRemovals.contains(mbr) || newLeaves.contains(mbr)) {
continue; // no need to check this member - it's already been checked or is leaving
}
- checkers.add(new Callable<GMSMember>() {
+ checkers.add(new Callable<MemberIdentifier>() {
@Override
- public GMSMember call() throws Exception {
+ public MemberIdentifier call() throws Exception {
boolean available = GMSJoinLeave.this.checkIfAvailable(mbr);
synchronized (viewRequests) {
@@ -2785,16 +2788,16 @@ public class GMSJoinLeave implements JoinLeave {
* @param matchingMembers collection to store matching IDs in
* @param requestType leave/remove/join
*/
- protected void filterMembers(Collection<GMSMember> mbrs,
- Set<GMSMember> matchingMembers, short requestType) {
- Set<GMSMember> requests = getPendingRequestIDs(requestType);
+ protected void filterMembers(Collection<MemberIdentifier> mbrs,
+ Set<MemberIdentifier> matchingMembers, short requestType) {
+ Set<MemberIdentifier> requests = getPendingRequestIDs(requestType);
if (!requests.isEmpty()) {
logger.debug(
"filterMembers: processing " + requests.size() + " requests for type " + requestType);
- Iterator<GMSMember> itr = requests.iterator();
+ Iterator<MemberIdentifier> itr = requests.iterator();
while (itr.hasNext()) {
- GMSMember memberID = itr.next();
+ MemberIdentifier memberID = itr.next();
if (mbrs.contains(memberID)) {
testFlagForRemovalRequest = true;
matchingMembers.add(memberID);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/AbstractGMSMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/AbstractGMSMessage.java
index de51cdb..59cb11c 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/AbstractGMSMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/AbstractGMSMessage.java
@@ -18,15 +18,15 @@ import java.util.Collections;
import java.util.List;
import org.apache.geode.annotations.Immutable;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.interfaces.GMSMessage;
import org.apache.geode.internal.serialization.DataSerializableFixedID;
public abstract class AbstractGMSMessage implements DataSerializableFixedID, GMSMessage {
@Immutable
- public static final GMSMember ALL_RECIPIENTS = null;
- private List<GMSMember> recipients;
- private GMSMember sender;
+ public static final MemberIdentifier ALL_RECIPIENTS = null;
+ private List<MemberIdentifier> recipients;
+ private MemberIdentifier sender;
@Override
public void registerProcessor() {
@@ -39,17 +39,17 @@ public abstract class AbstractGMSMessage implements DataSerializableFixedID, GMS
}
@Override
- public void setRecipient(GMSMember member) {
+ public void setRecipient(MemberIdentifier member) {
recipients = Collections.singletonList(member);
}
@Override
- public void setRecipients(List<GMSMember> recipients) {
+ public void setRecipients(List<MemberIdentifier> recipients) {
this.recipients = recipients;
}
@Override
- public List<GMSMember> getRecipients() {
+ public List<MemberIdentifier> getRecipients() {
if (getMulticast()) {
return Collections.singletonList(ALL_RECIPIENTS);
} else if (this.recipients != null) {
@@ -64,18 +64,18 @@ public abstract class AbstractGMSMessage implements DataSerializableFixedID, GMS
if (getMulticast()) {
return true;
}
- List<GMSMember> recipients = getRecipients();
+ List<MemberIdentifier> recipients = getRecipients();
return recipients == ALL_RECIPIENTS ||
(recipients.size() == 1 && recipients.get(0) == ALL_RECIPIENTS);
}
@Override
- public void setSender(GMSMember sender) {
+ public void setSender(MemberIdentifier sender) {
this.sender = sender;
}
@Override
- public GMSMember getSender() {
+ public MemberIdentifier getSender() {
return sender;
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/FinalCheckPassedMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/FinalCheckPassedMessage.java
index ac956a3..ede8575 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/FinalCheckPassedMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/FinalCheckPassedMessage.java
@@ -18,20 +18,20 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
import org.apache.geode.internal.serialization.Version;
public class FinalCheckPassedMessage extends AbstractGMSMessage {
- private GMSMember suspect;
+ private MemberIdentifier suspect;
public FinalCheckPassedMessage() {}
- public FinalCheckPassedMessage(GMSMember recipient,
- GMSMember suspect) {
+ public FinalCheckPassedMessage(MemberIdentifier recipient,
+ MemberIdentifier suspect) {
super();
setRecipient(recipient);
this.suspect = suspect;
@@ -64,7 +64,7 @@ public class FinalCheckPassedMessage extends AbstractGMSMessage {
suspect = GMSUtil.readMemberID(in, context);
}
- public GMSMember getSuspect() {
+ public MemberIdentifier getSuspect() {
return suspect;
}
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HasMemberID.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HasMemberID.java
index 4ecd976..15a63e0 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HasMemberID.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HasMemberID.java
@@ -14,10 +14,10 @@
*/
package org.apache.geode.distributed.internal.membership.gms.messages;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
public interface HasMemberID {
- GMSMember getMemberID();
+ MemberIdentifier getMemberID();
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HeartbeatRequestMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HeartbeatRequestMessage.java
index 55fc820..6fe358a 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HeartbeatRequestMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HeartbeatRequestMessage.java
@@ -18,8 +18,8 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
import org.apache.geode.internal.serialization.Version;
@@ -27,16 +27,16 @@ import org.apache.geode.internal.serialization.Version;
public class HeartbeatRequestMessage extends AbstractGMSMessage {
int requestId;
- GMSMember target;
+ MemberIdentifier target;
- public HeartbeatRequestMessage(GMSMember neighbour, int id) {
+ public HeartbeatRequestMessage(MemberIdentifier neighbour, int id) {
requestId = id;
this.target = neighbour;
}
public HeartbeatRequestMessage() {}
- public GMSMember getTarget() {
+ public MemberIdentifier getTarget() {
return target;
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/JoinRequestMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/JoinRequestMessage.java
index 738b6fa..a6936d7 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/JoinRequestMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/JoinRequestMessage.java
@@ -19,20 +19,20 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.Objects;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
import org.apache.geode.internal.serialization.Version;
public class JoinRequestMessage extends AbstractGMSMessage {
- private GMSMember memberID;
+ private MemberIdentifier memberID;
private Object credentials;
private int failureDetectionPort = -1;
private int requestId;
private boolean useMulticast;
- public JoinRequestMessage(GMSMember coord, GMSMember id,
+ public JoinRequestMessage(MemberIdentifier coord, MemberIdentifier id,
Object credentials, int fdPort, int requestId) {
super();
if (coord != null) {
@@ -67,7 +67,7 @@ public class JoinRequestMessage extends AbstractGMSMessage {
this.useMulticast = useMulticast;
}
- public GMSMember getMemberID() {
+ public MemberIdentifier getMemberID() {
return memberID;
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/JoinResponseMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/JoinResponseMessage.java
index 0f07dc2..f7fc43d 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/JoinResponseMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/JoinResponseMessage.java
@@ -20,9 +20,9 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.Objects;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
import org.apache.geode.internal.serialization.StaticSerialization;
@@ -39,19 +39,19 @@ public class JoinResponseMessage extends AbstractGMSMessage {
private GMSMembershipView currentView;
private String rejectionMessage;
- private GMSMember memberID;
+ private MemberIdentifier memberID;
private byte[] messengerData;
private int requestId;
private byte[] secretPk;
- public JoinResponseMessage(GMSMember memberID, GMSMembershipView view, int requestId) {
+ public JoinResponseMessage(MemberIdentifier memberID, GMSMembershipView view, int requestId) {
this.currentView = view;
this.memberID = memberID;
this.requestId = requestId;
setRecipient(memberID);
}
- public JoinResponseMessage(GMSMember memberID, byte[] sPk, int requestId) {
+ public JoinResponseMessage(MemberIdentifier memberID, byte[] sPk, int requestId) {
this.memberID = memberID;
this.requestId = requestId;
this.secretPk = sPk;
@@ -79,7 +79,7 @@ public class JoinResponseMessage extends AbstractGMSMessage {
return currentView;
}
- public GMSMember getMemberID() {
+ public MemberIdentifier getMemberID() {
return memberID;
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/LeaveRequestMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/LeaveRequestMessage.java
index 65c4663..10463fb 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/LeaveRequestMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/LeaveRequestMessage.java
@@ -19,26 +19,26 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
import org.apache.geode.internal.serialization.StaticSerialization;
import org.apache.geode.internal.serialization.Version;
public class LeaveRequestMessage extends AbstractGMSMessage implements HasMemberID {
- private GMSMember memberID;
+ private MemberIdentifier memberID;
private String reason;
- public LeaveRequestMessage(List<GMSMember> coords,
- GMSMember id, String reason) {
+ public LeaveRequestMessage(List<MemberIdentifier> coords,
+ MemberIdentifier id, String reason) {
super();
setRecipients(coords);
this.memberID = id;
this.reason = reason;
}
- public LeaveRequestMessage(GMSMember coord, GMSMember id,
+ public LeaveRequestMessage(MemberIdentifier coord, MemberIdentifier id,
String reason) {
super();
setRecipient(coord);
@@ -56,7 +56,7 @@ public class LeaveRequestMessage extends AbstractGMSMessage implements HasMember
}
@Override
- public GMSMember getMemberID() {
+ public MemberIdentifier getMemberID() {
return memberID;
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/NetworkPartitionMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/NetworkPartitionMessage.java
index 4489e72..b9bcf81 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/NetworkPartitionMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/NetworkPartitionMessage.java
@@ -19,7 +19,7 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
import org.apache.geode.internal.serialization.Version;
@@ -28,7 +28,7 @@ public class NetworkPartitionMessage extends AbstractGMSMessage {
public NetworkPartitionMessage() {}
- public NetworkPartitionMessage(List<GMSMember> recipients) {
+ public NetworkPartitionMessage(List<MemberIdentifier> recipients) {
setRecipients(recipients);
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/RemoveMemberMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/RemoveMemberMessage.java
index c4b7bd6..05255a8 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/RemoveMemberMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/RemoveMemberMessage.java
@@ -19,19 +19,19 @@ import java.io.DataOutput;
import java.io.IOException;
import java.util.List;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
import org.apache.geode.internal.serialization.StaticSerialization;
import org.apache.geode.internal.serialization.Version;
public class RemoveMemberMessage extends AbstractGMSMessage implements HasMemberID {
- private GMSMember memberID;
+ private MemberIdentifier memberID;
private String reason;
- public RemoveMemberMessage(GMSMember recipient, GMSMember id,
+ public RemoveMemberMessage(MemberIdentifier recipient, MemberIdentifier id,
String reason) {
super();
setRecipient(recipient);
@@ -39,8 +39,8 @@ public class RemoveMemberMessage extends AbstractGMSMessage implements HasMember
this.reason = reason;
}
- public RemoveMemberMessage(List<GMSMember> recipients,
- GMSMember id, String reason) {
+ public RemoveMemberMessage(List<MemberIdentifier> recipients,
+ MemberIdentifier id, String reason) {
super();
setRecipients(recipients);
this.memberID = id;
@@ -57,7 +57,7 @@ public class RemoveMemberMessage extends AbstractGMSMessage implements HasMember
}
@Override
- public GMSMember getMemberID() {
+ public MemberIdentifier getMemberID() {
return memberID;
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/SuspectMembersMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/SuspectMembersMessage.java
index 312f7c1..ce5c62b 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/SuspectMembersMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/SuspectMembersMessage.java
@@ -20,8 +20,8 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
import org.apache.geode.internal.serialization.StaticSerialization;
@@ -30,7 +30,7 @@ import org.apache.geode.internal.serialization.Version;
public class SuspectMembersMessage extends AbstractGMSMessage {
final List<SuspectRequest> suspectRequests;
- public SuspectMembersMessage(List<GMSMember> recipients, List<SuspectRequest> s) {
+ public SuspectMembersMessage(List<MemberIdentifier> recipients, List<SuspectRequest> s) {
super();
setRecipients(recipients);
this.suspectRequests = s;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/SuspectRequest.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/SuspectRequest.java
index f16a1a9..131ec0b 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/SuspectRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/SuspectRequest.java
@@ -14,18 +14,18 @@
*/
package org.apache.geode.distributed.internal.membership.gms.messages;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
public class SuspectRequest {
- final GMSMember suspectMember;
+ final MemberIdentifier suspectMember;
final String reason;
- public SuspectRequest(GMSMember m, String r) {
+ public SuspectRequest(MemberIdentifier m, String r) {
suspectMember = m;
reason = r;
}
- public GMSMember getSuspectMember() {
+ public MemberIdentifier getSuspectMember() {
return suspectMember;
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/ViewAckMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/ViewAckMessage.java
index 67e71bf..95a5761 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/ViewAckMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/ViewAckMessage.java
@@ -18,8 +18,8 @@ import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.serialization.DeserializationContext;
import org.apache.geode.internal.serialization.SerializationContext;
import org.apache.geode.internal.serialization.Version;
@@ -30,14 +30,14 @@ public class ViewAckMessage extends AbstractGMSMessage {
boolean preparing;
GMSMembershipView alternateView;
- public ViewAckMessage(GMSMember recipient, int viewId, boolean preparing) {
+ public ViewAckMessage(MemberIdentifier recipient, int viewId, boolean preparing) {
super();
setRecipient(recipient);
this.viewId = viewId;
this.preparing = preparing;
}
- public ViewAckMessage(int viewId, GMSMember recipient, GMSMembershipView alternateView) {
+ public ViewAckMessage(int viewId, MemberIdentifier recipient, GMSMembershipView alternateView) {
super();
setRecipient(recipient);
this.viewId = viewId;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncrypt.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncrypt.java
index 95dc356..92995ae 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncrypt.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSEncrypt.java
@@ -31,9 +31,9 @@ import javax.crypto.spec.DHParameterSpec;
import javax.crypto.spec.IvParameterSpec;
import javax.crypto.spec.SecretKeySpec;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.locator.GMSLocator;
import org.apache.geode.internal.util.JavaWorkarounds;
@@ -68,10 +68,10 @@ public final class GMSEncrypt {
/**
* it keeps PK for peers
*/
- private final Map<GMSMember.GMSMemberWrapper, byte[]> memberToPublicKey =
+ private final Map<GMSMemberWrapper, byte[]> memberToPublicKey =
new ConcurrentHashMap<>();
- private final ConcurrentHashMap<GMSMember, GMSEncryptionCipherPool> peerEncryptors =
+ private final ConcurrentHashMap<MemberIdentifier, GMSEncryptionCipherPool> peerEncryptors =
new ConcurrentHashMap<>();
private GMSEncryptionCipherPool clusterEncryptor;
@@ -103,7 +103,7 @@ public final class GMSEncrypt {
this.clusterEncryptor = new GMSEncryptionCipherPool(this, secretBytes);
}
- private byte[] getPublicKeyIfIAmLocator(GMSMember mbr) {
+ private byte[] getPublicKeyIfIAmLocator(MemberIdentifier mbr) {
GMSLocator locator = (GMSLocator) services.getLocator();
if (locator != null) {
return locator.getPublicKey(mbr);
@@ -134,11 +134,11 @@ public final class GMSEncrypt {
}
}
- byte[] decryptData(byte[] data, GMSMember member) throws Exception {
+ byte[] decryptData(byte[] data, MemberIdentifier member) throws Exception {
return getPeerEncryptor(member).decryptBytes(data);
}
- byte[] encryptData(byte[] data, GMSMember member) throws Exception {
+ byte[] encryptData(byte[] data, MemberIdentifier member) throws Exception {
return getPeerEncryptor(member).encryptBytes(data);
}
@@ -160,8 +160,8 @@ public final class GMSEncrypt {
return dhPublicKey.getEncoded();
}
- private byte[] lookupKeyByMember(GMSMember member) {
- byte[] pk = memberToPublicKey.get(new GMSMember.GMSMemberWrapper(member));
+ private byte[] lookupKeyByMember(MemberIdentifier member) {
+ byte[] pk = memberToPublicKey.get(new GMSMemberWrapper(member));
if (pk == null) {
pk = getPublicKeyIfIAmLocator(member);
}
@@ -174,9 +174,9 @@ public final class GMSEncrypt {
return pk;
}
- protected byte[] getPublicKey(GMSMember member) {
+ protected byte[] getPublicKey(MemberIdentifier member) {
try {
- GMSMember localMbr = services.getMessenger().getMemberID();
+ MemberIdentifier localMbr = services.getMessenger().getMemberID();
if (localMbr != null && localMbr.equals(member)) {
return this.dhPublicKey.getEncoded();// local one
}
@@ -186,16 +186,16 @@ public final class GMSEncrypt {
}
}
- protected void setPublicKey(byte[] publickey, GMSMember mbr) {
+ protected void setPublicKey(byte[] publickey, MemberIdentifier mbr) {
try {
- memberToPublicKey.put(new GMSMember.GMSMemberWrapper(mbr), publickey);
+ memberToPublicKey.put(new GMSMemberWrapper(mbr), publickey);
peerEncryptors.replace(mbr, new GMSEncryptionCipherPool(this, generateSecret(publickey)));
} catch (Exception e) {
throw new RuntimeException("Unable to create peer encryptor " + mbr, e);
}
}
- private GMSEncryptionCipherPool getPeerEncryptor(GMSMember member)
+ private GMSEncryptionCipherPool getPeerEncryptor(MemberIdentifier member)
throws Exception {
return JavaWorkarounds.computeIfAbsent(peerEncryptors, member, (mbr) -> {
try {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSMemberWrapper.java
similarity index 51%
copy from geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java
copy to geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSMemberWrapper.java
index 304bcaa..acda84e 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/SuspectMember.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSMemberWrapper.java
@@ -12,42 +12,40 @@
* or implied. See the License for the specific language governing permissions and limitations under
* the License.
*/
-package org.apache.geode.distributed.internal.membership.gms;
+package org.apache.geode.distributed.internal.membership.gms.messenger;
-/** represents a suspicion raised about a member */
-public class SuspectMember {
- /** the source of suspicion */
- public GMSMember whoSuspected;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
- /** suspected member */
- public GMSMember suspectedMember;
+/**
+ * A wrapper for GMSMember objects used in encryption services
+ */
+public class GMSMemberWrapper {
- /** the reason */
- public String reason;
+ MemberIdentifier mbr;
- /** create a new SuspectMember */
- public SuspectMember(GMSMember whoSuspected,
- GMSMember suspectedMember, String reason) {
- this.whoSuspected = whoSuspected;
- this.suspectedMember = suspectedMember;
- this.reason = reason;
+ public GMSMemberWrapper(MemberIdentifier m) {
+ this.mbr = m;
}
- @Override
- public String toString() {
- return "{source=" + whoSuspected + "; suspect=" + suspectedMember + "}";
+ public MemberIdentifier getMbr() {
+ return mbr;
}
@Override
public int hashCode() {
- return this.suspectedMember.hashCode();
+ return mbr.hashCode();
}
@Override
- public boolean equals(Object other) {
- if (!(other instanceof SuspectMember)) {
- return false;
- }
- return this.suspectedMember.equals(((SuspectMember) other).suspectedMember);
+ public boolean equals(Object obj) {
+ MemberIdentifier other = ((GMSMemberWrapper) obj).mbr;
+ // here we must compare member data rather than identifiers since the view identifiers and
+ // UUID identifiers need to be ignored
+ return mbr.getMemberData().compareTo(other.getMemberData(), false, false) == 0;
+ }
+
+ @Override
+ public String toString() {
+ return "GMSMemberWrapper [mbr=" + mbr + "]";
}
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumChecker.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumChecker.java
index 3a5d716..dd2c504 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumChecker.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/GMSQuorumChecker.java
@@ -33,18 +33,18 @@ import org.jgroups.Message;
import org.jgroups.Receiver;
import org.jgroups.View;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.concurrent.ConcurrentHashSet;
import org.apache.geode.internal.logging.LogService;
public class GMSQuorumChecker {
private static final Logger logger = LogService.getLogger();
private boolean isInfoEnabled = false;
- private Map<SocketAddress, GMSMember> addressConversionMap;
+ private Map<SocketAddress, MemberIdentifier> addressConversionMap;
private GMSPingPonger pingPonger;
- private Set<GMSMember> receivedAcks;
+ private Set<MemberIdentifier> receivedAcks;
private final GMSMembershipView lastView;
@@ -53,11 +53,11 @@ public class GMSQuorumChecker {
private final JChannel channel;
private JGAddress myAddress;
private final long partitionThreshold;
- private Set<GMSMember> oldMemberIdentifiers;
+ private Set<MemberIdentifier> oldMemberIdentifiers;
private ConcurrentLinkedQueue<Message> messageQueue = new ConcurrentLinkedQueue<>();
public GMSQuorumChecker(GMSMembershipView jgView, int partitionThreshold, JChannel channel,
- Set<GMSMember> oldMemberIdentifiers) {
+ Set<MemberIdentifier> oldMemberIdentifiers) {
this.lastView = jgView;
this.partitionThreshold = partitionThreshold;
this.channel = channel;
@@ -71,8 +71,8 @@ public class GMSQuorumChecker {
myAddress = (JGAddress) channel.down(new Event(Event.GET_LOCAL_ADDRESS));
addressConversionMap = new ConcurrentHashMap<>(this.lastView.size());
- List<GMSMember> members = this.lastView.getMembers();
- for (GMSMember addr : members) {
+ List<MemberIdentifier> members = this.lastView.getMembers();
+ for (MemberIdentifier addr : members) {
SocketAddress sockaddr =
new InetSocketAddress(addr.getInetAddress(), addr.getPort());
addressConversionMap.put(sockaddr, addr);
@@ -164,10 +164,10 @@ public class GMSQuorumChecker {
return false;
}
- private int getWeight(Collection<GMSMember> idms,
- GMSMember leader) {
+ private int getWeight(Collection<MemberIdentifier> idms,
+ MemberIdentifier leader) {
int weight = 0;
- for (GMSMember mbr : idms) {
+ for (MemberIdentifier mbr : idms) {
int thisWeight = mbr.getMemberWeight();
if (mbr.getVmKind() == 10 /* NORMAL_DM_KIND */) {
thisWeight += 10;
@@ -184,8 +184,8 @@ public class GMSQuorumChecker {
private void sendPingMessages() {
// send a ping message to each member in the last view seen
- List<GMSMember> members = this.lastView.getMembers();
- for (GMSMember addr : members) {
+ List<MemberIdentifier> members = this.lastView.getMembers();
+ for (MemberIdentifier addr : members) {
if (!receivedAcks.contains(addr)) {
JGAddress dest = new JGAddress(addr);
if (isInfoEnabled) {
@@ -244,7 +244,7 @@ public class GMSQuorumChecker {
logger.info("received ping-pong response from {}", sender);
JGAddress jgSender = (JGAddress) sender;
SocketAddress sockaddr = new InetSocketAddress(jgSender.getInetAddress(), jgSender.getPort());
- GMSMember memberAddr = addressConversionMap.get(sockaddr);
+ MemberIdentifier memberAddr = addressConversionMap.get(sockaddr);
if (memberAddr != null) {
logger.info("quorum check: mapped address to member ID {}", memberAddr);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGAddress.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGAddress.java
index 731b702..ebe5f91 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGAddress.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGAddress.java
@@ -28,7 +28,8 @@ import org.jgroups.stack.IpAddress;
import org.jgroups.util.UUID;
import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberData;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.net.SocketCreator;
/**
@@ -52,13 +53,14 @@ public class JGAddress extends UUID {
// Used only by Externalization
public JGAddress() {}
- public JGAddress(GMSMember mbr) {
+ public JGAddress(MemberIdentifier mbr) {
super();
- this.mostSigBits = mbr.getUuidMSBs();
- this.leastSigBits = mbr.getUuidLSBs();
this.ip_addr = mbr.getInetAddress();
this.port = mbr.getPort();
- this.vmViewId = mbr.getVmViewId();
+ MemberData memberData = mbr.getMemberData();
+ this.mostSigBits = memberData.getUuidMSBs();
+ this.leastSigBits = memberData.getUuidLSBs();
+ this.vmViewId = memberData.getVmViewId();
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessenger.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
index be4ebe6..adf9768 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/JGroupsMessenger.java
@@ -77,10 +77,12 @@ import org.apache.geode.alerting.internal.spi.AlertingAction;
import org.apache.geode.annotations.internal.MutableForTesting;
import org.apache.geode.distributed.DistributedSystemDisconnectedException;
import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.GMSMemberData;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.GMSUtil;
import org.apache.geode.distributed.internal.membership.gms.Services;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberData;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipStatistics;
import org.apache.geode.distributed.internal.membership.gms.interfaces.GMSMessage;
@@ -128,7 +130,7 @@ public class JGroupsMessenger implements Messenger {
protected String jgStackConfig;
JChannel myChannel;
- GMSMember localAddress;
+ MemberIdentifier localAddress;
JGAddress jgAddress;
private Services services;
@@ -142,7 +144,7 @@ public class JGroupsMessenger implements Messenger {
protected final AtomicLong pongsReceived = new AtomicLong(0);
/** tracks multicast messages that have been scheduled for processing */
- protected final Map<GMSMember, MessageTracker> scheduledMcastSeqnos = new HashMap<>();
+ protected final Map<MemberIdentifier, MessageTracker> scheduledMcastSeqnos = new HashMap<>();
protected short nackack2HeaderId;
@@ -167,7 +169,7 @@ public class JGroupsMessenger implements Messenger {
* Member identifiers already used, either in this JGroupsMessenger instance
* or in a past one & retained through an auto-reconnect.
*/
- private Set<GMSMember> usedMemberIdentifiers = new HashSet<>();
+ private Set<MemberIdentifier> usedMemberIdentifiers = new HashSet<>();
/**
* During reconnect a QuorumChecker holds the JGroups channel and responds to Ping
@@ -397,7 +399,7 @@ public class JGroupsMessenger implements Messenger {
}
@Override
- public boolean isOldMembershipIdentifier(GMSMember id) {
+ public boolean isOldMembershipIdentifier(MemberIdentifier id) {
return usedMemberIdentifiers.contains(id);
}
@@ -449,8 +451,8 @@ public class JGroupsMessenger implements Messenger {
public void stopped() {}
@Override
- public void memberSuspected(GMSMember initiator,
- GMSMember suspect, String reason) {}
+ public void memberSuspected(MemberIdentifier initiator,
+ MemberIdentifier suspect, String reason) {}
@Override
public void installView(GMSMembershipView v) {
@@ -470,10 +472,10 @@ public class JGroupsMessenger implements Messenger {
encrypt.installView(v);
}
synchronized (scheduledMcastSeqnos) {
- for (GMSMember mbr : v.getCrashedMembers()) {
+ for (MemberIdentifier mbr : v.getCrashedMembers()) {
scheduledMcastSeqnos.remove(mbr);
}
- for (GMSMember mbr : v.getShutdownMembers()) {
+ for (MemberIdentifier mbr : v.getShutdownMembers()) {
scheduledMcastSeqnos.remove(mbr);
}
}
@@ -495,12 +497,13 @@ public class JGroupsMessenger implements Messenger {
GMSMembershipView v = this.view;
JGAddress jgMbr = (JGAddress) dest;
if (jgMbr != null && v != null) {
- List<GMSMember> members = v.getMembers();
- GMSMember recipient = null;
- for (GMSMember gmsMbr : members) {
- if (jgMbr.getUUIDLsbs() == gmsMbr.getUuidLSBs()
- && jgMbr.getUUIDMsbs() == gmsMbr.getUuidMSBs()
- && jgMbr.getVmViewId() == gmsMbr.getVmViewId()) {
+ List<MemberIdentifier> members = v.getMembers();
+ MemberIdentifier recipient = null;
+ for (MemberIdentifier gmsMbr : members) {
+ MemberData memberData = gmsMbr.getMemberData();
+ if (jgMbr.getUUIDLsbs() == memberData.getUuidLSBs()
+ && jgMbr.getUUIDMsbs() == memberData.getUuidMSBs()
+ && jgMbr.getVmViewId() == memberData.getVmViewId()) {
recipient = gmsMbr;
break;
}
@@ -546,14 +549,14 @@ public class JGroupsMessenger implements Messenger {
MembershipConfig config = services.getConfig();
boolean isLocator = (config
- .getVmKind() == GMSMember.LOCATOR_DM_TYPE)
+ .getVmKind() == MemberIdentifier.LOCATOR_DM_TYPE)
|| !config.getStartLocator().isEmpty();
// establish the DistributedSystem's address
String hostname =
SocketCreator.resolve_dns ? SocketCreator.getHostName(jgAddress.getInetAddress())
: jgAddress.getInetAddress().getHostAddress();
- GMSMember gmsMember = new GMSMember(jgAddress.getInetAddress(),
+ GMSMemberData gmsMember = new GMSMemberData(jgAddress.getInetAddress(),
hostname, jgAddress.getPort(),
OSProcess.getId(), (byte) services.getConfig().getVmKind(),
-1 /* directport */, -1 /* viewID */, config.getName(),
@@ -561,11 +564,9 @@ public class JGroupsMessenger implements Messenger {
config.getDurableClientTimeout(),
config.getEnableNetworkPartitionDetection(), isLocator,
Version.getCurrentVersion().ordinal(),
- jgAddress.getUUIDMsbs(), jgAddress.getUUIDLsbs());
- gmsMember.setMemberWeight((byte) (services.getConfig().getMemberWeight() & 0xff));
- gmsMember.setNetworkPartitionDetectionEnabled(
- services.getConfig().getEnableNetworkPartitionDetection());
- localAddress = gmsMember;
+ jgAddress.getUUIDMsbs(), jgAddress.getUUIDLsbs(),
+ (byte) (services.getConfig().getMemberWeight() & 0xff));
+ localAddress = services.getMemberFactory().create(gmsMember);
logger.info("Established local address {}", localAddress);
services.setLocalAddress(localAddress);
}
@@ -604,7 +605,7 @@ public class JGroupsMessenger implements Messenger {
}
@Override
- public void getMessageState(GMSMember target, Map<String, Long> state,
+ public void getMessageState(MemberIdentifier target, Map<String, Long> state,
boolean includeMulticast) {
if (includeMulticast) {
NAKACK2 nakack = (NAKACK2) myChannel.getProtocolStack().findProtocol("NAKACK2");
@@ -616,7 +617,7 @@ public class JGroupsMessenger implements Messenger {
}
@Override
- public void waitForMessageState(GMSMember sender, Map<String, Long> state)
+ public void waitForMessageState(MemberIdentifier sender, Map<String, Long> state)
throws InterruptedException {
Long seqno = state.get("JGroups.mcastState");
if (seqno == null) {
@@ -664,16 +665,16 @@ public class JGroupsMessenger implements Messenger {
}
@Override
- public Set<GMSMember> sendUnreliably(GMSMessage msg) {
+ public Set<MemberIdentifier> sendUnreliably(GMSMessage msg) {
return send(msg, false);
}
@Override
- public Set<GMSMember> send(GMSMessage msg) {
+ public Set<MemberIdentifier> send(GMSMessage msg) {
return send(msg, true);
}
- private Set<GMSMember> send(GMSMessage msg, boolean reliably) {
+ private Set<MemberIdentifier> send(GMSMessage msg, boolean reliably) {
// perform the same jgroups messaging as in 8.2's GMSMembershipManager.send() method
@@ -691,7 +692,7 @@ public class JGroupsMessenger implements Messenger {
filterOutgoingMessage(msg);
- List<GMSMember> destinations = msg.getRecipients();
+ List<MemberIdentifier> destinations = msg.getRecipients();
boolean allDestinations = msg.forAll();
boolean useMcast = false;
@@ -753,7 +754,7 @@ public class JGroupsMessenger implements Messenger {
} // useMcast
else { // ! useMcast
int len = destinations.size();
- List<GMSMember> calculatedMembers; // explicit list of members
+ List<MemberIdentifier> calculatedMembers; // explicit list of members
int calculatedLen; // == calculatedMembers.len
if (len == 1 && destinations.get(0) == ALL_RECIPIENTS) { // send to all
// Grab a copy of the current membership
@@ -761,23 +762,23 @@ public class JGroupsMessenger implements Messenger {
// Construct the list
calculatedLen = v.size();
- calculatedMembers = new LinkedList<GMSMember>();
+ calculatedMembers = new LinkedList<MemberIdentifier>();
for (int i = 0; i < calculatedLen; i++) {
- GMSMember m = (GMSMember) v.get(i);
- calculatedMembers.add((GMSMember) m);
+ MemberIdentifier m = (MemberIdentifier) v.get(i);
+ calculatedMembers.add((MemberIdentifier) m);
}
} // send to all
else { // send to explicit list
calculatedLen = len;
- calculatedMembers = new LinkedList<GMSMember>();
+ calculatedMembers = new LinkedList<MemberIdentifier>();
for (int i = 0; i < calculatedLen; i++) {
- calculatedMembers.add((GMSMember) destinations.get(i));
+ calculatedMembers.add((MemberIdentifier) destinations.get(i));
}
} // send to explicit list
Int2ObjectOpenHashMap<Message> messages = new Int2ObjectOpenHashMap<>();
long startSer = theStats.startMsgSerialization();
boolean firstMessage = true;
- for (GMSMember mbr : calculatedMembers) {
+ for (MemberIdentifier mbr : calculatedMembers) {
short version = mbr.getVersionOrdinal();
if (!messages.containsKey(version)) {
Message jmsg = createJGMessage(msg, local, mbr, version);
@@ -791,7 +792,7 @@ public class JGroupsMessenger implements Messenger {
theStats.endMsgSerialization(startSer);
Collections.shuffle(calculatedMembers);
int i = 0;
- for (GMSMember mbr : calculatedMembers) {
+ for (MemberIdentifier mbr : calculatedMembers) {
JGAddress to = new JGAddress(mbr);
short version = mbr.getVersionOrdinal();
Message jmsg = messages.get(version);
@@ -836,10 +837,10 @@ public class JGroupsMessenger implements Messenger {
if (msg.forAll()) {
return Collections.emptySet();
}
- Set<GMSMember> result = new HashSet<>();
+ Set<MemberIdentifier> result = new HashSet<>();
GMSMembershipView newView = this.view;
if (newView != null && newView != oldView) {
- for (GMSMember d : destinations) {
+ for (MemberIdentifier d : destinations) {
if (!newView.contains(d)) {
logger.debug("messenger: member has left the view: {} view is now {}", d, newView);
result.add(d);
@@ -859,7 +860,7 @@ public class JGroupsMessenger implements Messenger {
* @param version the version of the recipient
* @return the new message
*/
- Message createJGMessage(GMSMessage gfmsg, JGAddress src, GMSMember dst, short version) {
+ Message createJGMessage(GMSMessage gfmsg, JGAddress src, MemberIdentifier dst, short version) {
gfmsg.registerProcessor();
Message msg = new Message();
msg.setDest(null);
@@ -899,7 +900,7 @@ public class JGroupsMessenger implements Messenger {
return msg;
}
- void writeEncryptedMessage(GMSMessage gfmsg, GMSMember recipient, short version,
+ void writeEncryptedMessage(GMSMessage gfmsg, MemberIdentifier recipient, short version,
BufferDataOutputStream out)
throws Exception {
long start = services.getStatistics().startUDPMsgEncryption();
@@ -907,7 +908,7 @@ public class JGroupsMessenger implements Messenger {
services.getSerializer().writeDSFIDHeader(gfmsg.getDSFID(), out);
byte[] pk = null;
int requestId = 0;
- GMSMember pkMbr = null;
+ MemberIdentifier pkMbr = null;
switch (gfmsg.getDSFID()) {
case FIND_COORDINATOR_REQ:
case JOIN_REQUEST:
@@ -949,7 +950,7 @@ public class JGroupsMessenger implements Messenger {
}
}
- int getRequestId(GMSMessage gfmsg, GMSMember destination, boolean add) {
+ int getRequestId(GMSMessage gfmsg, MemberIdentifier destination, boolean add) {
int requestId = 0;
if (gfmsg instanceof FindCoordinatorRequest) {
requestId = ((FindCoordinatorRequest) gfmsg).getRequestId();
@@ -970,8 +971,8 @@ public class JGroupsMessenger implements Messenger {
byte[] serializeMessage(GMSMessage gfmsg, BufferDataOutputStream out_stream)
throws IOException {
- GMSMember m = this.localAddress;
- m.writeEssentialData(out_stream,
+ MemberIdentifier m = this.localAddress;
+ m.getMemberData().writeEssentialData(out_stream,
services.getSerializer().createSerializationContext(out_stream));
services.getSerializer().getObjectSerializer()
.writeObject(services.getManager().unwrapMessage(gfmsg), out_stream);
@@ -1058,8 +1059,8 @@ public class JGroupsMessenger implements Messenger {
return result;
}
- void setSender(GMSMessage dm, GMSMember m, short ordinal) {
- GMSMember sender = null;
+ void setSender(GMSMessage dm, MemberIdentifier m, short ordinal) {
+ MemberIdentifier sender = null;
// JoinRequestMessages are sent with an ID that may have been
// reused from a previous life by way of auto-reconnect,
// so we don't want to find a canonical reference for the
@@ -1083,7 +1084,7 @@ public class JGroupsMessenger implements Messenger {
logger.debug("readEncryptedMessage Reading Request id " + dfsid + " and requestid is "
+ requestId + " myid " + this.localAddress);
}
- GMSMember pkMbr = null;
+ MemberIdentifier pkMbr = null;
boolean readPK = false;
switch (dfsid) {
case FIND_COORDINATOR_REQ:
@@ -1130,7 +1131,7 @@ public class JGroupsMessenger implements Messenger {
if (pk != null) {
logger.info("Setting public key for " + result.getSender() + " len " + pk.length);
- setPublicKey(pk, (GMSMember) result.getSender());
+ setPublicKey(pk, (MemberIdentifier) result.getSender());
}
return result;
@@ -1145,8 +1146,9 @@ public class JGroupsMessenger implements Messenger {
GMSMessage deserializeMessage(DataInputStream in, short ordinal)
throws ClassNotFoundException, IOException {
- GMSMember m = new GMSMember();
- m.readEssentialData(in, services.getSerializer().createDeserializationContext(in));
+ GMSMemberData info = new GMSMemberData();
+ info.readEssentialData(in, services.getSerializer().createDeserializationContext(in));
+ MemberIdentifier m = services.getMemberFactory().create(info);
GMSMessage result = services.getManager()
.wrapMessage(services.getSerializer().getObjectDeserializer().readObject(in));
@@ -1208,7 +1210,7 @@ public class JGroupsMessenger implements Messenger {
}
@Override
- public GMSMember getMemberID() {
+ public MemberIdentifier getMemberID() {
return localAddress;
}
@@ -1216,7 +1218,7 @@ public class JGroupsMessenger implements Messenger {
* returns the member ID for the given GMSMember object
*/
@SuppressWarnings("UnusedParameters")
- private GMSMember getMemberFromView(GMSMember jgId, short version) {
+ private MemberIdentifier getMemberFromView(MemberIdentifier jgId, short version) {
return this.services.getJoinLeave().getMemberID(jgId);
}
@@ -1306,7 +1308,7 @@ public class JGroupsMessenger implements Messenger {
// multicast to them, avoiding deserialization cost and classpath
// problems
if ((services.getConfig()
- .getVmKind() == GMSMember.ADMIN_ONLY_DM_TYPE)
+ .getVmKind() == MemberIdentifier.ADMIN_ONLY_DM_TYPE)
&& (msg instanceof DistributedCacheOperation.CacheOperationMessage)) {
return;
}
@@ -1343,7 +1345,7 @@ public class JGroupsMessenger implements Messenger {
}
}
- private void recordScheduledSeqno(GMSMember member, long seqno) {
+ private void recordScheduledSeqno(MemberIdentifier member, long seqno) {
synchronized (scheduledMcastSeqnos) {
MessageTracker counter = scheduledMcastSeqnos.get(member);
if (counter == null) {
@@ -1378,7 +1380,7 @@ public class JGroupsMessenger implements Messenger {
}
@Override
- public Set<GMSMember> send(GMSMessage msg, GMSMembershipView alternateView) {
+ public Set<MemberIdentifier> send(GMSMessage msg, GMSMembershipView alternateView) {
if (this.encrypt != null) {
this.encrypt.installView(alternateView);
}
@@ -1386,7 +1388,7 @@ public class JGroupsMessenger implements Messenger {
}
@Override
- public byte[] getPublicKey(GMSMember mbr) {
+ public byte[] getPublicKey(MemberIdentifier mbr) {
if (encrypt != null) {
return encrypt.getPublicKey(mbr);
}
@@ -1394,7 +1396,7 @@ public class JGroupsMessenger implements Messenger {
}
@Override
- public void setPublicKey(byte[] publickey, GMSMember mbr) {
+ public void setPublicKey(byte[] publickey, MemberIdentifier mbr) {
if (encrypt != null) {
logger.debug("Setting PK for member " + mbr);
encrypt.setPublicKey(publickey, mbr);
@@ -1418,13 +1420,13 @@ public class JGroupsMessenger implements Messenger {
}
private AtomicInteger requestId = new AtomicInteger((new Random().nextInt()));
- private HashMap<Integer, GMSMember> requestIdVsRecipients = new HashMap<>();
+ private HashMap<Integer, MemberIdentifier> requestIdVsRecipients = new HashMap<>();
- GMSMember getRequestedMember(int requestId) {
+ MemberIdentifier getRequestedMember(int requestId) {
return requestIdVsRecipients.remove(requestId);
}
- void addRequestId(int requestId, GMSMember mbr) {
+ void addRequestId(int requestId, MemberIdentifier mbr) {
requestIdVsRecipients.put(requestId, mbr);
}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/MembershipInformation.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/MembershipInformation.java
index 85be721..88b33f9 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/MembershipInformation.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messenger/MembershipInformation.java
@@ -20,7 +20,7 @@ import java.util.Set;
import org.jgroups.JChannel;
import org.jgroups.Message;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
/**
* Class MembershipInformation is used to pass membership data from a GMS that was
@@ -28,11 +28,11 @@ import org.apache.geode.distributed.internal.membership.gms.GMSMember;
*/
public class MembershipInformation {
private final JChannel channel;
- private final Set<GMSMember> membershipIdentifiers;
+ private final Set<MemberIdentifier> membershipIdentifiers;
private final Queue<Message> queuedMessages;
protected MembershipInformation(JChannel channel,
- Set<GMSMember> oldMembershipIdentifiers,
+ Set<MemberIdentifier> oldMembershipIdentifiers,
Queue<Message> queuedMessages) {
this.channel = channel;
@@ -44,7 +44,7 @@ public class MembershipInformation {
return channel;
}
- public Set<GMSMember> getMembershipIdentifiers() {
+ public Set<MemberIdentifier> getMembershipIdentifiers() {
return membershipIdentifiers;
}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/DSFIDFactory.java b/geode-core/src/main/java/org/apache/geode/internal/DSFIDFactory.java
index 39d4879..8ed861d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DSFIDFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DSFIDFactory.java
@@ -79,7 +79,6 @@ import org.apache.geode.distributed.internal.locks.GrantorRequestProcessor;
import org.apache.geode.distributed.internal.locks.NonGrantorDestroyedProcessor;
import org.apache.geode.distributed.internal.locks.NonGrantorDestroyedProcessor.NonGrantorDestroyedReplyMessage;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorRequest;
import org.apache.geode.distributed.internal.membership.gms.locator.FindCoordinatorResponse;
@@ -462,7 +461,6 @@ public class DSFIDFactory implements DataSerializableFixedID {
serializer.registerDSFID(LEAVE_REQUEST_MESSAGE, LeaveRequestMessage.class);
serializer.registerDSFID(VIEW_ACK_MESSAGE, ViewAckMessage.class);
serializer.registerDSFID(INSTALL_VIEW_MESSAGE, InstallViewMessage.class);
- serializer.registerDSFID(GMSMEMBER, GMSMember.class);
serializer.registerDSFID(NETVIEW, GMSMembershipView.class);
serializer.registerDSFID(GET_VIEW_REQ, GetViewRequest.class);
serializer.registerDSFID(GET_VIEW_RESP, GetViewResponse.class);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java
index be617c4..1ab307a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/CacheServerImpl.java
@@ -58,7 +58,7 @@ import org.apache.geode.distributed.internal.DistributionManager;
import org.apache.geode.distributed.internal.InternalDistributedSystem;
import org.apache.geode.distributed.internal.ResourceEvent;
import org.apache.geode.distributed.internal.ServerLocation;
-import org.apache.geode.distributed.internal.membership.MemberAttributes;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberDataBuilder;
import org.apache.geode.internal.Assert;
import org.apache.geode.internal.OSProcess;
import org.apache.geode.internal.admin.ClientHealthMonitoringRegion;
@@ -751,7 +751,7 @@ public class CacheServerImpl extends AbstractCacheServer implements Distribution
public String[] getCombinedGroups() {
ArrayList<String> groupList = new ArrayList<String>();
if (includeMembershipGroups) {
- for (String g : MemberAttributes.parseGroups(null, getSystem().getConfig().getGroups())) {
+ for (String g : MemberDataBuilder.parseGroups(null, getSystem().getConfig().getGroups())) {
if (!groupList.contains(g)) {
groupList.add(g);
}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/EventStateHelper.java b/geode-core/src/main/java/org/apache/geode/internal/cache/EventStateHelper.java
index 4b832b8..b7d8c54 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/EventStateHelper.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/EventStateHelper.java
@@ -56,14 +56,14 @@ public class EventStateHelper {
InternalDistributedMember myId) throws IOException {
// For HARegionQueues, the event state map is uses different values
// than a regular region :(
- Map<MemberIdentifier, Map<ThreadIdentifier, Object>> groupedThreadIds =
+ Map<EventStateMemberIdentifier, Map<ThreadIdentifier, Object>> groupedThreadIds =
groupThreadIds(eventState);
- List<MemberIdentifier> orderedIds = new LinkedList();
- Map<MemberIdentifier, Integer> seenIds = new HashMap();
+ List<EventStateMemberIdentifier> orderedIds = new LinkedList();
+ Map<EventStateMemberIdentifier, Integer> seenIds = new HashMap();
myId.writeEssentialData(dop); // added in 7.0 for version tag processing in fromData
- for (MemberIdentifier memberId : groupedThreadIds.keySet()) {
+ for (EventStateMemberIdentifier memberId : groupedThreadIds.keySet()) {
if (!seenIds.containsKey(memberId)) {
orderedIds.add(memberId);
seenIds.put(memberId, Integer.valueOf(seenIds.size()));
@@ -71,14 +71,14 @@ public class EventStateHelper {
}
dop.writeInt(seenIds.size());
- for (MemberIdentifier memberId : orderedIds) {
+ for (EventStateMemberIdentifier memberId : orderedIds) {
DataSerializer.writeByteArray(memberId.bytes, dop);
}
dop.writeInt(groupedThreadIds.size());
- for (Map.Entry<MemberIdentifier, Map<ThreadIdentifier, Object>> memberIdEntry : groupedThreadIds
+ for (Map.Entry<EventStateMemberIdentifier, Map<ThreadIdentifier, Object>> memberIdEntry : groupedThreadIds
.entrySet()) {
- MemberIdentifier memberId = memberIdEntry.getKey();
+ EventStateMemberIdentifier memberId = memberIdEntry.getKey();
dop.writeInt(seenIds.get(memberId).intValue());
Map<ThreadIdentifier, Object> threadIdMap = memberIdEntry.getValue();
dop.writeInt(threadIdMap.size());
@@ -146,14 +146,14 @@ public class EventStateHelper {
}
- private static Map<MemberIdentifier, Map<ThreadIdentifier, Object>> groupThreadIds(
+ private static Map<EventStateMemberIdentifier, Map<ThreadIdentifier, Object>> groupThreadIds(
Map eventState) {
- Map<MemberIdentifier, Map<ThreadIdentifier, Object>> results =
- new HashMap<MemberIdentifier, Map<ThreadIdentifier, Object>>();
+ Map<EventStateMemberIdentifier, Map<ThreadIdentifier, Object>> results =
+ new HashMap<EventStateMemberIdentifier, Map<ThreadIdentifier, Object>>();
for (Object next : eventState.entrySet()) {
Map.Entry entry = (Map.Entry) next;
ThreadIdentifier key = (ThreadIdentifier) entry.getKey();
- MemberIdentifier memberId = new MemberIdentifier(key.getMembershipID());
+ EventStateMemberIdentifier memberId = new EventStateMemberIdentifier(key.getMembershipID());
Object value = entry.getValue();
Map<ThreadIdentifier, Object> subMap = results.get(memberId);
if (subMap == null) {
@@ -166,10 +166,10 @@ public class EventStateHelper {
return results;
}
- private static class MemberIdentifier {
+ private static class EventStateMemberIdentifier {
private final byte[] bytes;
- public MemberIdentifier(byte[] bytes) {
+ public EventStateMemberIdentifier(byte[] bytes) {
this.bytes = bytes;
}
@@ -187,9 +187,9 @@ public class EventStateHelper {
return true;
if (obj == null)
return false;
- if (!(obj instanceof MemberIdentifier))
+ if (!(obj instanceof EventStateMemberIdentifier))
return false;
- MemberIdentifier other = (MemberIdentifier) obj;
+ EventStateMemberIdentifier other = (EventStateMemberIdentifier) obj;
if (!Arrays.equals(bytes, other.bytes))
return false;
return true;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
index 33c4fb9..a4c1083 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
@@ -155,7 +155,7 @@ import org.apache.geode.distributed.internal.ReplyProcessor21;
import org.apache.geode.distributed.internal.locks.DLockRemoteToken;
import org.apache.geode.distributed.internal.locks.DLockService;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.MemberAttributes;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberDataBuilder;
import org.apache.geode.internal.Assert;
import org.apache.geode.internal.NanoTimer;
import org.apache.geode.internal.cache.BucketAdvisor.ServerBucketProfile;
@@ -5504,7 +5504,7 @@ public class PartitionedRegion extends LocalRegion
servers = cache.getCacheServers();
- Collections.addAll(localServerGroups, MemberAttributes.parseGroups(null,
+ Collections.addAll(localServerGroups, MemberDataBuilder.parseGroups(null,
cache.getInternalDistributedSystem().getConfig().getGroups()));
for (Object object : servers) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientUpdater.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientUpdater.java
index f294470..35c7054 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientUpdater.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientUpdater.java
@@ -54,14 +54,12 @@ import org.apache.geode.cache.client.internal.PoolImpl;
import org.apache.geode.cache.client.internal.QueueManager;
import org.apache.geode.cache.query.internal.cq.CqService;
import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.ClusterDistributionManager;
import org.apache.geode.distributed.internal.DistributionConfig;
import org.apache.geode.distributed.internal.DistributionStats;
import org.apache.geode.distributed.internal.InternalDistributedSystem;
import org.apache.geode.distributed.internal.InternalDistributedSystem.DisconnectListener;
import org.apache.geode.distributed.internal.ServerLocation;
import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.MemberAttributes;
import org.apache.geode.internal.Assert;
import org.apache.geode.internal.InternalDataSerializer;
import org.apache.geode.internal.InternalInstantiator;
@@ -361,14 +359,8 @@ public class CacheClientUpdater extends LoggingThread implements ClientUpdater,
}
cb = ServerConnection.allocateCommBuffer(bufSize, mySock);
- // create a "server" memberId we currently don't know much about the server.
- // Would be nice for it to send us its member id
- // TODO: change the serverId to use the endpoint's getMemberId() which returns a
- // DistributedMember (once gfecq branch is merged to trunk).
- MemberAttributes ma = new MemberAttributes(0, -1, ClusterDistributionManager.NORMAL_DM_TYPE,
- -1, null, null, null);
sid =
- new InternalDistributedMember(mySock.getInetAddress(), mySock.getPort(), false, true, ma);
+ new InternalDistributedMember(mySock.getInetAddress(), mySock.getPort(), false, true);
success = true;
} catch (ConnectException ignore) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyMembershipID.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyMembershipID.java
index deef598..1c84c7f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyMembershipID.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ClientProxyMembershipID.java
@@ -463,7 +463,7 @@ public class ClientProxyMembershipID
*/
public void updateDurableTimeout(int newValue) {
InternalDistributedMember member = (InternalDistributedMember) getDistributedMember();
- member.getNetMember().setDurableTimeout(newValue);
+ member.getMemberData().setDurableTimeout(newValue);
}
/**
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/InternalDistributedMemberTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/InternalDistributedMemberTest.java
index 5fa0dc4..0f2afa8 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/InternalDistributedMemberTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/InternalDistributedMemberTest.java
@@ -24,6 +24,7 @@ import java.net.UnknownHostException;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.apache.geode.distributed.internal.membership.gms.GMSMemberData;
import org.apache.geode.test.junit.categories.MembershipTest;
/**
@@ -72,9 +73,9 @@ public class InternalDistributedMemberTest {
@Test
public void equalsReturnsTrueIfNullInetAddress() throws UnknownHostException {
InetAddress localHost = InetAddress.getLocalHost();
- NetMember netMember1 = mock(NetMember.class);
+ GMSMemberData netMember1 = mock(GMSMemberData.class);
when(netMember1.getInetAddress()).thenReturn(localHost).thenReturn(null);
- NetMember netMember2 = mock(NetMember.class);
+ GMSMemberData netMember2 = mock(GMSMemberData.class);
when(netMember2.getInetAddress()).thenReturn(localHost).thenReturn(null);
InternalDistributedMember member = new InternalDistributedMember(netMember1);
InternalDistributedMember other = new InternalDistributedMember(netMember2);
@@ -88,9 +89,9 @@ public class InternalDistributedMemberTest {
public void equalsReturnsFalseIfInetAddressDiffer() throws UnknownHostException {
InetAddress host1 = InetAddress.getByAddress(new byte[] {127, 0, 0, 1});
InetAddress host2 = InetAddress.getByAddress(new byte[] {127, 0, 0, 2});
- NetMember netMember1 = mock(NetMember.class);
+ GMSMemberData netMember1 = mock(GMSMemberData.class);
when(netMember1.getInetAddress()).thenReturn(host1);
- NetMember netMember2 = mock(NetMember.class);
+ GMSMemberData netMember2 = mock(GMSMemberData.class);
when(netMember2.getInetAddress()).thenReturn(host2);
InternalDistributedMember member = new InternalDistributedMember(netMember1);
InternalDistributedMember other = new InternalDistributedMember(netMember2);
@@ -103,9 +104,9 @@ public class InternalDistributedMemberTest {
@Test
public void equalsReturnsFalseIfGetViewIdDiffer() throws UnknownHostException {
InetAddress host1 = InetAddress.getByAddress(new byte[] {127, 0, 0, 1});
- NetMember netMember1 = mock(NetMember.class);
+ GMSMemberData netMember1 = mock(GMSMemberData.class);
when(netMember1.getInetAddress()).thenReturn(host1);
- NetMember netMember2 = mock(NetMember.class);
+ GMSMemberData netMember2 = mock(GMSMemberData.class);
when(netMember2.getInetAddress()).thenReturn(host1);
when(netMember1.getVmViewId()).thenReturn(1);
when(netMember2.getVmViewId()).thenReturn(2);
@@ -120,9 +121,9 @@ public class InternalDistributedMemberTest {
@Test
public void equalsReturnsFalseIfUniqueTagsDiffer() throws UnknownHostException {
InetAddress host1 = InetAddress.getByAddress(new byte[] {127, 0, 0, 1});
- NetMember netMember1 = mock(NetMember.class);
+ GMSMemberData netMember1 = mock(GMSMemberData.class);
when(netMember1.getInetAddress()).thenReturn(host1);
- NetMember netMember2 = mock(NetMember.class);
+ GMSMemberData netMember2 = mock(GMSMemberData.class);
when(netMember2.getInetAddress()).thenReturn(host1);
InternalDistributedMember member = new InternalDistributedMember(netMember1);
member.setUniqueTag("tag1");
@@ -137,10 +138,10 @@ public class InternalDistributedMemberTest {
@Test
public void equalsReturnsFalseIfNotPartialAndNamesDiffer() throws UnknownHostException {
InetAddress host1 = InetAddress.getByAddress(new byte[] {127, 0, 0, 1});
- NetMember netMember1 = mock(NetMember.class);
+ GMSMemberData netMember1 = mock(GMSMemberData.class);
when(netMember1.getInetAddress()).thenReturn(host1);
when(netMember1.getName()).thenReturn("name1");
- NetMember netMember2 = mock(NetMember.class);
+ GMSMemberData netMember2 = mock(GMSMemberData.class);
when(netMember2.getInetAddress()).thenReturn(host1);
when(netMember2.getName()).thenReturn("name2");
InternalDistributedMember member = new InternalDistributedMember(netMember1);
@@ -156,9 +157,9 @@ public class InternalDistributedMemberTest {
@Test
public void equalsReturnsFalseIfCompareAdditionalDataDiffer() throws UnknownHostException {
InetAddress host1 = InetAddress.getByAddress(new byte[] {127, 0, 0, 1});
- NetMember netMember1 = mock(NetMember.class);
+ GMSMemberData netMember1 = mock(GMSMemberData.class);
when(netMember1.getInetAddress()).thenReturn(host1);
- NetMember netMember2 = mock(NetMember.class);
+ GMSMemberData netMember2 = mock(GMSMemberData.class);
when(netMember2.getInetAddress()).thenReturn(host1);
when(netMember1.compareAdditionalData(netMember2)).thenReturn(1);
when(netMember2.compareAdditionalData(netMember1)).thenReturn(-1);
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/MembershipViewJUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/MembershipViewJUnitTest.java
index 01044ad..770fcae 100755
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/MembershipViewJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/MembershipViewJUnitTest.java
@@ -31,41 +31,39 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.membership.adapter.GMSMemberAdapter;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
import org.apache.geode.distributed.internal.membership.gms.GMSMembershipView;
+import org.apache.geode.distributed.internal.membership.gms.api.MemberIdentifier;
import org.apache.geode.internal.net.SocketCreator;
import org.apache.geode.test.junit.categories.MembershipTest;
@Category({MembershipTest.class})
public class MembershipViewJUnitTest {
- private List<GMSMember> members;
+ private List<MemberIdentifier> members;
@Before
public void initMembers() throws Exception {
int numMembers = 10;
members = new ArrayList<>(numMembers);
for (int i = 0; i < numMembers; i++) {
- members.add(
- ((GMSMemberAdapter) new InternalDistributedMember(SocketCreator.getLocalHost(), 1000 + i)
- .getNetMember()).getGmsMember());
+ members.add(new InternalDistributedMember(SocketCreator.getLocalHost(), 1000 + i));
}
// view creator is a locator
- members.get(0).setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- members.get(0).setVmViewId(0);
- members.get(0).setPreferredForCoordinator(true);
+ members.get(0).getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ members.get(0).getMemberData().setVmViewId(0);
+ members.get(0).getMemberData().setPreferredForCoordinator(true);
// members who joined in view #1
for (int i = 1; i < (numMembers - 1); i++) {
- members.get(i).setVmViewId(1);
- members.get(i).setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
- members.get(i).setPreferredForCoordinator(false);
+ members.get(i).getMemberData().setVmViewId(1);
+ members.get(i).getMemberData().setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
+ members.get(i).getMemberData().setPreferredForCoordinator(false);
}
// member joining in this view
- members.get(numMembers - 1).setVmViewId(2);
- members.get(numMembers - 1).setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
+ members.get(numMembers - 1).getMemberData().setVmViewId(2);
+ members.get(numMembers - 1).getMemberData()
+ .setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
}
private void setFailureDetectionPorts(GMSMembershipView view) {
@@ -93,7 +91,7 @@ public class MembershipViewJUnitTest {
assertEquals(members.get(0), view.getCoordinator());
for (int i = 0; i < numMembers; i++) {
- GMSMember mbr = members.get(i);
+ MemberIdentifier mbr = members.get(i);
assertEquals(mbr.getPort(), view.getFailureDetectionPort(mbr));
}
@@ -101,12 +99,12 @@ public class MembershipViewJUnitTest {
assertTrue(view.shouldBeCoordinator(members.get(0)));
assertEquals(members.get(numMembers - 1),
view.getCoordinator(Collections.singletonList(members.get(0))));
- members.get(numMembers - 1).setPreferredForCoordinator(false);
+ members.get(numMembers - 1).getMemberData().setPreferredForCoordinator(false);
assertEquals(members.get(1), view.getCoordinator(Collections.singletonList(members.get(0))));
- members.get(numMembers - 1).setPreferredForCoordinator(true);
- List<GMSMember> preferred = view.getPreferredCoordinators(
- Collections.<GMSMember>singleton(members.get(1)), members.get(0), 2);
+ members.get(numMembers - 1).getMemberData().setPreferredForCoordinator(true);
+ List<MemberIdentifier> preferred = view.getPreferredCoordinators(
+ Collections.<MemberIdentifier>singleton(members.get(1)), members.get(0), 2);
assertEquals(2, preferred.size());
assertEquals(members.get(numMembers - 1), preferred.get(0));
}
@@ -122,11 +120,11 @@ public class MembershipViewJUnitTest {
assertFalse(view.contains(members.get(i)));
}
- List<GMSMember> remainingMembers = view.getMembers();
+ List<MemberIdentifier> remainingMembers = view.getMembers();
int num = remainingMembers.size();
for (int i = 0; i < num; i++) {
- GMSMember mbr = remainingMembers.get(i);
- assertEquals(mbr.getPort(), view.getFailureDetectionPort((GMSMember) mbr));
+ MemberIdentifier mbr = remainingMembers.get(i);
+ assertEquals(mbr.getPort(), view.getFailureDetectionPort((MemberIdentifier) mbr));
}
}
@@ -136,22 +134,22 @@ public class MembershipViewJUnitTest {
GMSMembershipView view = new GMSMembershipView(members.get(0), 2, new ArrayList<>(members));
setFailureDetectionPorts(view);
- Collection<GMSMember> removals = new ArrayList<>(numMembers / 2);
+ Collection<MemberIdentifier> removals = new ArrayList<>(numMembers / 2);
for (int i = 1; i < numMembers; i += 2) {
removals.add(members.get(i));
}
view.removeAll(removals);
- for (GMSMember mbr : removals) {
+ for (MemberIdentifier mbr : removals) {
assertFalse(view.contains(mbr));
}
assertEquals(numMembers - removals.size(), view.size());
- List<GMSMember> remainingMembers = view.getMembers();
+ List<MemberIdentifier> remainingMembers = view.getMembers();
int num = remainingMembers.size();
for (int i = 0; i < num; i++) {
- GMSMember mbr = remainingMembers.get(i);
- assertEquals(mbr.getPort(), view.getFailureDetectionPort((GMSMember) mbr));
+ MemberIdentifier mbr = remainingMembers.get(i);
+ assertEquals(mbr.getPort(), view.getFailureDetectionPort((MemberIdentifier) mbr));
}
}
@@ -184,17 +182,15 @@ public class MembershipViewJUnitTest {
int oldSize = view.size();
for (int i = 0; i < 100; i++) {
- GMSMember mbr =
- ((GMSMemberAdapter) new InternalDistributedMember(SocketCreator.getLocalHost(), 2000 + i)
- .getNetMember()).getGmsMember();
- mbr.setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
- mbr.setVmViewId(2);
+ MemberIdentifier mbr = new InternalDistributedMember(SocketCreator.getLocalHost(), 2000 + i);
+ mbr.getMemberData().setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
+ mbr.getMemberData().setVmViewId(2);
view.add(mbr);
view.setFailureDetectionPort(mbr, 2000 + i);
}
assertEquals(oldSize + 100, view.size());
- for (GMSMember mbr : view.getMembers()) {
+ for (MemberIdentifier mbr : view.getMembers()) {
assertEquals(mbr.getPort(), view.getFailureDetectionPort(mbr));
}
@@ -207,13 +203,13 @@ public class MembershipViewJUnitTest {
setFailureDetectionPorts(view);
GMSMembershipView newView = new GMSMembershipView(view, 3);
- for (GMSMember member : view.getMembers()) {
- view.setPublicKey((GMSMember) member, null);
+ for (MemberIdentifier member : view.getMembers()) {
+ view.setPublicKey((MemberIdentifier) member, null);
}
newView.setPublicKeys(view);
- for (GMSMember member : view.getMembers()) {
- assertNull(newView.getPublicKey((GMSMember) member));
- assertNull(view.getPublicKey((GMSMember) member));
+ for (MemberIdentifier member : view.getMembers()) {
+ assertNull(newView.getPublicKey((MemberIdentifier) member));
+ assertNull(view.getPublicKey((MemberIdentifier) member));
}
}
@@ -225,75 +221,68 @@ public class MembershipViewJUnitTest {
// in #47342 a new view was created that contained a member that was joining but
// was no longer reachable. The member was included in the failed-weight and not
// in the previous view-weight, causing a spurious network partition to be declared
- GMSMember members[] =
- new GMSMember[] {
- ((GMSMemberAdapter) new InternalDistributedMember("localhost", 1).getNetMember())
- .getGmsMember(),
- ((GMSMemberAdapter) new InternalDistributedMember("localhost", 2).getNetMember())
- .getGmsMember(),
- ((GMSMemberAdapter) new InternalDistributedMember("localhost", 3).getNetMember())
- .getGmsMember(),
- ((GMSMemberAdapter) new InternalDistributedMember("localhost", 4).getNetMember())
- .getGmsMember(),
- ((GMSMemberAdapter) new InternalDistributedMember("localhost", 5).getNetMember())
- .getGmsMember(),
- ((GMSMemberAdapter) new InternalDistributedMember("localhost", 6).getNetMember())
- .getGmsMember()};
+ MemberIdentifier members[] =
+ new MemberIdentifier[] {
+ new InternalDistributedMember("localhost", 1),
+ new InternalDistributedMember("localhost", 2),
+ new InternalDistributedMember("localhost", 3),
+ new InternalDistributedMember("localhost", 4),
+ new InternalDistributedMember("localhost", 5),
+ new InternalDistributedMember("localhost", 6)};
int i = 0;
// weight 3
- members[i].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- members[i++].setPreferredForCoordinator(true);
+ members[i].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ members[i++].getMemberData().setPreferredForCoordinator(true);
// weight 3
- members[i].setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
- members[i++].setPreferredForCoordinator(true);
+ members[i].getMemberData().setVmKind(ClusterDistributionManager.LOCATOR_DM_TYPE);
+ members[i++].getMemberData().setPreferredForCoordinator(true);
// weight 15 (cache+leader)
- members[i].setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
- members[i++].setPreferredForCoordinator(false);
+ members[i].getMemberData().setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
+ members[i++].getMemberData().setPreferredForCoordinator(false);
// weight 0
- members[i].setVmKind(ClusterDistributionManager.ADMIN_ONLY_DM_TYPE);
- members[i++].setPreferredForCoordinator(false);
+ members[i].getMemberData().setVmKind(ClusterDistributionManager.ADMIN_ONLY_DM_TYPE);
+ members[i++].getMemberData().setPreferredForCoordinator(false);
// weight 0
- members[i].setVmKind(ClusterDistributionManager.ADMIN_ONLY_DM_TYPE);
- members[i++].setPreferredForCoordinator(false);
+ members[i].getMemberData().setVmKind(ClusterDistributionManager.ADMIN_ONLY_DM_TYPE);
+ members[i++].getMemberData().setPreferredForCoordinator(false);
// weight 10
- members[i].setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
- members[i++].setPreferredForCoordinator(false);
+ members[i].getMemberData().setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
+ members[i++].getMemberData().setPreferredForCoordinator(false);
- List<GMSMember> vmbrs = new ArrayList<>(members.length);
+ List<MemberIdentifier> vmbrs = new ArrayList<>(members.length);
for (i = 0; i < members.length; i++) {
vmbrs.add(members[i]);
}
GMSMembershipView lastView =
- new GMSMembershipView((GMSMember) members[0], 4, (List<GMSMember>) (List<?>) vmbrs);
- GMSMember leader = (GMSMember) members[2];
+ new GMSMembershipView((MemberIdentifier) members[0], 4,
+ (List<MemberIdentifier>) (List<?>) vmbrs);
+ MemberIdentifier leader = (MemberIdentifier) members[2];
assertTrue(!leader.preferredForCoordinator());
- GMSMember joiningMember =
- ((GMSMemberAdapter) new InternalDistributedMember("localhost", 7).getNetMember())
- .getGmsMember();
- joiningMember.setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
- joiningMember.setPreferredForCoordinator(false);
+ MemberIdentifier joiningMember = new InternalDistributedMember("localhost", 7);
+ joiningMember.getMemberData().setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
+ joiningMember.getMemberData().setPreferredForCoordinator(false);
// have the joining member and another cache process (weight 10) in the failed members
// collection and check to make sure that the joining member is not included in failed
// weight calcs.
- Set<GMSMember> failedMembers = new HashSet<>(3);
+ Set<MemberIdentifier> failedMembers = new HashSet<>(3);
failedMembers.add(joiningMember);
failedMembers.add(members[members.length - 1]); // cache
failedMembers.add(members[members.length - 2]); // admin
- List<GMSMember> newMbrs =
+ List<MemberIdentifier> newMbrs =
new ArrayList<>(lastView.getGMSMembers());
newMbrs.removeAll(failedMembers);
GMSMembershipView newView =
- new GMSMembershipView((GMSMember) members[0], 5, (List<GMSMember>) (List<?>) newMbrs,
- Collections.emptySet(), (Set<GMSMember>) (Set<?>) failedMembers);
+ new GMSMembershipView(members[0], 5, (List<MemberIdentifier>) (List<?>) newMbrs,
+ Collections.emptySet(), (Set<MemberIdentifier>) (Set<?>) failedMembers);
int failedWeight = newView.getCrashedMemberWeight(lastView);
// System.out.println("last view = " + lastView);
// System.out.println("failed mbrs = " + failedMembers);
// System.out.println("failed weight = " + failedWeight);
assertEquals("failure weight calculation is incorrect", 10, failedWeight);
- Set<GMSMember> actual = newView.getActualCrashedMembers(lastView);
+ Set<MemberIdentifier> actual = newView.getActualCrashedMembers(lastView);
assertTrue(!actual.contains(members[members.length - 2]));
}
}
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/adapter/auth/AbstractGMSAuthenticatorTestCase.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/adapter/auth/AbstractGMSAuthenticatorTestCase.java
index ddfd7f5..c88fa3a 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/adapter/auth/AbstractGMSAuthenticatorTestCase.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/adapter/auth/AbstractGMSAuthenticatorTestCase.java
@@ -28,7 +28,7 @@ import org.mockito.MockitoAnnotations;
import org.apache.geode.LogWriter;
import org.apache.geode.distributed.DistributedMember;
import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.membership.gms.GMSMember;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
import org.apache.geode.distributed.internal.membership.gms.Services;
import org.apache.geode.distributed.internal.membership.gms.api.MembershipConfig;
import org.apache.geode.internal.net.SocketCreator;
@@ -48,7 +48,7 @@ public abstract class AbstractGMSAuthenticatorTestCase {
@Mock
protected Services services;
@Mock
- protected GMSMember member;
+ protected InternalDistributedMember member;
@Mock
protected Subject subject;
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberJUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberDataJUnitTest.java
similarity index 69%
rename from geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberJUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberDataJUnitTest.java
index 1d42638..966c24b 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/GMSMemberDataJUnitTest.java
@@ -43,128 +43,128 @@ import org.apache.geode.internal.serialization.VersionedDataInputStream;
import org.apache.geode.test.junit.categories.SecurityTest;
@Category({SecurityTest.class})
-public class GMSMemberJUnitTest {
+public class GMSMemberDataJUnitTest {
@Test
public void testEqualsNotSameType() {
- GMSMember member = new GMSMember();
- assertFalse(member.equals("Not a GMSMember"));
+ GMSMemberData member = new GMSMemberData();
+ assertFalse(member.equals("Not a GMSMemberData"));
}
@Test
public void testEqualsIsSame() {
- GMSMember member = new GMSMember();
+ GMSMemberData member = new GMSMemberData();
assertTrue(member.equals(member));
}
@Test
public void testCompareToIsSame() {
- GMSMember member = new GMSMember();
+ GMSMemberData member = new GMSMemberData();
UUID uuid = new UUID(0, 0);
member.setUUID(uuid);
assertEquals(0, member.compareTo(member));
}
- private GMSMember createGMSMember(byte[] inetAddress, int viewId, long msb, long lsb) {
- GMSMember member = new GMSMember();
+ private GMSMemberData createGMSMember(byte[] inetAddress, int viewId, long msb, long lsb) {
+ GMSMemberData member = new GMSMemberData();
InetAddress addr1 = mock(InetAddress.class);
when(addr1.getAddress()).thenReturn(inetAddress);
member.setInetAddr(addr1);
- member.setBirthViewId(viewId);
+ member.setVmViewId(viewId);
member.setUUID(new UUID(msb, lsb));
return member;
}
@Test
public void testCompareToInetAddressIsLongerThan() {
- GMSMember member1 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
- GMSMember member2 = createGMSMember(new byte[] {1, 1, 1, 1}, 1, 1, 1);
+ GMSMemberData member1 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
+ GMSMemberData member2 = createGMSMember(new byte[] {1, 1, 1, 1}, 1, 1, 1);
assertEquals(1, member1.compareTo(member2));
}
@Test
public void testShallowMemberEquals() {
- GMSMember member1 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
- GMSMember member2 =
- new GMSMember(member1.getInetAddress(), member1.getPort(), member1.getVersionOrdinal(),
+ GMSMemberData member1 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
+ GMSMemberData member2 =
+ new GMSMemberData(member1.getInetAddress(), member1.getPort(), member1.getVersionOrdinal(),
member1.getUuidMSBs(), member1.getUuidLSBs(), member1.getVmViewId());
assertEquals(0, member1.compareTo(member2));
}
@Test
public void testShallowMemberNotEquals() {
- GMSMember member1 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
- GMSMember member2 = new GMSMember(member1.getInetAddress(), member1.getPort(),
+ GMSMemberData member1 = createGMSMember(new byte[] {1, 1, 1, 1, 1}, 1, 1, 1);
+ GMSMemberData member2 = new GMSMemberData(member1.getInetAddress(), member1.getPort(),
member1.getVersionOrdinal(), member1.getUuidMSBs(), member1.getUuidLSBs(), 100);
assertEquals(false, member1.equals(member2));
}
... 208 lines suppressed ...