You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2018/01/05 21:02:54 UTC

[geode] branch develop updated (193bb07 -> 1f4c907)

This is an automated email from the ASF dual-hosted git repository.

klund pushed a change to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git.


    from 193bb07  GEODE-4158: Correct the detection of Geode-internal classes for serialization. - Remove consideration of any class under org.apache as a Geode-internal   class that resulted from a search-and-replace.
     new 75c0413  GEODE-3965: rename DistributionManager classes
     new 1788b5f  GEODE-3965: define exceptionInThreads methods in DistributionManager
     new 1f4c907  GEODE-3965: rename and cleanup DistributionManager tests

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../geode/modules/util/BootstrappingFunction.java  |    5 +-
 .../apache/geode/admin/GemFireMemberStatus.java    |    4 +-
 .../admin/internal/AbstractHealthEvaluator.java    |    4 +-
 .../admin/internal/AdminDistributedSystemImpl.java |   35 +-
 .../geode/admin/internal/CacheHealthEvaluator.java |    8 +-
 .../geode/admin/internal/CacheServerImpl.java      |    7 +-
 .../internal/DistributedSystemHealthEvaluator.java |   14 +-
 .../admin/internal/DistributionLocatorImpl.java    |    5 +-
 .../admin/internal/GemFireHealthEvaluator.java     |    4 +-
 .../admin/internal/MemberHealthEvaluator.java      |    4 +-
 .../internal/SystemMemberCacheEventProcessor.java  |    4 +-
 .../apache/geode/admin/jmx/internal/AgentImpl.java |    6 +-
 .../geode/admin/jmx/internal/AgentLauncher.java    |    4 +-
 .../client/internal/CacheServerLoadMessage.java    |    4 +-
 .../geode/distributed/DistributedSystem.java       |    6 +-
 .../internal/ClusterConfigurationService.java      |    2 +-
 ...anager.java => ClusterDistributionManager.java} |  146 +-
 .../internal/CollectingReplyProcessor.java         |    2 +-
 .../org/apache/geode/distributed/internal/DM.java  |  489 ---
 .../distributed/internal/DirectReplyProcessor.java |   23 +-
 .../distributed/internal/DistributionAdvisee.java  |    4 +-
 .../distributed/internal/DistributionAdvisor.java  |    6 +-
 .../distributed/internal/DistributionManager.java  | 4539 ++------------------
 .../distributed/internal/DistributionMessage.java  |   18 +-
 .../internal/DistributionMessageObserver.java      |    6 +-
 .../distributed/internal/HealthMonitorImpl.java    |    4 +-
 .../internal/HighPriorityAckedMessage.java         |    6 +-
 .../internal/HighPriorityDistributionMessage.java  |    2 +-
 .../internal/InternalDistributedSystem.java        |   19 +-
 .../distributed/internal/InternalLocator.java      |    6 +-
 .../internal/LonerDistributionManager.java         |   14 +-
 .../distributed/internal/MembershipListener.java   |    2 +-
 .../internal/PooledDistributionMessage.java        |    2 +-
 .../geode/distributed/internal/ProductUseLog.java  |    2 +-
 .../internal/ReliableReplyProcessor21.java         |    4 +-
 .../geode/distributed/internal/ReplyMessage.java   |    6 +-
 .../distributed/internal/ReplyProcessor21.java     |   39 +-
 .../geode/distributed/internal/ReplySender.java    |    4 +-
 .../distributed/internal/SerialAckedMessage.java   |    6 +-
 .../internal/SerialDistributionMessage.java        |    2 +-
 .../geode/distributed/internal/ServerLocator.java  |    2 +-
 .../distributed/internal/ShutdownMessage.java      |    2 +-
 .../geode/distributed/internal/StartupMessage.java |    9 +-
 .../internal/StartupMessageReplyProcessor.java     |    6 +-
 .../distributed/internal/StartupOperation.java     |    4 +-
 .../internal/StartupResponseMessage.java           |    4 +-
 .../StartupResponseWithVersionMessage.java         |    5 +-
 .../internal/WaitForViewInstallation.java          |    6 +-
 .../distributed/internal/direct/DirectChannel.java |   10 +-
 .../internal/direct/DirectChannelListener.java     |    4 +-
 .../distributed/internal/locks/DLockGrantor.java   |    8 +-
 .../internal/locks/DLockQueryProcessor.java        |   16 +-
 .../locks/DLockRecoverGrantorProcessor.java        |   23 +-
 .../internal/locks/DLockReleaseProcessor.java      |   16 +-
 .../internal/locks/DLockRequestProcessor.java      |   33 +-
 .../distributed/internal/locks/DLockService.java   |   14 +-
 .../distributed/internal/locks/DLockToken.java     |    8 +-
 .../internal/locks/DeposeGrantorProcessor.java     |   14 +-
 .../internal/locks/DistributedMemberLock.java      |    4 +-
 .../internal/locks/ElderInitProcessor.java         |   15 +-
 .../distributed/internal/locks/ElderState.java     |    8 +-
 .../internal/locks/GrantorRequestProcessor.java    |   20 +-
 .../distributed/internal/locks/LockGrantorId.java  |    8 +-
 .../locks/NonGrantorDestroyedProcessor.java        |   20 +-
 .../membership/DistributedMembershipListener.java  |    1 -
 .../membership/InternalDistributedMember.java      |   39 +-
 .../internal/membership/InternalRole.java          |    6 +-
 .../internal/membership/MemberAttributes.java      |    4 +-
 .../distributed/internal/membership/NetView.java   |   24 +-
 .../gms/locator/FindCoordinatorRequest.java        |    5 +-
 .../gms/locator/FindCoordinatorResponse.java       |    4 +-
 .../membership/gms/locator/GMSLocator.java         |    4 +-
 .../membership/gms/membership/GMSJoinLeave.java    |    9 +-
 .../gms/messages/FinalCheckPassedMessage.java      |    4 +-
 .../membership/gms/messages/HeartbeatMessage.java  |    4 +-
 .../gms/messages/HeartbeatRequestMessage.java      |    4 +-
 .../gms/messages/InstallViewMessage.java           |    8 +-
 .../gms/messages/JoinRequestMessage.java           |    4 +-
 .../gms/messages/JoinResponseMessage.java          |    8 +-
 .../gms/messages/LeaveRequestMessage.java          |    4 +-
 .../gms/messages/NetworkPartitionMessage.java      |    4 +-
 .../gms/messages/RemoveMemberMessage.java          |    5 +-
 .../gms/messages/SuspectMembersMessage.java        |    4 +-
 .../membership/gms/messages/ViewAckMessage.java    |    4 +-
 .../membership/gms/messenger/JGroupsMessenger.java |   12 +-
 .../membership/gms/mgr/GMSMembershipManager.java   |   12 +-
 .../membership/gms/mgr/LocalViewMessage.java       |    6 +-
 .../internal/streaming/StreamingOperation.java     |   17 +-
 .../geode/internal/InternalDataSerializer.java     |    4 +-
 .../geode/internal/InternalInstantiator.java       |    6 +-
 .../internal/admin/ClientMembershipMessage.java    |    6 +-
 .../geode/internal/admin/GfManagerAgent.java       |    4 +-
 .../geode/internal/admin/StatAlertsManager.java    |   10 +-
 .../admin/remote/AddHealthListenerRequest.java     |    2 +-
 .../admin/remote/AddHealthListenerResponse.java    |    4 +-
 .../admin/remote/AddStatListenerRequest.java       |    2 +-
 .../admin/remote/AddStatListenerResponse.java      |    4 +-
 .../remote/AdminConsoleDisconnectMessage.java      |    4 +-
 .../internal/admin/remote/AdminConsoleMessage.java |    9 +-
 .../admin/remote/AdminMultipleReplyProcessor.java  |    4 +-
 .../geode/internal/admin/remote/AdminRequest.java  |    8 +-
 .../geode/internal/admin/remote/AdminResponse.java |    2 +-
 .../geode/internal/admin/remote/AdminWaiters.java  |    6 +-
 .../admin/remote/AlertLevelChangeMessage.java      |    4 +-
 .../admin/remote/AlertListenerMessage.java         |    2 +-
 .../admin/remote/AlertsNotificationMessage.java    |    4 +-
 .../admin/remote/AppCacheSnapshotMessage.java      |    2 +-
 .../internal/admin/remote/BridgeServerRequest.java |    4 +-
 .../admin/remote/BridgeServerResponse.java         |    4 +-
 .../internal/admin/remote/CacheConfigRequest.java  |    2 +-
 .../internal/admin/remote/CacheConfigResponse.java |    6 +-
 .../internal/admin/remote/CacheInfoRequest.java    |    2 +-
 .../internal/admin/remote/CacheInfoResponse.java   |    5 +-
 .../admin/remote/CancelStatListenerRequest.java    |    2 +-
 .../admin/remote/CancelStatListenerResponse.java   |    4 +-
 .../internal/admin/remote/CancellationMessage.java |    2 +-
 .../admin/remote/ChangeRefreshIntervalMessage.java |    4 +-
 .../internal/admin/remote/CliLegacyMessage.java    |    4 +-
 .../internal/admin/remote/CompactRequest.java      |   12 +-
 .../internal/admin/remote/DestroyEntryMessage.java |    4 +-
 .../admin/remote/DestroyRegionMessage.java         |    4 +-
 .../admin/remote/DurableClientInfoRequest.java     |    2 +-
 .../admin/remote/DurableClientInfoResponse.java    |    6 +-
 .../admin/remote/FetchDistLockInfoRequest.java     |    2 +-
 .../admin/remote/FetchDistLockInfoResponse.java    |    3 +-
 .../admin/remote/FetchHealthDiagnosisRequest.java  |    2 +-
 .../admin/remote/FetchHealthDiagnosisResponse.java |    4 +-
 .../internal/admin/remote/FetchHostRequest.java    |    2 +-
 .../internal/admin/remote/FetchHostResponse.java   |    5 +-
 .../remote/FetchResourceAttributesRequest.java     |    2 +-
 .../remote/FetchResourceAttributesResponse.java    |    6 +-
 .../internal/admin/remote/FetchStatsRequest.java   |    2 +-
 .../internal/admin/remote/FetchStatsResponse.java  |    6 +-
 .../internal/admin/remote/FetchSysCfgRequest.java  |    2 +-
 .../internal/admin/remote/FetchSysCfgResponse.java |    3 +-
 .../admin/remote/FlushAppCacheSnapshotMessage.java |    2 +-
 .../admin/remote/HealthListenerMessage.java        |    2 +-
 .../internal/admin/remote/LicenseInfoRequest.java  |    2 +-
 .../internal/admin/remote/LicenseInfoResponse.java |    4 +-
 .../admin/remote/MissingPersistentIDsRequest.java  |   10 +-
 .../admin/remote/MissingPersistentIDsResponse.java |    5 +-
 .../admin/remote/ObjectDetailsRequest.java         |    2 +-
 .../admin/remote/ObjectDetailsResponse.java        |    3 +-
 .../internal/admin/remote/ObjectNamesRequest.java  |    2 +-
 .../internal/admin/remote/ObjectNamesResponse.java |    3 +-
 .../remote/PrepareRevokePersistentIDRequest.java   |   13 +-
 .../admin/remote/RefreshMemberSnapshotRequest.java |    2 +-
 .../remote/RefreshMemberSnapshotResponse.java      |    5 +-
 .../admin/remote/RegionAttributesRequest.java      |    2 +-
 .../admin/remote/RegionAttributesResponse.java     |    4 +-
 .../geode/internal/admin/remote/RegionRequest.java |    2 +-
 .../internal/admin/remote/RegionResponse.java      |    2 +-
 .../internal/admin/remote/RegionSizeRequest.java   |    2 +-
 .../internal/admin/remote/RegionSizeResponse.java  |    3 +-
 .../admin/remote/RegionStatisticsRequest.java      |    2 +-
 .../admin/remote/RegionStatisticsResponse.java     |    4 +-
 .../admin/remote/RegionSubRegionSizeRequest.java   |    4 +-
 .../admin/remote/RegionSubRegionsSizeResponse.java |    7 +-
 .../admin/remote/RemoteGfManagerAgent.java         |   40 +-
 .../admin/remote/RemoveHealthListenerRequest.java  |    2 +-
 .../admin/remote/RemoveHealthListenerResponse.java |    4 +-
 .../admin/remote/ResetHealthStatusRequest.java     |    2 +-
 .../admin/remote/ResetHealthStatusResponse.java    |    4 +-
 .../admin/remote/RevokePersistentIDRequest.java    |    8 +-
 .../internal/admin/remote/RootRegionRequest.java   |    2 +-
 .../internal/admin/remote/RootRegionResponse.java  |    5 +-
 .../remote/ShutdownAllGatewayHubsRequest.java      |    6 +-
 .../internal/admin/remote/ShutdownAllRequest.java  |   13 +-
 .../admin/remote/SnapshotResultMessage.java        |    2 +-
 .../remote/StatAlertsManagerAssignMessage.java     |    6 +-
 .../internal/admin/remote/StatListenerMessage.java |    2 +-
 .../internal/admin/remote/StoreSysCfgRequest.java  |    2 +-
 .../internal/admin/remote/StoreSysCfgResponse.java |    3 +-
 .../internal/admin/remote/SubRegionRequest.java    |    2 +-
 .../internal/admin/remote/SubRegionResponse.java   |    3 +-
 .../internal/admin/remote/TailLogRequest.java      |    2 +-
 .../internal/admin/remote/TailLogResponse.java     |    5 +-
 .../admin/remote/UpdateAlertDefinitionMessage.java |    4 +-
 .../internal/admin/remote/VersionInfoRequest.java  |    2 +-
 .../internal/admin/remote/VersionInfoResponse.java |    3 +-
 .../geode/internal/cache/AbstractCacheServer.java  |    4 +-
 .../geode/internal/cache/AbstractRegion.java       |    4 +-
 .../internal/cache/AbstractUpdateOperation.java    |   10 +-
 .../cache/AddCacheServerProfileMessage.java        |    4 +-
 .../apache/geode/internal/cache/BucketAdvisor.java |    4 +-
 .../internal/cache/CacheDistributionAdvisor.java   |    6 +-
 .../geode/internal/cache/CacheServerAdvisor.java   |    6 +-
 .../geode/internal/cache/CacheServerImpl.java      |    5 +-
 .../geode/internal/cache/CloseCacheMessage.java    |    4 +-
 .../geode/internal/cache/ControllerAdvisor.java    |    7 +-
 .../internal/cache/CreateRegionProcessor.java      |    5 +-
 .../geode/internal/cache/DestroyOperation.java     |    6 +-
 .../cache/DestroyPartitionedRegionMessage.java     |    4 +-
 .../internal/cache/DestroyRegionOperation.java     |   12 +-
 .../geode/internal/cache/DirectReplyMessage.java   |    7 +-
 .../geode/internal/cache/DistPeerTXStateStub.java  |   10 +-
 .../internal/cache/DistTXAdjunctCommitMessage.java |    4 +-
 .../geode/internal/cache/DistTXCommitMessage.java  |    9 +-
 .../internal/cache/DistTXCoordinatorInterface.java |    9 +-
 .../internal/cache/DistTXPrecommitMessage.java     |    9 +-
 .../internal/cache/DistTXRollbackMessage.java      |   10 +-
 .../internal/cache/DistTXStateOnCoordinator.java   |    8 +-
 .../cache/DistTXStateProxyImplOnCoordinator.java   |   14 +-
 .../internal/cache/DistributedCacheOperation.java  |   21 +-
 .../internal/cache/DistributedClearOperation.java  |    8 +-
 .../geode/internal/cache/DistributedRegion.java    |   11 +-
 .../DistributedRegionFunctionStreamingMessage.java |   26 +-
 .../cache/DistributedTombstoneOperation.java       |    8 +-
 .../apache/geode/internal/cache/EntriesMap.java    |    5 +-
 .../apache/geode/internal/cache/FilterProfile.java |    6 +-
 .../internal/cache/FindDurableQueueProcessor.java  |   10 +-
 .../geode/internal/cache/FindRemoteTXMessage.java  |    9 +-
 .../internal/cache/FindVersionTagOperation.java    |   10 +-
 .../FunctionStreamingOrderedReplyMessage.java      |    7 +-
 .../cache/FunctionStreamingReplyMessage.java       |    5 +-
 .../geode/internal/cache/GemFireCacheImpl.java     |   26 +-
 .../internal/cache/InitialImageFlowControl.java    |   16 +-
 .../internal/cache/InitialImageOperation.java      |   71 +-
 .../apache/geode/internal/cache/InternalCache.java |    4 +-
 .../geode/internal/cache/InternalRegion.java       |    4 +-
 .../geode/internal/cache/InvalidateOperation.java  |    6 +-
 .../cache/InvalidatePartitionedRegionMessage.java  |    4 +-
 .../internal/cache/InvalidateRegionOperation.java  |    4 +-
 .../internal/cache/JtaAfterCompletionMessage.java  |    9 +-
 .../internal/cache/JtaBeforeCompletionMessage.java |    4 +-
 .../cache/LatestLastAccessTimeMessage.java         |    4 +-
 .../cache/LatestLastAccessTimeOperation.java       |    4 +-
 .../cache/LatestLastAccessTimeReplyProcessor.java  |    5 +-
 .../apache/geode/internal/cache/LocalRegion.java   |    4 +-
 .../cache/MemberFunctionStreamingMessage.java      |   23 +-
 .../geode/internal/cache/NonLocalRegionEntry.java  |    4 +-
 .../org/apache/geode/internal/cache/Oplog.java     |    4 +-
 .../internal/cache/PRHARedundancyProvider.java     |    6 +-
 .../geode/internal/cache/PartitionedRegion.java    |   20 +-
 .../internal/cache/PartitionedRegionDataStore.java |    6 +-
 .../internal/cache/PartitionedRegionHelper.java    |    4 +-
 .../geode/internal/cache/ProxyBucketRegion.java    |    4 +-
 .../geode/internal/cache/ProxyRegionMap.java       |    4 +-
 .../apache/geode/internal/cache/RegionEntry.java   |    4 +-
 .../internal/cache/ReleaseClearLockMessage.java    |    8 +-
 .../cache/RemoteContainsKeyValueMessage.java       |    6 +-
 .../geode/internal/cache/RemoteDestroyMessage.java |   15 +-
 .../internal/cache/RemoteFetchEntryMessage.java    |    8 +-
 .../internal/cache/RemoteFetchVersionMessage.java  |   10 +-
 .../geode/internal/cache/RemoteGetMessage.java     |   10 +-
 .../internal/cache/RemoteInvalidateMessage.java    |   22 +-
 .../internal/cache/RemoteOperationMessage.java     |   21 +-
 .../RemoteOperationMessageWithDirectReply.java     |    6 +-
 .../geode/internal/cache/RemotePutAllMessage.java  |   12 +-
 .../geode/internal/cache/RemotePutMessage.java     |   18 +-
 .../internal/cache/RemoteRegionOperation.java      |    6 +-
 .../internal/cache/RemoteRemoveAllMessage.java     |   12 +-
 .../cache/SearchLoadAndWriteProcessor.java         |   53 +-
 .../geode/internal/cache/ServerPingMessage.java    |    6 +-
 .../geode/internal/cache/StateFlushOperation.java  |   24 +-
 .../geode/internal/cache/TXCommitMessage.java      |   37 +-
 .../geode/internal/cache/TXFarSideCMTracker.java   |   11 +-
 .../apache/geode/internal/cache/TXManagerImpl.java |   10 +-
 .../org/apache/geode/internal/cache/TXMessage.java |    8 +-
 .../internal/cache/TXRemoteCommitMessage.java      |    9 +-
 .../internal/cache/TXRemoteRollbackMessage.java    |    4 +-
 .../internal/cache/UpdateAttributesProcessor.java  |   13 +-
 .../cache/UpdateEntryVersionOperation.java         |    6 +-
 .../geode/internal/cache/UpdateOperation.java      |   12 +-
 .../geode/internal/cache/ValidatingDiskRegion.java |    4 +-
 .../cache/backup/BackupDataStoreHelper.java        |   12 +-
 .../geode/internal/cache/backup/BackupManager.java |    4 +-
 .../internal/cache/backup/BackupOperation.java     |    8 +-
 .../cache/backup/BackupReplyProcessor.java         |    4 +-
 .../geode/internal/cache/backup/BackupUtil.java    |    6 +-
 .../internal/cache/backup/FinishBackupFactory.java |    6 +-
 .../cache/backup/FinishBackupOperation.java        |    8 +-
 .../internal/cache/backup/FinishBackupRequest.java |    4 +-
 .../internal/cache/backup/FlushToDiskFactory.java  |    5 +-
 .../cache/backup/FlushToDiskOperation.java         |    9 +-
 .../cache/backup/FlushToDiskProcessor.java         |    4 +-
 .../internal/cache/backup/FlushToDiskRequest.java  |    4 +-
 .../cache/backup/PrepareBackupFactory.java         |    6 +-
 .../cache/backup/PrepareBackupOperation.java       |    7 +-
 .../cache/backup/PrepareBackupRequest.java         |    4 +-
 .../internal/cache/control/ResourceAdvisor.java    |   12 +-
 .../entries/AbstractOplogDiskRegionEntry.java      |    6 +-
 .../cache/entries/AbstractRegionEntry.java         |    6 +-
 .../geode/internal/cache/entries/DiskEntry.java    |    5 +-
 .../internal/cache/execute/AbstractExecution.java  |   18 +-
 .../DistributedRegionFunctionResultSender.java     |   12 +-
 .../cache/execute/MemberFunctionExecutor.java      |    4 +-
 .../cache/execute/MemberFunctionResultSender.java  |   11 +-
 .../cache/execute/MultiRegionFunctionExecutor.java |    4 +-
 .../PartitionedRegionFunctionResultSender.java     |   16 +-
 .../geode/internal/cache/ha/HARegionQueue.java     |    4 +-
 .../internal/cache/ha/QueueRemovalMessage.java     |    5 +-
 .../cache/locks/TXLessorDepartureHandler.java      |    4 +-
 .../locks/TXLockUpdateParticipantsMessage.java     |    4 +-
 .../cache/locks/TXOriginatorRecoveryProcessor.java |   12 +-
 .../locks/TXRecoverGrantorMessageProcessor.java    |    8 +-
 .../AllBucketProfilesUpdateMessage.java            |    8 +-
 .../partitioned/BecomePrimaryBucketMessage.java    |   14 +-
 .../cache/partitioned/BucketBackupMessage.java     |    6 +-
 .../partitioned/BucketProfileUpdateMessage.java    |   10 +-
 .../cache/partitioned/BucketSizeMessage.java       |   12 +-
 .../cache/partitioned/ContainsKeyValueMessage.java |    6 +-
 .../cache/partitioned/CreateBucketMessage.java     |   12 +-
 .../partitioned/DeposePrimaryBucketMessage.java    |   12 +-
 .../internal/cache/partitioned/DestroyMessage.java |   12 +-
 .../DestroyRegionOnDataStoreMessage.java           |   14 +-
 .../cache/partitioned/DumpAllPRConfigMessage.java  |    4 +-
 .../internal/cache/partitioned/DumpB2NRegion.java  |   12 +-
 .../cache/partitioned/DumpBucketsMessage.java      |    4 +-
 .../partitioned/EndBucketCreationMessage.java      |    8 +-
 .../cache/partitioned/FetchBulkEntriesMessage.java |   12 +-
 .../cache/partitioned/FetchEntriesMessage.java     |   13 +-
 .../cache/partitioned/FetchEntryMessage.java       |    8 +-
 .../cache/partitioned/FetchKeysMessage.java        |   14 +-
 .../partitioned/FetchPartitionDetailsMessage.java  |   12 +-
 .../internal/cache/partitioned/FlushMessage.java   |    4 +-
 .../internal/cache/partitioned/GetMessage.java     |   14 +-
 .../cache/partitioned/IdentityRequestMessage.java  |   11 +-
 .../cache/partitioned/IdentityUpdateMessage.java   |    6 +-
 .../cache/partitioned/IndexCreationMsg.java        |   20 +-
 .../cache/partitioned/InterestEventMessage.java    |   14 +-
 .../cache/partitioned/InvalidateMessage.java       |   10 +-
 .../partitioned/ManageBackupBucketMessage.java     |   16 +-
 .../cache/partitioned/ManageBucketMessage.java     |   16 +-
 .../cache/partitioned/MoveBucketMessage.java       |   12 +-
 .../cache/partitioned/PRSanityCheckMessage.java    |   10 +-
 .../cache/partitioned/PRTombstoneMessage.java      |    6 +-
 .../partitioned/PRUpdateEntryVersionMessage.java   |    4 +-
 .../cache/partitioned/PartitionMessage.java        |   22 +-
 .../PartitionMessageWithDirectReply.java           |    3 +-
 .../PartitionedRegionFunctionStreamingMessage.java |   16 +-
 .../partitioned/PartitionedRegionRebalanceOp.java  |    4 +-
 .../cache/partitioned/PrimaryRequestMessage.java   |    8 +-
 .../cache/partitioned/PutAllPRMessage.java         |   12 +-
 .../internal/cache/partitioned/PutMessage.java     |   14 +-
 .../internal/cache/partitioned/QueryMessage.java   |    8 +-
 .../cache/partitioned/RemoteFetchKeysMessage.java  |   14 +-
 .../cache/partitioned/RemoteSizeMessage.java       |    9 +-
 .../cache/partitioned/RemoveAllPRMessage.java      |   12 +-
 .../cache/partitioned/RemoveBucketMessage.java     |   12 +-
 .../cache/partitioned/RemoveIndexesMessage.java    |   17 +-
 .../internal/cache/partitioned/SizeMessage.java    |   10 +-
 .../partitioned/StreamingPartitionOperation.java   |   10 +-
 .../cache/persistence/MembershipFlushRequest.java  |    9 +-
 .../cache/persistence/MembershipViewRequest.java   |   27 +-
 .../cache/persistence/PersistenceAdvisorImpl.java  |    4 +-
 .../cache/persistence/PersistentMemberManager.java |    8 +-
 .../persistence/PersistentStateQueryMessage.java   |   14 +-
 .../PrepareNewPersistentMemberMessage.java         |   10 +-
 .../persistence/RemovePersistentMemberMessage.java |   11 +-
 .../internal/cache/snapshot/FlowController.java    |   27 +-
 .../internal/cache/tier/sockets/AcceptorImpl.java  |    4 +-
 .../cache/tier/sockets/CacheClientNotifier.java    |    7 +-
 .../cache/tier/sockets/CacheClientProxy.java       |    4 +-
 .../cache/tier/sockets/CacheClientUpdater.java     |    7 +-
 .../tier/sockets/ClientBlacklistProcessor.java     |   12 +-
 .../internal/cache/tier/sockets/HandShake.java     |    5 +-
 .../sockets/RemoveClientFromBlacklistMessage.java  |    5 +-
 .../sockets/ServerInterestRegistrationMessage.java |    6 +-
 .../cache/tier/sockets/VersionedObjectList.java    |    5 +-
 .../tier/sockets/command/ExecuteFunction66.java    |   11 +-
 .../tier/sockets/command/TXFailoverCommand.java    |    4 +-
 .../geode/internal/cache/tx/ClientTXStateStub.java |    4 +-
 .../internal/cache/tx/DistClientTXStateStub.java   |    9 +-
 .../internal/cache/tx/DistributedTXRegionStub.java |   19 +-
 .../cache/versions/RegionVersionVector.java        |    7 +-
 .../internal/cache/versions/VMVersionTag.java      |    4 +-
 .../geode/internal/cache/versions/VersionTag.java  |    4 +-
 .../internal/cache/wan/AbstractGatewaySender.java  |    4 +-
 .../internal/cache/wan/GatewaySenderAdvisor.java   |    7 +-
 ...aySenderQueueEntrySynchronizationOperation.java |    8 +-
 .../wan/parallel/ParallelGatewaySenderQueue.java   |    5 +-
 .../parallel/ParallelQueueBatchRemovalMessage.java |    8 +-
 .../wan/parallel/ParallelQueueRemovalMessage.java  |    5 +-
 ...tilParallelGatewaySenderFlushedCoordinator.java |    3 +-
 .../cache/wan/serial/BatchDestroyOperation.java    |    6 +-
 .../internal/cache/xmlcache/CacheCreation.java     |    4 +-
 .../xmlcache/ParallelAsyncEventQueueCreation.java  |    4 +-
 .../xmlcache/ParallelGatewaySenderCreation.java    |    4 +-
 .../xmlcache/SerialAsyncEventQueueCreation.java    |    4 +-
 .../xmlcache/SerialGatewaySenderCreation.java      |    4 +-
 .../geode/internal/jta/GlobalTransaction.java      |    4 +-
 .../internal/logging/log4j/AlertAppender.java      |    4 +-
 .../org/apache/geode/internal/tcp/Connection.java  |   12 +-
 .../apache/geode/internal/tcp/ConnectionTable.java |    6 +-
 .../org/apache/geode/internal/tcp/TCPConduit.java  |   10 +-
 .../management/internal/JmxManagerAdvisee.java     |    4 +-
 .../management/internal/JmxManagerAdvisor.java     |   11 +-
 .../geode/management/internal/ManagementAgent.java |    6 +-
 .../management/internal/ManagerStartupMessage.java |    7 +-
 .../geode/management/internal/MemberMessenger.java |    8 +-
 .../internal/beans/DistributedSystemBridge.java    |   10 +-
 .../internal/beans/ManagementAdapter.java          |    4 +-
 .../internal/beans/MemberMBeanBridge.java          |    9 +-
 .../cli/commands/BackupDiskStoreCommand.java       |    4 +-
 .../callbacks/ConfigurationChangeListener.java     |    4 -
 .../internal/messages/CompactRequest.java          |   12 +-
 .../geode/pdx/internal/CheckTypeRegistryState.java |    7 +-
 .../org/apache/geode/admin/AdminTestHelper.java    |   12 +-
 .../geode/cache/ConnectionPoolDUnitTest.java       |    6 +-
 .../geode/cache30/ClientServerCCEDUnitTest.java    |    4 +-
 .../cache30/DistributedAckRegionCCEDUnitTest.java  |    7 +-
 .../geode/cache30/RegionReliabilityTestCase.java   |    4 +-
 .../org/apache/geode/cache30/SlowRecDUnitTest.java |   14 +-
 .../DistributedLockServiceDUnitTest.java           |    7 +-
 .../distributed/DistributedMemberDUnitTest.java    |   15 +-
 .../distributed/DistributedSystemDUnitTest.java    |   17 +-
 .../geode/distributed/HostedLocatorsDUnitTest.java |   28 +-
 .../apache/geode/distributed/LocatorDUnitTest.java |   10 +-
 .../apache/geode/distributed/RoleDUnitTest.java    |    8 +-
 .../geode/distributed/SystemAdminDUnitTest.java    |    4 +-
 .../ClusterDistributionManagerDUnitTest.java       |  399 ++
 ...lusterDistributionManagerForAdminDUnitTest.java |  317 ++
 ...st.java => ClusterDistributionManagerTest.java} |    6 +-
 .../ConsoleDistributionManagerDUnitTest.java       |  446 --
 .../geode/distributed/internal/DateMessage.java    |    2 +-
 .../internal/DistributionAdvisorDUnitTest.java     |    2 +-
 .../internal/DistributionManagerDUnitTest.java     |  545 ---
 .../InternalDistributedSystemJUnitTest.java        |    2 +-
 .../org/apache/geode/distributed/internal/LDM.java |    4 +-
 .../distributed/internal/ProduceDateMessages.java  |    2 +-
 .../internal/membership/MembershipJUnitTest.java   |   18 +-
 .../internal/membership/NetViewJUnitTest.java      |   24 +-
 .../membership/gms/MembershipManagerHelper.java    |    5 +-
 .../gms/fd/GMSHealthMonitorJUnitTest.java          |    5 +-
 .../gms/locator/GMSLocatorRecoveryJUnitTest.java   |    4 +-
 .../gms/membership/GMSJoinLeaveJUnitTest.java      |    7 +-
 .../gms/membership/GMSJoinLeaveTestHelper.java     |    4 +-
 .../gms/membership/StatRecorderJUnitTest.java      |    6 +-
 .../gms/messenger/GMSEncryptJUnitTest.java         |    4 +-
 .../gms/messenger/GMSQuorumCheckerJUnitTest.java   |   27 +-
 .../gms/messenger/JGroupsMessengerJUnitTest.java   |   12 +-
 .../gms/mgr/GMSMembershipManagerJUnitTest.java     |    6 +-
 .../geode/internal/cache/Bug41091DUnitTest.java    |    5 +-
 .../geode/internal/cache/Bug41733DUnitTest.java    |    8 +-
 .../internal/cache/CacheOperationMessageTest.java  |    4 +-
 .../internal/cache/DistributedCacheTestCase.java   |   17 +-
 ...tributedRegionFunctionStreamingMessageTest.java |    4 +-
 .../geode/internal/cache/GIIDeltaDUnitTest.java    |   10 +-
 .../internal/cache/GIIFlowControlDUnitTest.java    |    6 +-
 .../cache/InterruptClientServerDUnitTest.java      |    8 +-
 .../geode/internal/cache/InterruptsDUnitTest.java  |    8 +-
 ...FOEvictionAlgoMemoryEnabledRegionJUnitTest.java |    5 +-
 .../cache/NetSearchMessagingDUnitTest.java         |    8 +-
 .../cache/PartitionedRegionQueryDUnitTest.java     |    5 +-
 .../cache/RemoteFetchEntryMessageTest.java         |    4 +-
 .../internal/cache/RemoteOperationMessageTest.java |    6 +-
 .../cache/SearchLoadAndWriteProcessorTest.java     |    4 +-
 .../internal/cache/SystemFailureDUnitTest.java     |    4 +-
 .../geode/internal/cache/TXManagerImplTest.java    |    7 +-
 .../cache/backup/BackupDistributedTest.java        |   10 +-
 .../backup/BackupPrepareAndFinishMsgDUnitTest.java |    6 +-
 .../cache/backup/BackupReplyProcessorTest.java     |    6 +-
 .../cache/backup/FinishBackupFactoryTest.java      |    6 +-
 .../cache/backup/FinishBackupOperationTest.java    |    6 +-
 .../cache/backup/FinishBackupRequestTest.java      |    6 +-
 .../cache/backup/FlushToDiskFactoryTest.java       |    6 +-
 .../cache/backup/FlushToDiskOperationTest.java     |    6 +-
 .../cache/backup/FlushToDiskRequestTest.java       |    6 +-
 .../backup/IncrementalBackupDistributedTest.java   |    8 +-
 .../cache/backup/PrepareBackupFactoryTest.java     |    6 +-
 .../cache/backup/PrepareBackupOperationTest.java   |    6 +-
 .../cache/backup/PrepareBackupRequestTest.java     |    6 +-
 .../internal/cache/eviction/LRUTestEntry.java      |    5 +-
 ...istributedRegionFunctionExecutionDUnitTest.java |    4 +-
 .../execute/MemberFunctionExecutionDUnitTest.java  |    4 +-
 .../execute/PRFunctionExecutionDUnitTest.java      |    4 +-
 .../cache/locks/TXLockServiceDUnitTest.java        |    6 +-
 .../cache/partitioned/Bug39356DUnitTest.java       |    6 +-
 .../DeposePrimaryBucketMessageTest.java            |    4 +-
 .../cache/partitioned/FetchEntryMessageTest.java   |    4 +-
 .../FetchPartitionDetailsMessageTest.java          |    4 +-
 .../cache/partitioned/MoveBucketMessageTest.java   |    4 +-
 .../cache/partitioned/PartitionMessageTest.java    |    6 +-
 ...sistentColocatedPartitionedRegionDUnitTest.java |    8 +-
 .../PersistentPartitionedRegionDUnitTest.java      |   19 +-
 .../cache/partitioned/RemoveBucketMessageTest.java |    4 +-
 .../PersistentRVVRecoveryDUnitTest.java            |    6 +-
 .../PersistentRecoveryOrderDUnitTest.java          |   41 +-
 .../ParallelQueueRemovalMessageJUnitTest.java      |    4 +-
 ...nectingOutOfOffHeapMemoryListenerJUnitTest.java |    4 +-
 .../offheap/OutOfOffHeapMemoryDUnitTest.java       |    6 +-
 .../geode/internal/tcp/ConnectionJUnitTest.java    |    4 +-
 .../geode/internal/tcp/ConnectionTableTest.java    |    4 +-
 .../geode/management/ManagementTestRule.java       |    4 +-
 .../management/RegionManagementDUnitTest.java      |    4 +-
 .../beans/DistributedSystemBridgeJUnitTest.java    |    6 +-
 .../pdx/ClientsWithVersioningRetryDUnitTest.java   |    7 +-
 .../geode/pdx/DistributedSystemIdDUnitTest.java    |    6 +-
 .../apache/geode/pdx/PdxSerializableDUnitTest.java |    4 +-
 .../security/ClusterConfigNotEnabledDUnitTest.java |    4 +-
 .../java/org/apache/geode/test/fake/Fakes.java     |    4 +-
 .../lucene/internal/DestroyLuceneIndexMessage.java |    5 +-
 .../lucene/internal/LuceneBucketListener.java      |    6 +-
 .../internal/LuceneIndexForPartitionedRegion.java  |    6 +-
 .../web/controllers/AbstractBaseController.java    |    4 +-
 496 files changed, 2980 insertions(+), 7761 deletions(-)
 copy geode-core/src/main/java/org/apache/geode/distributed/internal/{DistributionManager.java => ClusterDistributionManager.java} (96%)
 delete mode 100644 geode-core/src/main/java/org/apache/geode/distributed/internal/DM.java
 create mode 100644 geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java
 create mode 100644 geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerForAdminDUnitTest.java
 rename geode-core/src/test/java/org/apache/geode/distributed/internal/{DistributionManagerTest.java => ClusterDistributionManagerTest.java} (92%)
 delete mode 100644 geode-core/src/test/java/org/apache/geode/distributed/internal/ConsoleDistributionManagerDUnitTest.java
 delete mode 100644 geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionManagerDUnitTest.java

-- 
To stop receiving notification emails like this one, please contact
['"commits@geode.apache.org" <co...@geode.apache.org>'].

[geode] 02/03: GEODE-3965: define exceptionInThreads methods in DistributionManager

Posted by kl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

klund pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 1788b5fd6761fe83d7a973cd1b12a204e41f1fe6
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Thu Jan 4 15:38:53 2018 -0800

    GEODE-3965: define exceptionInThreads methods in DistributionManager
    
    This change removes the dependency on DistributionManagerDUnitTest from
    DistributedCacheTestCase.
---
 .../internal/ClusterDistributionManager.java       | 32 +++++++++++++++++++++-
 .../distributed/internal/DistributionManager.java  |  4 +++
 .../internal/LonerDistributionManager.java         | 10 +++++++
 .../internal/cache/DistributedCacheTestCase.java   |  7 ++---
 4 files changed, 48 insertions(+), 5 deletions(-)

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 7f96a7b..0359b76 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
@@ -706,9 +706,11 @@ public class ClusterDistributionManager implements DistributionManager {
           poolQueue = this.serialQueue;
         }
         ThreadFactory tf = new ThreadFactory() {
+          @Override
           public Thread newThread(final Runnable command) {
             ClusterDistributionManager.this.stats.incSerialThreadStarts();
             final Runnable r = new Runnable() {
+              @Override
               public void run() {
                 ClusterDistributionManager.this.stats.incNumSerialThreads(1);
                 try {
@@ -735,9 +737,11 @@ public class ClusterDistributionManager implements DistributionManager {
       {
         BlockingQueue q = new LinkedBlockingQueue();
         ThreadFactory tf = new ThreadFactory() {
+          @Override
           public Thread newThread(final Runnable command) {
             ClusterDistributionManager.this.stats.incViewThreadStarts();
             final Runnable r = new Runnable() {
+              @Override
               public void run() {
                 ClusterDistributionManager.this.stats.incNumViewThreads(1);
                 try {
@@ -771,9 +775,11 @@ public class ClusterDistributionManager implements DistributionManager {
         ThreadFactory tf = new ThreadFactory() {
           private int next = 0;
 
+          @Override
           public Thread newThread(final Runnable command) {
             ClusterDistributionManager.this.stats.incProcessingThreadStarts();
             final Runnable r = new Runnable() {
+              @Override
               public void run() {
                 ClusterDistributionManager.this.stats.incNumProcessingThreads(1);
                 try {
@@ -810,9 +816,11 @@ public class ClusterDistributionManager implements DistributionManager {
         ThreadFactory tf = new ThreadFactory() {
           private int next = 0;
 
+          @Override
           public Thread newThread(final Runnable command) {
             ClusterDistributionManager.this.stats.incHighPriorityThreadStarts();
             final Runnable r = new Runnable() {
+              @Override
               public void run() {
                 ClusterDistributionManager.this.stats.incHighPriorityThreads(1);
                 try {
@@ -841,9 +849,11 @@ public class ClusterDistributionManager implements DistributionManager {
         ThreadFactory tf = new ThreadFactory() {
           private int next = 0;
 
+          @Override
           public Thread newThread(final Runnable command) {
             ClusterDistributionManager.this.stats.incWaitingThreadStarts();
             final Runnable r = new Runnable() {
+              @Override
               public void run() {
                 ClusterDistributionManager.this.stats.incWaitingThreads(1);
                 try {
@@ -878,9 +888,11 @@ public class ClusterDistributionManager implements DistributionManager {
         ThreadFactory tf = new ThreadFactory() {
           private int next = 0;
 
+          @Override
           public Thread newThread(final Runnable command) {
             ClusterDistributionManager.this.stats.incWaitingThreadStarts();// will it be ok?
             final Runnable r = new Runnable() {
+              @Override
               public void run() {
                 ClusterDistributionManager.this.stats.incWaitingThreads(1);// will it be ok
                 try {
@@ -917,9 +929,11 @@ public class ClusterDistributionManager implements DistributionManager {
         ThreadFactory tf = new ThreadFactory() {
           private int next = 0;
 
+          @Override
           public Thread newThread(final Runnable command) {
             ClusterDistributionManager.this.stats.incPartitionedRegionThreadStarts();
             final Runnable r = new Runnable() {
+              @Override
               public void run() {
                 stats.incPartitionedRegionThreads(1);
                 try {
@@ -959,9 +973,11 @@ public class ClusterDistributionManager implements DistributionManager {
         ThreadFactory tf = new ThreadFactory() {
           private int next = 0;
 
+          @Override
           public Thread newThread(final Runnable command) {
             ClusterDistributionManager.this.stats.incFunctionExecutionThreadStarts();
             final Runnable r = new Runnable() {
+              @Override
               public void run() {
                 stats.incFunctionExecutionThreads(1);
                 isFunctionExecutionThread.set(Boolean.TRUE);
@@ -1225,10 +1241,12 @@ public class ClusterDistributionManager implements DistributionManager {
     }
   }
 
+  @Override
   public int getDMType() {
     return this.dmType;
   }
 
+  @Override
   public List<InternalDistributedMember> getViewMembers() {
     return membershipManager.getView().getMembers();
   }
@@ -1285,6 +1303,7 @@ public class ClusterDistributionManager implements DistributionManager {
     }
     try {
       getWaitingThreadPool().execute(new Runnable() {
+        @Override
         public void run() {
           // call in background since it might need to send a reply
           // and we are not ready to send messages until startup is finished
@@ -1407,6 +1426,7 @@ public class ClusterDistributionManager implements DistributionManager {
   /**
    * Did an exception occur in one of the threads launched by this distribution manager?
    */
+  @Override
   public boolean exceptionInThreads() {
     return this.exceptionInThreads || this.threadGroup.getUncaughtExceptionsCount() > 0;
   }
@@ -1415,7 +1435,8 @@ public class ClusterDistributionManager implements DistributionManager {
    * Clears the boolean that determines whether or not an exception occurred in one of the worker
    * threads. This method should be used for testing purposes only!
    */
-  void clearExceptionInThreads() {
+  @Override
+  public void clearExceptionInThreads() {
     this.exceptionInThreads = false;
     this.threadGroup.clearUncaughtExceptionsCount();
   }
@@ -1553,6 +1574,7 @@ public class ClusterDistributionManager implements DistributionManager {
    *
    * @since GemFire 6.6.3
    */
+  @Override
   public Map<InternalDistributedMember, Collection<String>> getAllHostedLocators() {
     synchronized (this.membersLock) {
       return this.hostedLocatorsAll;
@@ -1728,6 +1750,7 @@ public class ClusterDistributionManager implements DistributionManager {
         // Don't block indefinitely trying to send the shutdown message, in
         // case other VMs in the system are ill-behaved. (bug 34710)
         final Runnable r = new Runnable() {
+          @Override
           public void run() {
             try {
               ConnectionTable.threadWantsSharedResources();
@@ -2196,6 +2219,7 @@ public class ClusterDistributionManager implements DistributionManager {
   protected class MemberEventInvoker implements Runnable {
 
 
+    @Override
     @SuppressWarnings("synthetic-access")
     public void run() {
       for (;;) {
@@ -3363,21 +3387,25 @@ public class ClusterDistributionManager implements DistributionManager {
           }
           if (l == null) {
             l = new MembershipListener() {
+              @Override
               public void memberJoined(InternalDistributedMember theId) {
                 // nothing needed
               }
 
+              @Override
               public void memberDeparted(InternalDistributedMember theId, boolean crashed) {
                 if (desiredElder.equals(theId)) {
                   notifyElderChangeWaiters();
                 }
               }
 
+              @Override
               public void memberSuspect(InternalDistributedMember id,
                   InternalDistributedMember whoSuspected, String reason) {}
 
               public void viewInstalled(NetView view) {}
 
+              @Override
               public void quorumLost(Set<InternalDistributedMember> failures,
                   List<InternalDistributedMember> remaining) {}
             };
@@ -3863,9 +3891,11 @@ public class ClusterDistributionManager implements DistributionManager {
       serialQueuedMap.put(id, poolQueue);
 
       ThreadFactory tf = new ThreadFactory() {
+        @Override
         public Thread newThread(final Runnable command) {
           SerialQueuedExecutorPool.this.stats.incSerialPooledThreadStarts();
           final Runnable r = new Runnable() {
+            @Override
             public void run() {
               ConnectionTable.threadWantsSharedResources();
               Connection.makeReaderThread();
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
index 94c9d52..3719bb2 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
@@ -485,4 +485,8 @@ public interface DistributionManager extends ReplySender {
   void removeHealthMonitor(InternalDistributedMember owner, int theId);
 
   void createHealthMonitor(InternalDistributedMember owner, GemFireHealthConfig cfg);
+
+  boolean exceptionInThreads();
+
+  void clearExceptionInThreads();
 }
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 b0d04ec..cde316a 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
@@ -1421,4 +1421,14 @@ public class LonerDistributionManager implements DistributionManager {
     throw new UnsupportedOperationException(
         "createHealthMonitor is not supported by " + getClass().getSimpleName());
   }
+
+  @Override
+  public boolean exceptionInThreads() {
+    return false;
+  }
+
+  @Override
+  public void clearExceptionInThreads() {
+    // no-op
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedCacheTestCase.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedCacheTestCase.java
index 9496e98..b1c9230 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedCacheTestCase.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DistributedCacheTestCase.java
@@ -28,7 +28,7 @@ import org.apache.geode.cache.LoaderHelper;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.distributed.internal.ClusterDistributionManager;
-import org.apache.geode.distributed.internal.DistributionManagerDUnitTest;
+import org.apache.geode.distributed.internal.ClusterDistributionManagerDUnitTest;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.test.dunit.Host;
@@ -141,11 +141,10 @@ public abstract class DistributedCacheTestCase extends JUnit4DistributedTestCase
     Assert.assertTrue(cache != null, "No cache on this VM?");
     Assert.assertTrue(!cache.isClosed(), "Who closed my cache?");
 
-    InternalDistributedSystem system =
-        (InternalDistributedSystem) ((GemFireCacheImpl) cache).getDistributedSystem();
+    InternalDistributedSystem system = (InternalDistributedSystem) cache.getDistributedSystem();
     ClusterDistributionManager dm = (ClusterDistributionManager) system.getDistributionManager();
     boolean exceptionInThreads = dm.exceptionInThreads();
-    DistributionManagerDUnitTest.clearExceptionInThreads(dm);
+    dm.clearExceptionInThreads();
 
     cache.close();
     cache = null;

-- 
To stop receiving notification emails like this one, please contact
"commits@geode.apache.org" <co...@geode.apache.org>.

[geode] 03/03: GEODE-3965: rename and cleanup DistributionManager tests

Posted by kl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

klund pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 1f4c90794af85b4db6adeee86555f486ecf3f6e6
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Thu Jan 4 15:40:57 2018 -0800

    GEODE-3965: rename and cleanup DistributionManager tests
    
    * DistributionManagerTest -> ClusterDistributionManagerTest
    * DistributionManagerDUnitTest -> ClusterDistributionManagerDUnitTest
    * ConsoleDistributionManagerDUnitTest -> ClusterDistributionManagerForAdminDUnitTest
---
 .../ClusterDistributionManagerDUnitTest.java       | 399 +++++++++++++++
 ...lusterDistributionManagerForAdminDUnitTest.java | 317 ++++++++++++
 ...st.java => ClusterDistributionManagerTest.java} |   4 +-
 .../ConsoleDistributionManagerDUnitTest.java       | 446 -----------------
 .../internal/DistributionManagerDUnitTest.java     | 545 ---------------------
 5 files changed, 719 insertions(+), 992 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java
new file mode 100644
index 0000000..0ba2381
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerDUnitTest.java
@@ -0,0 +1,399 @@
+/*
+ * 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;
+
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.geode.distributed.ConfigurationProperties.ACK_SEVERE_ALERT_THRESHOLD;
+import static org.apache.geode.distributed.ConfigurationProperties.ACK_WAIT_THRESHOLD;
+import static org.apache.geode.distributed.ConfigurationProperties.BIND_ADDRESS;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+import static org.apache.geode.distributed.ConfigurationProperties.NAME;
+import static org.apache.geode.distributed.internal.DistributionConfig.GEMFIRE_PREFIX;
+import static org.apache.geode.test.dunit.Assert.assertEquals;
+import static org.apache.geode.test.dunit.Assert.assertTrue;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.Invoke.invokeInEveryVM;
+import static org.apache.geode.test.dunit.NetworkUtils.getIPLiteral;
+import static org.apache.geode.test.dunit.Wait.pause;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatCode;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.awaitility.Awaitility.await;
+
+import java.net.InetAddress;
+import java.util.Properties;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.ForcedDisconnectException;
+import org.apache.geode.LogWriter;
+import org.apache.geode.admin.AdminDistributedSystem;
+import org.apache.geode.admin.AdminDistributedSystemFactory;
+import org.apache.geode.admin.AlertLevel;
+import org.apache.geode.admin.DistributedSystemConfig;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheListener;
+import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.EntryEvent;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionEvent;
+import org.apache.geode.cache.RegionFactory;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.cache.util.CacheListenerAdapter;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.distributed.internal.membership.MembershipManager;
+import org.apache.geode.distributed.internal.membership.NetView;
+import org.apache.geode.distributed.internal.membership.gms.MembershipManagerHelper;
+import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.test.dunit.DistributedTestCase;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.SerializableRunnable;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.rules.DistributedRestoreSystemProperties;
+import org.apache.geode.test.dunit.rules.SharedErrorCollector;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.MembershipTest;
+
+/**
+ * This class tests the functionality of the {@link ClusterDistributionManager} class.
+ */
+@Category({DistributedTest.class, MembershipTest.class})
+public class ClusterDistributionManagerDUnitTest extends DistributedTestCase {
+  private static final Logger logger = LogService.getLogger();
+
+  private static volatile boolean regionDestroyedInvoked;
+  private static volatile Cache myCache;
+  private static volatile boolean alertReceived;
+
+  private transient ExecutorService executorService;
+
+  private VM vm1;
+
+  @Rule
+  public DistributedRestoreSystemProperties restoreSystemProperties =
+      new DistributedRestoreSystemProperties();
+
+  @Rule
+  public SharedErrorCollector errorCollector = new SharedErrorCollector();
+
+  @Before
+  public void setUp() throws Exception {
+    executorService = Executors.newSingleThreadExecutor();
+    vm1 = Host.getHost(0).getVM(1);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    disconnectAllFromDS();
+    invokeInEveryVM(() -> {
+      regionDestroyedInvoked = false;
+      myCache = null;
+      alertReceived = false;
+    });
+    assertThat(executorService.shutdownNow()).isEmpty();
+  }
+
+  @Test
+  public void testGetDistributionVMType() {
+    DistributionManager dm = getSystem().getDistributionManager();
+    InternalDistributedMember member = dm.getId();
+
+    assertEquals(ClusterDistributionManager.NORMAL_DM_TYPE, member.getVmKind());
+  }
+
+  /**
+   * Demonstrate that a new UDP port is used when an attempt is made to reconnect using a shunned
+   * port
+   */
+  @Test
+  public void testConnectAfterBeingShunned() {
+    InternalDistributedSystem system = getSystem();
+    MembershipManager membershipManager = MembershipManagerHelper.getMembershipManager(system);
+    InternalDistributedMember memberBefore = membershipManager.getLocalMember();
+
+    // TODO GMS needs to have a system property allowing the bind-port to be set
+    System.setProperty(GEMFIRE_PREFIX + "jg-bind-port", "" + memberBefore.getPort());
+    system.disconnect();
+    system = getSystem();
+    membershipManager = MembershipManagerHelper.getMembershipManager(system);
+    system.disconnect();
+    InternalDistributedMember memberAfter = membershipManager.getLocalMember();
+
+    assertThat(memberAfter.getPort()).isEqualTo(memberBefore.getPort());
+  }
+
+  /**
+   * Test the handling of "surprise members" in the membership manager. Create a DistributedSystem
+   * in this VM and then add a fake member to its surpriseMember set. Then ensure that it stays in
+   * the set when a new membership view arrives that doesn't contain it. Then wait until the member
+   * should be gone and force more view processing to have it scrubbed from the set.
+   **/
+  @Test
+  public void testSurpriseMemberHandling() throws Exception {
+    System.setProperty(GEMFIRE_PREFIX + "surprise-member-timeout", "3000");
+    InternalDistributedSystem system = getSystem();
+    GMSMembershipManager membershipManager =
+        (GMSMembershipManager) MembershipManagerHelper.getMembershipManager(system);
+    assertThat(membershipManager.isCleanupTimerStarted()).isTrue();
+
+    InternalDistributedMember member = new InternalDistributedMember(getIPLiteral(), 12345);
+
+    // first make sure we can't add this as a surprise member (bug #44566)
+    // if the view number isn't being recorded correctly the test will pass but the
+    // functionality is broken
+    assertThat(membershipManager.getView().getViewId()).isGreaterThan(0);
+
+    int oldViewId = member.getVmViewId();
+    member.setVmViewId(membershipManager.getView().getViewId() - 1);
+
+    addIgnoredException("attempt to add old member");
+    addIgnoredException("Removing shunned GemFire node");
+
+    boolean accepted = membershipManager.addSurpriseMember(member);
+    assertThat(accepted).as("member with old ID was not rejected (bug #44566)").isFalse();
+
+    member.setVmViewId(oldViewId);
+
+    // now forcibly add it as a surprise member and show that it is reaped
+    long gracePeriod = 5000;
+    long startTime = System.currentTimeMillis();
+    long timeout = membershipManager.getSurpriseMemberTimeout();
+    long birthTime = startTime - timeout + gracePeriod;
+    MembershipManagerHelper.addSurpriseMember(system, member, birthTime);
+    assertThat(membershipManager.isSurpriseMember(member)).as("Member was not a surprise member")
+        .isTrue();
+
+    await("waiting for member to be removed")
+        .atMost((timeout / 3) + gracePeriod, TimeUnit.MILLISECONDS)
+        .until(() -> !membershipManager.isSurpriseMember(member));
+  }
+
+  /**
+   * Tests that a severe-level alert is generated if a member does not respond with an ack quickly
+   * enough. vm0 and vm1 create a region and set ack-severe-alert-threshold. vm1 has a cache
+   * listener in its region that sleeps when notified, forcing the operation to take longer than
+   * ack-wait-threshold + ack-severe-alert-threshold
+   */
+  @Test
+  public void testAckSevereAlertThreshold() throws Exception {
+    // in order to set a small ack-wait-threshold, we have to remove the
+    // system property established by the dunit harness
+    System.clearProperty(GEMFIRE_PREFIX + ACK_WAIT_THRESHOLD);
+
+    Properties config = getDistributedSystemProperties();
+    config.setProperty(MCAST_PORT, "0");
+    config.setProperty(ACK_WAIT_THRESHOLD, "3");
+    config.setProperty(ACK_SEVERE_ALERT_THRESHOLD, "3");
+    config.setProperty(NAME, "putter");
+
+    getSystem(config);
+    Region<String, String> region =
+        new RegionFactory<String, String>().setScope(Scope.DISTRIBUTED_ACK).setEarlyAck(false)
+            .setDataPolicy(DataPolicy.REPLICATE).create("testRegion");
+
+    vm1.invoke("Connect to distributed system", () -> {
+      config.setProperty(NAME, "sleeper");
+      getSystem(config);
+      addIgnoredException("elapsed while waiting for replies");
+
+      RegionFactory<String, String> regionFactory = new RegionFactory<>();
+      Region<String, String> sameRegion =
+          regionFactory.setScope(Scope.DISTRIBUTED_ACK).setDataPolicy(DataPolicy.REPLICATE)
+              .setEarlyAck(false).addCacheListener(getSleepingListener(false)).create("testRegion");
+      myCache = sameRegion.getCache();
+
+      createAlertListener();
+    });
+
+    // now we have two caches set up. vm1 has a listener that will sleep
+    // and cause the severe-alert threshold to be crossed
+
+    region.put("bomb", "pow!"); // this will hang until vm1 responds
+    disconnectAllFromDS();
+
+    vm1.invoke(() -> {
+      assertThat(alertReceived).isTrue();
+    });
+  }
+
+  /**
+   * Tests that a sick member is kicked out
+   */
+  @Test
+  public void testKickOutSickMember() throws Exception {
+    addIgnoredException("10 seconds have elapsed while waiting");
+
+    // in order to set a small ack-wait-threshold, we have to remove the
+    // system property established by the dunit harness
+    System.clearProperty(GEMFIRE_PREFIX + ACK_WAIT_THRESHOLD);
+
+    Properties config = getDistributedSystemProperties();
+    config.setProperty(MCAST_PORT, "0"); // loner
+    config.setProperty(ACK_WAIT_THRESHOLD, "5");
+    config.setProperty(ACK_SEVERE_ALERT_THRESHOLD, "5");
+    config.setProperty(NAME, "putter");
+
+    getSystem(config);
+    Region<String, String> region = new RegionFactory<String, String>()
+        .setScope(Scope.DISTRIBUTED_ACK).setDataPolicy(DataPolicy.REPLICATE).create("testRegion");
+
+    addIgnoredException("sec have elapsed while waiting for replies");
+
+    vm1.invoke(new SerializableRunnable("Connect to distributed system") {
+      @Override
+      public void run() {
+        config.setProperty(NAME, "sleeper");
+        getSystem(config);
+
+        addIgnoredException("service failure");
+        addIgnoredException(ForcedDisconnectException.class.getName());
+
+        RegionFactory<String, String> regionFactory = new RegionFactory<>();
+        Region sameRegion =
+            regionFactory.setScope(Scope.DISTRIBUTED_ACK).setDataPolicy(DataPolicy.REPLICATE)
+                .addCacheListener(getSleepingListener(true)).create("testRegion");
+        myCache = sameRegion.getCache();
+      }
+    });
+
+    // now we have two caches set up, each having an alert listener. Vm1
+    // also has a cache listener that will turn off its ability to respond
+    // to "are you dead" messages and then sleep
+
+    region.put("bomb", "pow!");
+    disconnectFromDS();
+
+    vm1.invoke("wait for forced disconnect", () -> {
+      await("vm1's system should have been disconnected").atMost(1, MINUTES)
+          .until(() -> assertThat(basicGetSystem().isConnected()).isFalse());
+
+      await("vm1's cache is not closed").atMost(30, SECONDS)
+          .until(() -> assertThat(myCache.isClosed()).isTrue());
+
+      await("vm1's listener should have received afterRegionDestroyed notification")
+          .atMost(30, SECONDS).until(() -> assertThat(regionDestroyedInvoked).isTrue());
+    });
+  }
+
+  /**
+   * test use of a bad bind-address for bug #32565
+   */
+  @Test
+  public void testBadBindAddress() throws Exception {
+    Properties config = getDistributedSystemProperties();
+    config.setProperty(MCAST_PORT, "0"); // loner
+    config.setProperty(ACK_WAIT_THRESHOLD, "5");
+    config.setProperty(ACK_SEVERE_ALERT_THRESHOLD, "5");
+
+    // use a valid address that's not proper for this machine
+    config.setProperty(BIND_ADDRESS, "www.yahoo.com");
+    assertThatThrownBy(() -> getSystem(config)).isInstanceOf(IllegalArgumentException.class);
+
+    // use an invalid address
+    config.setProperty(BIND_ADDRESS, "bruce.schuchardt");
+    assertThatThrownBy(() -> getSystem(config)).isInstanceOf(IllegalArgumentException.class);
+
+    // use a valid bind address
+    config.setProperty(BIND_ADDRESS, InetAddress.getLocalHost().getCanonicalHostName());
+    assertThatCode(() -> getSystem()).doesNotThrowAnyException();
+  }
+
+  /**
+   * install a new view and show that waitForViewInstallation works as expected
+   */
+  @Test
+  public void testWaitForViewInstallation() {
+    InternalDistributedSystem system = getSystem(new Properties());
+    ClusterDistributionManager dm = (ClusterDistributionManager) system.getDM();
+    GMSMembershipManager membershipManager = (GMSMembershipManager) dm.getMembershipManager();
+    NetView view = membershipManager.getView();
+
+    AtomicBoolean waitForViewInstallationDone = new AtomicBoolean();
+    executorService.submit(() -> {
+      try {
+        dm.waitForViewInstallation(view.getViewId() + 1);
+        waitForViewInstallationDone.set(true);
+      } catch (InterruptedException e) {
+        errorCollector.addError(e);
+      }
+    });
+
+    pause(2000);
+
+    NetView newView = new NetView(view, view.getViewId() + 1);
+    membershipManager.installView(newView);
+
+    await().atMost(30, SECONDS).until(() -> assertThat(waitForViewInstallationDone.get()).isTrue());
+  }
+
+  private CacheListener<String, String> getSleepingListener(final boolean playDead) {
+    regionDestroyedInvoked = false;
+
+    return new CacheListenerAdapter<String, String>() {
+      @Override
+      public void afterCreate(EntryEvent event) {
+        try {
+          if (playDead) {
+            MembershipManagerHelper.beSickMember(getSystemStatic());
+            MembershipManagerHelper.playDead(getSystemStatic());
+          }
+          Thread.sleep(15 * 1000);
+        } catch (InterruptedException ie) {
+          errorCollector.addError(ie);
+        }
+      }
+
+      @Override
+      public void afterRegionDestroy(RegionEvent event) {
+        LogWriter logWriter = myCache.getLogger();
+        logWriter.info("afterRegionDestroyed invoked in sleeping listener");
+        logWriter.info("<ExpectedException action=remove>service failure</ExpectedException>");
+        logWriter.info(
+            "<ExpectedException action=remove>org.apache.geode.ForcedDisconnectException</ExpectedException>");
+        regionDestroyedInvoked = true;
+      }
+    };
+  }
+
+  private void createAlertListener() throws Exception {
+    DistributedSystemConfig config =
+        AdminDistributedSystemFactory.defineDistributedSystem(getSystemStatic(), null);
+    AdminDistributedSystem adminSystem = AdminDistributedSystemFactory.getDistributedSystem(config);
+    adminSystem.setAlertLevel(AlertLevel.SEVERE);
+    adminSystem.addAlertListener(alert -> {
+      try {
+        logger.info("alert listener invoked for alert originating in " + alert.getConnectionName());
+        logger.info("  alert text = " + alert.getMessage());
+        logger.info("  systemMember = " + alert.getSystemMember());
+      } catch (Exception e) {
+        errorCollector.addError(e);
+      }
+      alertReceived = true;
+    });
+    adminSystem.connect();
+    assertTrue(adminSystem.waitToBeConnected(5 * 1000));
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerForAdminDUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerForAdminDUnitTest.java
new file mode 100644
index 0000000..37191ff
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerForAdminDUnitTest.java
@@ -0,0 +1,317 @@
+/*
+ * 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;
+
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.geode.distributed.ConfigurationProperties.STATISTIC_SAMPLING_ENABLED;
+import static org.apache.geode.test.dunit.LogWriterUtils.getLogWriter;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+import java.net.InetAddress;
+import java.util.Set;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.AttributesFactory;
+import org.apache.geode.cache.CacheException;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.distributed.DistributedLockService;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.Config;
+import org.apache.geode.internal.admin.Alert;
+import org.apache.geode.internal.admin.AlertListener;
+import org.apache.geode.internal.admin.ApplicationVM;
+import org.apache.geode.internal.admin.DLockInfo;
+import org.apache.geode.internal.admin.EntryValueNode;
+import org.apache.geode.internal.admin.GemFireVM;
+import org.apache.geode.internal.admin.GfManagerAgent;
+import org.apache.geode.internal.admin.GfManagerAgentConfig;
+import org.apache.geode.internal.admin.GfManagerAgentFactory;
+import org.apache.geode.internal.admin.StatResource;
+import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.cache.CacheTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.FlakyTest;
+
+/**
+ * This class tests the functionality of the {@linkplain org.apache.geode.internal.admin internal
+ * admin} API.
+ */
+@Category(DistributedTest.class)
+public class ClusterDistributionManagerForAdminDUnitTest extends CacheTestCase
+    implements AlertListener {
+
+  private static Logger logger = LogService.getLogger();
+
+  private transient GfManagerAgent agent;
+
+  @Before
+  public void setUp() throws Exception {
+    IgnoredException.addIgnoredException("Error occurred while reading system log");
+
+    ClusterDistributionManager.setIsDedicatedAdminVM(true);
+
+    populateCache();
+
+    boolean created = !isConnectedToDS();
+    InternalDistributedSystem ds = getSystem();
+    RemoteTransportConfig transport =
+        new RemoteTransportConfig(ds.getConfig(), ClusterDistributionManager.ADMIN_ONLY_DM_TYPE);
+    if (created) {
+      disconnectFromDS();
+    }
+
+    // create a GfManagerAgent in the master vm.
+    this.agent = GfManagerAgentFactory.getManagerAgent(
+        new GfManagerAgentConfig(null, transport, getLogWriter(), Alert.SEVERE, this, null));
+
+    await().atMost(1, MINUTES).until(() -> assertThat(agent.isConnected()).isTrue());
+  }
+
+  @After
+  public void preTearDownCacheTestCase() throws Exception {
+    try {
+      if (this.agent != null) {
+        this.agent.disconnect();
+      }
+      disconnectFromDS();
+    } finally {
+      ClusterDistributionManager.setIsDedicatedAdminVM(false);
+    }
+  }
+
+  @Test
+  public void testGetDistributionVMType() {
+    DistributionManager dm = this.agent.getDM();
+    assertThat(dm.getId().getVmKind()).isEqualTo(ClusterDistributionManager.ADMIN_ONLY_DM_TYPE);
+  }
+
+  @Test
+  public void testAgent() {
+    assertThat(agent.listPeers()).hasSize(0);
+    assertThat(agent.isConnected()).isTrue();
+    agent.disconnect();
+    assertThat(agent.isConnected()).isFalse();
+  }
+
+  @Category(FlakyTest.class) // GEODE-1688
+  @Test
+  public void testApplications() throws Exception {
+    await().atMost(1, MINUTES)
+        .until(() -> assertThat(agent.listApplications().length).isGreaterThanOrEqualTo(4));
+
+    ApplicationVM[] applications = agent.listApplications();
+    for (int whichApplication = 0; whichApplication < applications.length; whichApplication++) {
+
+      InetAddress host = applications[whichApplication].getHost();
+      String appHostName = host.getHostName();
+
+      assertThat(host).isEqualTo(InetAddress.getByName(appHostName));
+
+      StatResource[] stats = applications[whichApplication].getStats(null);
+      assertThat(stats.length).isGreaterThan(0);
+
+      Config config = applications[whichApplication].getConfig();
+      String[] attributeNames = config.getAttributeNames();
+      boolean foundStatisticSamplingEnabled = false;
+      for (String attributeName : attributeNames) {
+        if (attributeName.equals(STATISTIC_SAMPLING_ENABLED)) {
+          foundStatisticSamplingEnabled = true;
+          assertThat(config.getAttribute(attributeName)).isEqualTo("true");
+          break;
+        }
+      }
+      assertThat(foundStatisticSamplingEnabled).isTrue();
+
+      String[] logs = applications[whichApplication].getSystemLogs();
+      assertThat(logs.length).isGreaterThan(0);
+
+      VM vm = findVMForAdminObject(applications[whichApplication]);
+      assertThat(vm).isNotNull();
+
+      String lockName = "cdm_testlock" + whichApplication;
+      assertThat(acquireDistributedLock(vm, lockName)).isTrue();
+
+      DLockInfo[] locks = applications[whichApplication].getDistributedLockInfo();
+      assertThat(locks.length).isGreaterThan(0);
+
+      boolean foundLock = false;
+      for (DLockInfo lock : locks) {
+        if (lock.getLockName().equals(lockName)) {
+          foundLock = true;
+          assertThat(lock.isAcquired()).isTrue();
+        }
+      }
+      assertThat(foundLock).isTrue();
+
+      Region[] roots = applications[whichApplication].getRootRegions();
+      assertThat(roots.length).isGreaterThan(0);
+
+      Region root = roots[0];
+      assertThat(root).isNotNull();
+      assertThat(root.getName()).isEqualTo("root");
+      assertThat(root.getFullPath()).isEqualTo("/root");
+
+      RegionAttributes attributes = root.getAttributes();
+      assertThat(attributes).isNotNull();
+      if (attributes.getStatisticsEnabled()) {
+        assertThat(root.getStatistics()).isNotNull();
+      }
+
+      Set subregions = root.subregions(false);
+      assertThat(subregions).hasSize(3);
+      assertThat(root.keySet()).hasSize(2);
+
+      Region.Entry entry = root.getEntry("cacheObj1");
+      assertThat(entry).isNotNull();
+      if (attributes.getStatisticsEnabled()) {
+        assertThat(entry.getStatistics()).isNotNull();
+      }
+      assertThat(entry.getValue()).isEqualTo("null");
+
+      /// test lightweight inspection;
+      entry = root.getEntry("cacheObj2");
+      assertThat(entry).isNotNull();
+
+      Object val = entry.getValue();
+      assertThat(val).isInstanceOf(String.class);
+      assertThat(((String) val)).contains("java.lang.StringBuffer");
+
+      /// test physical inspection
+      applications[whichApplication].setCacheInspectionMode(GemFireVM.PHYSICAL_CACHE_VALUE);
+      entry = root.getEntry("cacheObj2");
+      assertThat(entry).isNotNull();
+
+      val = entry.getValue();
+      assertThat(val).isInstanceOf(EntryValueNode.class);
+
+      EntryValueNode node = (EntryValueNode) val;
+      String type = node.getType();
+      assertThat(type).contains("java.lang.StringBuffer");
+      assertThat(node.isPrimitiveOrString()).isFalse();
+
+      EntryValueNode[] fields = node.getChildren();
+      assertThat(fields).isNotNull();
+
+      getLogWriter().warning(
+          "The tests use StringBuffers for values which might be implemented differently in jdk 1.5");
+
+      /// test destruction in the last valid app
+      int lastApplication = applications.length - 1;
+      if (whichApplication == lastApplication) {
+        int expectedSize = subregions.size() - 1;
+        Region subRegion = (Region) subregions.iterator().next();
+        Region rootRegion = root;
+        subRegion.destroyRegion();
+
+        await().atMost(30, SECONDS)
+            .until(() -> assertThat(rootRegion.subregions(false)).hasSize(expectedSize));
+      }
+    }
+  }
+
+  @Override
+  public void alert(Alert alert) {
+    getLogWriter().info("DEBUG: alert=" + alert);
+  }
+
+  private void populateCache() {
+    AttributesFactory attributesFactory = new AttributesFactory();
+    attributesFactory.setScope(Scope.DISTRIBUTED_NO_ACK);
+
+    RegionAttributes regionAttributes = attributesFactory.create();
+
+    for (int i = 0; i < Host.getHostCount(); i++) {
+      Host host = Host.getHost(i);
+
+      for (int j = 0; j < host.getVMCount(); j++) {
+        VM vm = host.getVM(j);
+        vm.invoke(() -> {
+          createRegion("cdm-testSubRegion1", regionAttributes);
+          createRegion("cdm-testSubRegion2", regionAttributes);
+          createRegion("cdm-testSubRegion3", regionAttributes);
+          remoteCreateEntry("", "cacheObj1", null);
+          StringBuffer val = new StringBuffer("userDefValue1");
+          remoteCreateEntry("", "cacheObj2", val);
+        });
+      }
+    }
+  }
+
+  /**
+   * Puts (or creates) a value in a region named <code>regionName</code> named
+   * <code>entryName</code>.
+   */
+  private void remoteCreateEntry(String regionName, String entryName, Object value)
+      throws CacheException {
+
+    Region root = getRootRegion();
+    Region region = root.getSubregion(regionName);
+    region.create(entryName, value);
+
+    logger.info("Put value " + value + " in entry " + entryName + " in region '"
+        + region.getFullPath() + "'");
+  }
+
+  private boolean acquireDistributedLock(VM vm, String lockName) {
+    return vm.invoke(() -> remoteAcquireDistLock(lockName));
+  }
+
+  private boolean remoteAcquireDistLock(String lockName) {
+    String serviceName = "cdmtest_service";
+    DistributedLockService service = DistributedLockService.getServiceNamed(serviceName);
+    if (service == null) {
+      service =
+          DistributedLockService.create(serviceName, InternalDistributedSystem.getAnyInstance());
+    }
+    assertThat(service).isNotNull();
+    return service.lock(lockName, 1000, 3000);
+  }
+
+  private VM findVMForAdminObject(GemFireVM gemFireVM) {
+    for (int i = 0; i < Host.getHostCount(); i++) {
+      Host host = Host.getHost(i);
+      for (int j = 0; j < host.getVMCount(); j++) {
+        VM vm = host.getVM(j);
+        InternalDistributedMember member = getJavaGroupsIdForVM(vm);
+        if (gemFireVM.getId().equals(member)) {
+          return vm;
+        }
+      }
+    }
+    return null;
+  }
+
+  private InternalDistributedMember getJavaGroupsIdForVM(VM vm) {
+    return vm.invoke(() -> remoteGetJavaGroupsIdForVM());
+  }
+
+  private InternalDistributedMember remoteGetJavaGroupsIdForVM() {
+    InternalDistributedSystem system = InternalDistributedSystem.getAnyInstance();
+    return system.getDistributionManager().getDistributionManagerId();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionManagerTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerTest.java
similarity index 97%
rename from geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionManagerTest.java
rename to geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerTest.java
index 6050d68..a941cfc 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionManagerTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/ClusterDistributionManagerTest.java
@@ -29,15 +29,17 @@ import org.apache.geode.distributed.internal.membership.InternalDistributedMembe
 import org.apache.geode.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
-public class DistributionManagerTest {
+public class ClusterDistributionManagerTest {
 
   @Test
   public void shouldBeMockable() throws Exception {
     ClusterDistributionManager mockDistributionManager = mock(ClusterDistributionManager.class);
     InternalDistributedMember mockInternalDistributedMember = mock(InternalDistributedMember.class);
     Executor mockExecutor = mock(Executor.class);
+
     when(mockDistributionManager.getExecutor(anyInt(), eq(mockInternalDistributedMember)))
         .thenReturn(mockExecutor);
+
     assertThat(mockDistributionManager.getExecutor(1, mockInternalDistributedMember))
         .isSameAs(mockExecutor);
   }
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/ConsoleDistributionManagerDUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/ConsoleDistributionManagerDUnitTest.java
deleted file mode 100644
index 8e100a7..0000000
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/ConsoleDistributionManagerDUnitTest.java
+++ /dev/null
@@ -1,446 +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;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Set;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.SystemFailure;
-import org.apache.geode.cache.AttributesFactory;
-import org.apache.geode.cache.CacheException;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.RegionAttributes;
-import org.apache.geode.cache.Scope;
-import org.apache.geode.distributed.DistributedLockService;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.Config;
-import org.apache.geode.internal.admin.Alert;
-import org.apache.geode.internal.admin.AlertListener;
-import org.apache.geode.internal.admin.ApplicationVM;
-import org.apache.geode.internal.admin.DLockInfo;
-import org.apache.geode.internal.admin.EntryValueNode;
-import org.apache.geode.internal.admin.GemFireVM;
-import org.apache.geode.internal.admin.GfManagerAgent;
-import org.apache.geode.internal.admin.GfManagerAgentConfig;
-import org.apache.geode.internal.admin.GfManagerAgentFactory;
-import org.apache.geode.internal.admin.StatResource;
-import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.IgnoredException;
-import org.apache.geode.test.dunit.LogWriterUtils;
-import org.apache.geode.test.dunit.SerializableRunnable;
-import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.Wait;
-import org.apache.geode.test.dunit.WaitCriterion;
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.categories.FlakyTest;
-
-/**
- * This class tests the functionality of the {@linkplain org.apache.geode.internal.admin internal
- * admin} API.
- */
-@Category(DistributedTest.class)
-public class ConsoleDistributionManagerDUnitTest extends JUnit4CacheTestCase
-    implements AlertListener {
-
-  protected GfManagerAgent agent = null;
-  private static boolean firstTime = true;
-
-  public void alert(Alert alert) {
-    LogWriterUtils.getLogWriter().info("DEBUG: alert=" + alert);
-  }
-
-  @Override
-  public final void postSetUp() throws Exception {
-    boolean finishedSetup = false;
-    IgnoredException.addIgnoredException("Error occurred while reading system log");
-    try {
-      if (firstTime) {
-        disconnectFromDS(); // make sure there's no ldm lying around
-        try {
-          Thread.sleep(5 * 1000);
-        } catch (InterruptedException ie) {
-          fail("interrupted");
-        }
-        firstTime = false;
-      }
-
-      ClusterDistributionManager.setIsDedicatedAdminVM(true);
-
-      populateCache();
-
-      RemoteTransportConfig transport = null;
-      {
-        boolean created = !isConnectedToDS();
-        InternalDistributedSystem ds = getSystem();
-        transport = new RemoteTransportConfig(ds.getConfig(),
-            ClusterDistributionManager.ADMIN_ONLY_DM_TYPE);
-        if (created) {
-          disconnectFromDS();
-        }
-      }
-      // create a GfManagerAgent in the master vm.
-      this.agent = GfManagerAgentFactory.getManagerAgent(new GfManagerAgentConfig(null, transport,
-          LogWriterUtils.getLogWriter(), Alert.SEVERE, this, null));
-      if (!agent.isConnected()) {
-        WaitCriterion ev = new WaitCriterion() {
-          public boolean done() {
-            return agent.isConnected();
-          }
-
-          public String description() {
-            return null;
-          }
-        };
-        Wait.waitForCriterion(ev, 60 * 1000, 200, true);
-      }
-      finishedSetup = true;
-    } finally {
-      if (!finishedSetup) {
-        try {
-          this.agent.disconnect();
-        } catch (VirtualMachineError e) {
-          SystemFailure.initiateFailure(e);
-          throw e;
-        } catch (Throwable ignore) {
-        }
-        try {
-          super.preTearDown();
-        } catch (VirtualMachineError e) {
-          SystemFailure.initiateFailure(e);
-          throw e;
-        } catch (Throwable ignore) {
-        }
-        try {
-          disconnectFromDS();
-        } catch (VirtualMachineError e) {
-          SystemFailure.initiateFailure(e);
-          throw e;
-        } catch (Throwable ignore) {
-        }
-        ClusterDistributionManager.setIsDedicatedAdminVM(false);
-      }
-    }
-  }
-
-  @Override
-  public final void preTearDownCacheTestCase() throws Exception {
-    this.agent.disconnect();
-  }
-
-  @Override
-  public final void postTearDownCacheTestCase() throws Exception {
-    try {
-      disconnectFromDS(); // make sure there's no ldm lying around
-    } finally {
-      ClusterDistributionManager.setIsDedicatedAdminVM(false);
-    }
-  }
-
-  @Test
-  public void testGetDistributionVMType() {
-    DistributionManager dm = this.agent.getDM();
-    InternalDistributedMember ipaddr = dm.getId();
-    assertEquals(ClusterDistributionManager.ADMIN_ONLY_DM_TYPE, ipaddr.getVmKind());
-  }
-
-  @Test
-  public void testAgent() {
-    assertEquals("expected empty peer array", 0, agent.listPeers().length);
-    int systemCount = 0;
-    for (int h = 0; h < Host.getHostCount(); h++) {
-      Host host = Host.getHost(h);
-      systemCount += host.getSystemCount();
-    }
-    // note that JoinLeaveListener is not tested since it would require
-    // this test to start and stop systems.
-    agent.disconnect();
-    assertTrue("agent should have been disconnected", !agent.isConnected());
-  }
-
-  @Category(FlakyTest.class) // GEODE-1688
-  @Test
-  public void testApplications() throws Exception {
-    {
-      WaitCriterion ev = new WaitCriterion() {
-        public boolean done() {
-          ApplicationVM[] apps = agent.listApplications();
-          return apps.length >= 4;
-        }
-
-        public String description() {
-          return null;
-        }
-      };
-      Wait.waitForCriterion(ev, 60 * 1000, 200, true);
-    }
-
-    // final Serializable controllerId = getSystem().getDistributionManager().getId(); //can't do
-    // this...
-    ApplicationVM[] apps = agent.listApplications();
-    for (int i = 0; i < apps.length; i++) {
-      // if (apps[i].getId().equals(controllerId)) {
-      // continue; // skip this one; its the locator vm
-      // }
-      InetAddress host = apps[i].getHost();
-      String appHostName = host.getHostName();
-      try {
-        InetAddress appHost = InetAddress.getByName(appHostName);
-        assertEquals(appHost, host);
-      } catch (UnknownHostException ex) {
-        fail("Lookup of address for host " + appHostName + " failed because " + ex);
-      }
-
-      StatResource[] stats = apps[i].getStats(null);
-      assertTrue(stats.length > 0);
-
-      Config conf = apps[i].getConfig();
-      String[] attNames = conf.getAttributeNames();
-      boolean foundIt = false;
-      for (int j = 0; j < attNames.length; j++) {
-        if (attNames[j].equals(STATISTIC_SAMPLING_ENABLED)) {
-          foundIt = true;
-          assertEquals(conf.getAttribute(attNames[j]), "true");
-          break;
-        }
-      }
-      assertTrue(foundIt);
-
-      String[] logs = apps[i].getSystemLogs();
-      assertTrue(logs.length > 0);
-      // assertTrue(logs[0].length() > 0);
-
-      {
-        VM vm = findVMForAdminObject(apps[i]);
-        // getLogWriter().info("DEBUG: found VM " + vm +
-        // " which corresponds to ApplicationVM "
-        // + apps[i] + " in testApplications");
-        assertNotNull(vm);
-        String lockName = "cdm_testlock" + i;
-        assertTrue(acquireDistLock(vm, lockName));
-        DLockInfo[] locks = apps[i].getDistributedLockInfo();
-        assertTrue(locks.length > 0);
-        boolean foundLock = false;
-        for (int j = 0; j < locks.length; j++) {
-          if (locks[j].getLockName().equals(lockName)) {
-            foundLock = true;
-            assertTrue(locks[j].isAcquired());
-          }
-        }
-        assertTrue(foundLock);
-      }
-
-      Region[] roots = apps[i].getRootRegions();
-      if (roots.length == 0) {
-        LogWriterUtils.getLogWriter().info(
-            "DEBUG: testApplications: apps[" + i + "]=" + apps[i] + " did not have a root region");
-      } else {
-        Region root = roots[0];
-        assertNotNull(root);
-        assertEquals("root", root.getName());
-        assertEquals("/root", root.getFullPath());
-        RegionAttributes attributes = root.getAttributes();
-        assertNotNull(attributes);
-        if (attributes.getStatisticsEnabled()) {
-          assertNotNull(root.getStatistics());
-        }
-        final Set subregions = root.subregions(false);
-        assertEquals(3, subregions.size());
-        assertEquals(2, root.keySet().size());
-        Region.Entry entry = root.getEntry("cacheObj1");
-        assertNotNull(entry);
-        if (attributes.getStatisticsEnabled()) {
-          assertNotNull(entry.getStatistics());
-        }
-        assertTrue(entry.getValue().equals("null"));
-
-        /// test lightweight inspection;
-        entry = root.getEntry("cacheObj2");
-        assertNotNull(entry);
-        Object val = entry.getValue();
-        assertTrue(val instanceof String);
-        assertTrue(((String) val).indexOf("java.lang.StringBuffer") != -1);
-
-        /// test physical inspection
-        // getLogWriter().info("DEBUG: Starting test of physical cache value inspection");
-        apps[i].setCacheInspectionMode(GemFireVM.PHYSICAL_CACHE_VALUE);
-        entry = root.getEntry("cacheObj2");
-        assertNotNull(entry);
-        val = entry.getValue();
-        assertTrue(val instanceof EntryValueNode);
-        EntryValueNode node = (EntryValueNode) val;
-        String type = node.getType();
-        assertTrue(type.indexOf("java.lang.StringBuffer") != -1);
-        assertTrue(!node.isPrimitiveOrString());
-        EntryValueNode[] fields = node.getChildren();
-        assertNotNull(fields);
-        LogWriterUtils.getLogWriter().warning(
-            "The tests use StringBuffers for values which might be implmented differently in jdk 1.5");
-        // assertTrue(fields.length > 0);
-
-        /// test destruction in the last valid app
-        int lastIdx = apps.length - 1;
-
-        /*
-         * if (i == lastIdx || (i == lastIdx-1 && apps[lastIdx].getId().equals(controllerId))) {
-         */
-
-        if (i == lastIdx) {
-          // getLogWriter().info("DEBUG: starting region destroy from admin apis");
-          final int expectedSize = subregions.size() - 1;
-          final Region r = (Region) subregions.iterator().next();
-          final Region rr = root;
-          r.destroyRegion();
-          WaitCriterion ev = new WaitCriterion() {
-            public boolean done() {
-              Set s = rr.subregions(false);
-              return s.size() == expectedSize;
-            }
-
-            public String description() {
-              return "Waited 20 seconds for region " + r.getFullPath() + "to be destroyed.";
-            }
-          };
-          Wait.waitForCriterion(ev, 20 * 1000, 200, true);
-        }
-      }
-    }
-  }
-
-  private void populateCache() {
-
-    AttributesFactory fact = new AttributesFactory();
-    fact.setScope(Scope.DISTRIBUTED_NO_ACK);
-    final RegionAttributes rAttr = fact.create();
-
-    final SerializableRunnable populator = new SerializableRunnable() {
-      public void run() {
-        try {
-          createRegion("cdm-testSubRegion1", rAttr);
-          createRegion("cdm-testSubRegion2", rAttr);
-          createRegion("cdm-testSubRegion3", rAttr);
-          remoteCreateEntry("", "cacheObj1", null);
-          StringBuffer val = new StringBuffer("userDefValue1");
-          remoteCreateEntry("", "cacheObj2", val);
-        } catch (CacheException ce) {
-          fail("Exception while populating cache:\n" + ce);
-        }
-      }
-    };
-
-    for (int h = 0; h < Host.getHostCount(); h++) {
-      Host host = Host.getHost(h);
-
-      for (int v = 0; v < host.getVMCount(); v++) {
-        VM vm = host.getVM(v);
-        vm.invoke(populator);
-      }
-    }
-  }
-
-
-
-  /**
-   * Puts (or creates) a value in a region named <code>regionName</code> named
-   * <code>entryName</code>.
-   */
-  protected void remoteCreateEntry(String regionName, String entryName, Object value)
-      throws CacheException {
-
-    Region root = getRootRegion();
-    Region region = root.getSubregion(regionName);
-    region.create(entryName, value);
-
-
-    LogWriterUtils.getLogWriter().info("Put value " + value + " in entry " + entryName
-        + " in region '" + region.getFullPath() + "'");
-
-  }
-
-  // private long getConId(VM vm) {
-  // return vm.invoke(() -> this.remoteGetConId());
-  // }
-  /**
-   * Accessed via reflection. DO NOT REMOVE
-   *
-   * @return
-   */
-  protected static long remoteGetConId() {
-    return InternalDistributedSystem.getAnyInstance().getId();
-  }
-
-  private boolean acquireDistLock(VM vm, String lockName) {
-    return vm.invoke(() -> remoteAcquireDistLock(lockName));
-  }
-
-  /**
-   * Accessed via reflection. DO NOT REMOVE
-   *
-   * @param lockName
-   * @return
-   */
-  protected static boolean remoteAcquireDistLock(String lockName) {
-    String serviceName = "cdmtest_service";
-    DistributedLockService service = DistributedLockService.getServiceNamed(serviceName);
-    if (service == null) {
-      service =
-          DistributedLockService.create(serviceName, InternalDistributedSystem.getAnyInstance());
-    }
-    assertNotNull(service);
-    try {
-      return service.lock(lockName, 1000, 3000);
-    } catch (Exception e) {
-      throw new RuntimeException("DEBUG: remoteAcquireDistLock", e);
-      // return false;
-    }
-  }
-
-  private VM findVMForAdminObject(GemFireVM adminObj) {
-    for (int i = 0; i < Host.getHostCount(); i++) {
-      Host host = Host.getHost(i);
-      for (int j = 0; j < host.getVMCount(); j++) {
-        VM vm = host.getVM(j);
-        InternalDistributedMember id = getJavaGroupsIdForVM(vm);
-        if (adminObj.getId().equals(id)) {
-          return vm;
-        }
-      }
-    }
-    return null;
-  }
-
-  private InternalDistributedMember getJavaGroupsIdForVM(VM vm) {
-    return (InternalDistributedMember) vm.invoke(() -> remoteGetJavaGroupsIdForVM());
-  }
-
-  /**
-   * INVOKED VIA REFLECTION
-   *
-   * @return
-   */
-  protected static InternalDistributedMember remoteGetJavaGroupsIdForVM() {
-    InternalDistributedSystem sys = InternalDistributedSystem.getAnyInstance();
-    return sys.getDistributionManager().getDistributionManagerId();
-
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionManagerDUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionManagerDUnitTest.java
deleted file mode 100644
index 6f8a421..0000000
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/DistributionManagerDUnitTest.java
+++ /dev/null
@@ -1,545 +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;
-
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.apache.geode.test.dunit.Assert.*;
-
-import java.net.InetAddress;
-import java.util.Properties;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.logging.log4j.Logger;
-import org.awaitility.Awaitility;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.LogWriter;
-import org.apache.geode.admin.AdminDistributedSystem;
-import org.apache.geode.admin.AdminDistributedSystemFactory;
-import org.apache.geode.admin.Alert;
-import org.apache.geode.admin.AlertLevel;
-import org.apache.geode.admin.AlertListener;
-import org.apache.geode.admin.DistributedSystemConfig;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheListener;
-import org.apache.geode.cache.DataPolicy;
-import org.apache.geode.cache.EntryEvent;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.RegionEvent;
-import org.apache.geode.cache.RegionFactory;
-import org.apache.geode.cache.Scope;
-import org.apache.geode.cache.util.CacheListenerAdapter;
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.MembershipManager;
-import org.apache.geode.distributed.internal.membership.NetView;
-import org.apache.geode.distributed.internal.membership.gms.MembershipManagerHelper;
-import org.apache.geode.distributed.internal.membership.gms.interfaces.Manager;
-import org.apache.geode.distributed.internal.membership.gms.mgr.GMSMembershipManager;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.IgnoredException;
-import org.apache.geode.test.dunit.NetworkUtils;
-import org.apache.geode.test.dunit.SerializableRunnable;
-import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.Wait;
-import org.apache.geode.test.dunit.WaitCriterion;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.dunit.rules.DistributedRestoreSystemProperties;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.categories.MembershipTest;
-
-/**
- * This class tests the functionality of the {@link ClusterDistributionManager} class.
- */
-@Category({DistributedTest.class, MembershipTest.class})
-public class DistributionManagerDUnitTest extends JUnit4DistributedTestCase {
-  private static final Logger logger = LogService.getLogger();
-
-  public static DistributedSystem ds;
-
-  @Rule
-  public DistributedRestoreSystemProperties restoreSystemProperties =
-      new DistributedRestoreSystemProperties();
-
-  /**
-   * Clears the exceptionInThread flag in the given distribution manager.
-   */
-  public static void clearExceptionInThreads(ClusterDistributionManager dm) {
-    dm.clearExceptionInThreads();
-  }
-
-  @Override
-  public void preSetUp() throws Exception {
-    disconnectAllFromDS();
-  }
-
-  protected static class ItsOkayForMyClassNotToBeFound extends SerialDistributionMessage {
-
-    public int getDSFID() {
-      return NO_FIXED_ID;
-    }
-
-    @Override
-    protected void process(ClusterDistributionManager dm) {
-      // We should never get here
-    }
-  };
-
-  @Test
-  public void testGetDistributionVMType() {
-    DistributionManager dm = getSystem().getDistributionManager();
-    InternalDistributedMember ipaddr = dm.getId();
-    assertEquals(ClusterDistributionManager.NORMAL_DM_TYPE, ipaddr.getVmKind());
-  }
-
-  /**
-   * Send the distribution manager a message it can't deserialize
-   */
-  @Ignore
-  @Test
-  public void testExceptionInThreads() throws InterruptedException {
-    ClusterDistributionManager dm =
-        (ClusterDistributionManager) getSystem().getDistributionManager();
-    String p1 = "ItsOkayForMyClassNotToBeFound";
-    logger.info("<ExpectedException action=add>" + p1 + "</ExpectedException>");
-    DistributionMessage m = new ItsOkayForMyClassNotToBeFound();
-    dm.putOutgoing(m);
-    Thread.sleep(1 * 1000);
-    logger.info("<ExpectedException action=remove>" + p1 + "</ExpectedException>");
-    Awaitility.await("waiting for exceptionInThreads to be true").atMost(15, TimeUnit.SECONDS)
-        .until(() -> {
-          return dm.exceptionInThreads();
-        });
-    dm.clearExceptionInThreads();
-    assertTrue(!dm.exceptionInThreads());
-  }
-
-  /**
-   * Demonstrate that a new UDP port is used when an attempt is made to reconnect using a shunned
-   * port
-   */
-  @Test
-  public void testConnectAfterBeingShunned() {
-    InternalDistributedSystem sys = getSystem();
-    MembershipManager mgr = MembershipManagerHelper.getMembershipManager(sys);
-    InternalDistributedMember idm = mgr.getLocalMember();
-    // TODO GMS needs to have a system property allowing the bind-port to be set
-    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "jg-bind-port", "" + idm.getPort());
-    sys.disconnect();
-    sys = getSystem();
-    mgr = MembershipManagerHelper.getMembershipManager(sys);
-    sys.disconnect();
-    InternalDistributedMember idm2 = mgr.getLocalMember();
-    org.apache.geode.test.dunit.LogWriterUtils.getLogWriter()
-        .info("original ID=" + idm + " and after connecting=" + idm2);
-    assertTrue("should not have used a different udp port", idm.getPort() == idm2.getPort());
-  }
-
-  /**
-   * Test the handling of "surprise members" in the membership manager. Create a DistributedSystem
-   * in this VM and then add a fake member to its surpriseMember set. Then ensure that it stays in
-   * the set when a new membership view arrives that doesn't contain it. Then wait until the member
-   * should be gone and force more view processing to have it scrubbed from the set.
-   **/
-  @Test
-  public void testSurpriseMemberHandling() throws Exception {
-
-    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "surprise-member-timeout", "3000");
-    InternalDistributedSystem sys = getSystem();
-    MembershipManager mgr = MembershipManagerHelper.getMembershipManager(sys);
-    assertTrue(((GMSMembershipManager) mgr).isCleanupTimerStarted());
-
-    try {
-      InternalDistributedMember mbr =
-          new InternalDistributedMember(NetworkUtils.getIPLiteral(), 12345);
-
-      // first make sure we can't add this as a surprise member (bug #44566)
-
-      // if the view number isn't being recorded correctly the test will pass but the
-      // functionality is broken
-      Assert.assertTrue("expected view ID to be greater than zero", mgr.getView().getViewId() > 0);
-
-      int oldViewId = mbr.getVmViewId();
-      mbr.setVmViewId((int) mgr.getView().getViewId() - 1);
-      org.apache.geode.test.dunit.LogWriterUtils.getLogWriter()
-          .info("current membership view is " + mgr.getView());
-      org.apache.geode.test.dunit.LogWriterUtils.getLogWriter()
-          .info("created ID " + mbr + " with view ID " + mbr.getVmViewId());
-
-      IgnoredException.addIgnoredException("attempt to add old member");
-      IgnoredException.addIgnoredException("Removing shunned GemFire node");
-
-      boolean accepted = mgr.addSurpriseMember(mbr);
-      Assert.assertTrue("member with old ID was not rejected (bug #44566)", !accepted);
-
-      mbr.setVmViewId(oldViewId);
-
-      // now forcibly add it as a surprise member and show that it is reaped
-      long gracePeriod = 5000;
-      long startTime = System.currentTimeMillis();
-      long timeout = ((GMSMembershipManager) mgr).getSurpriseMemberTimeout();
-      long birthTime = startTime - timeout + gracePeriod;
-      MembershipManagerHelper.addSurpriseMember(sys, mbr, birthTime);
-      assertTrue("Member was not a surprise member", mgr.isSurpriseMember(mbr));
-
-      // if (birthTime < (System.currentTimeMillis() - timeout)) {
-      // return; // machine is too busy and we didn't get enough CPU to perform more assertions
-      // }
-
-      Awaitility.await("waiting for member to be removed")
-          .atMost((timeout / 3) + gracePeriod, TimeUnit.MILLISECONDS)
-          .until(() -> !mgr.isSurpriseMember(mbr));
-
-    } finally {
-      if (sys != null && sys.isConnected()) {
-        sys.disconnect();
-      }
-    }
-  }
-
-  /**
-   * vm1 stores its cache in this static variable in testAckSeverAllertThreshold
-   */
-  static Cache myCache;
-
-  /**
-   * Tests that a severe-level alert is generated if a member does not respond with an ack quickly
-   * enough. vm0 and vm1 create a region and set ack-severe-alert-threshold. vm1 has a cache
-   * listener in its region that sleeps when notified, forcing the operation to take longer than
-   * ack-wait-threshold + ack-severe-alert-threshold
-   */
-  @Test
-  public void testAckSevereAlertThreshold() throws Exception {
-    disconnectAllFromDS();
-    Host host = Host.getHost(0);
-    // VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    // in order to set a small ack-wait-threshold, we have to remove the
-    // system property established by the dunit harness
-    String oldAckWait = (String) System.getProperties()
-        .remove(DistributionConfig.GEMFIRE_PREFIX + ACK_WAIT_THRESHOLD);
-
-    try {
-      final Properties props = getDistributedSystemProperties();
-      props.setProperty(MCAST_PORT, "0");
-      props.setProperty(ACK_WAIT_THRESHOLD, "3");
-      props.setProperty(ACK_SEVERE_ALERT_THRESHOLD, "3");
-      props.setProperty(NAME, "putter");
-
-      getSystem(props);
-      Region rgn = (new RegionFactory()).setScope(Scope.DISTRIBUTED_ACK).setEarlyAck(false)
-          .setDataPolicy(DataPolicy.REPLICATE).create("testRegion");
-
-      vm1.invoke(new SerializableRunnable("Connect to distributed system") {
-        public void run() {
-          props.setProperty(NAME, "sleeper");
-          getSystem(props);
-          IgnoredException.addIgnoredException("elapsed while waiting for replies");
-          RegionFactory rf = new RegionFactory();
-          Region r = rf.setScope(Scope.DISTRIBUTED_ACK).setDataPolicy(DataPolicy.REPLICATE)
-              .setEarlyAck(false).addCacheListener(getSleepingListener(false)).create("testRegion");
-          myCache = r.getCache();
-          try {
-            createAlertListener();
-          } catch (Exception e) {
-            throw new RuntimeException("failed to create alert listener", e);
-          }
-        }
-      });
-
-      // now we have two caches set up. vm1 has a listener that will sleep
-      // and cause the severe-alert threshold to be crossed
-
-      rgn.put("bomb", "pow!"); // this will hang until vm1 responds
-
-      rgn.getCache().close();
-      basicGetSystem().disconnect();
-
-      vm1.invoke(new SerializableRunnable("disconnect from ds") {
-        public void run() {
-          if (!myCache.isClosed()) {
-            if (basicGetSystem().isConnected()) {
-              basicGetSystem().disconnect();
-            }
-            myCache = null;
-          }
-          if (basicGetSystem().isConnected()) {
-            basicGetSystem().disconnect();
-          }
-          synchronized (alertGuard) {
-            assertTrue(alertReceived);
-          }
-        }
-      });
-
-    } finally {
-      if (oldAckWait != null) {
-        System.setProperty(DistributionConfig.GEMFIRE_PREFIX + ACK_WAIT_THRESHOLD, oldAckWait);
-      }
-    }
-  }
-
-  static volatile boolean regionDestroyedInvoked;
-
-  static CacheListener getSleepingListener(final boolean playDead) {
-    regionDestroyedInvoked = false;
-
-    return new CacheListenerAdapter() {
-      @Override
-      public void afterCreate(EntryEvent event) {
-        try {
-          if (playDead) {
-            MembershipManagerHelper.beSickMember(getSystemStatic());
-            MembershipManagerHelper.playDead(getSystemStatic());
-          }
-          Thread.sleep(15000);
-        } catch (InterruptedException ie) {
-          fail("interrupted", ie);
-        }
-      }
-
-      @Override
-      public void afterRegionDestroy(RegionEvent event) {
-        LogWriter logger = myCache.getLogger();
-        logger.info("afterRegionDestroyed invoked in sleeping listener");
-        logger.info("<ExpectedException action=remove>service failure</ExpectedException>");
-        logger.info(
-            "<ExpectedException action=remove>org.apache.geode.ForcedDisconnectException</ExpectedException>");
-        regionDestroyedInvoked = true;
-      }
-    };
-  }
-
-  static AdminDistributedSystem adminSystem;
-  static Object alertGuard = new Object();
-  static boolean alertReceived;
-
-  static void createAlertListener() throws Exception {
-    DistributedSystemConfig config =
-        AdminDistributedSystemFactory.defineDistributedSystem(getSystemStatic(), null);
-    adminSystem = AdminDistributedSystemFactory.getDistributedSystem(config);
-    adminSystem.setAlertLevel(AlertLevel.SEVERE);
-    adminSystem.addAlertListener(new AlertListener() {
-      public void alert(Alert alert) {
-        try {
-          logger
-              .info("alert listener invoked for alert originating in " + alert.getConnectionName());
-          logger.info("  alert text = " + alert.getMessage());
-          logger.info("  systemMember = " + alert.getSystemMember());
-        } catch (Exception e) {
-          logger.fatal("exception trying to use alert object", e);
-        }
-        synchronized (alertGuard) {
-          alertReceived = true;
-        }
-      }
-    });
-    adminSystem.connect();
-    assertTrue(adminSystem.waitToBeConnected(5 * 1000));
-  }
-
-  /**
-   * Tests that a sick member is kicked out
-   */
-  @Test
-  public void testKickOutSickMember() throws Exception {
-    disconnectAllFromDS();
-    IgnoredException.addIgnoredException("10 seconds have elapsed while waiting");
-    Host host = Host.getHost(0);
-    // VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-
-    // in order to set a small ack-wait-threshold, we have to remove the
-    // system property established by the dunit harness
-    String oldAckWait = (String) System.getProperties()
-        .remove(DistributionConfig.GEMFIRE_PREFIX + ACK_WAIT_THRESHOLD);
-
-    try {
-      final Properties props = getDistributedSystemProperties();
-      props.setProperty(MCAST_PORT, "0"); // loner
-      props.setProperty(ACK_WAIT_THRESHOLD, "5");
-      props.setProperty(ACK_SEVERE_ALERT_THRESHOLD, "5");
-      props.setProperty(NAME, "putter");
-
-      getSystem(props);
-      Region rgn = (new RegionFactory()).setScope(Scope.DISTRIBUTED_ACK)
-          .setDataPolicy(DataPolicy.REPLICATE).create("testRegion");
-      basicGetSystem().getLogWriter().info(
-          "<ExpectedException action=add>sec have elapsed while waiting for replies</ExpectedException>");
-
-      vm1.invoke(new SerializableRunnable("Connect to distributed system") {
-        public void run() {
-          props.setProperty(NAME, "sleeper");
-          getSystem(props);
-          LogWriter log = basicGetSystem().getLogWriter();
-          log.info("<ExpectedException action=add>service failure</ExpectedException>");
-          log.info(
-              "<ExpectedException action=add>org.apache.geode.ForcedDisconnectException</ExpectedException>");
-          RegionFactory rf = new RegionFactory();
-          Region r = rf.setScope(Scope.DISTRIBUTED_ACK).setDataPolicy(DataPolicy.REPLICATE)
-              .addCacheListener(getSleepingListener(true)).create("testRegion");
-          myCache = r.getCache();
-        }
-      });
-
-      // now we have two caches set up, each having an alert listener. Vm1
-      // also has a cache listener that will turn off its ability to respond
-      // to "are you dead" messages and then sleep
-
-      rgn.put("bomb", "pow!");
-
-
-      rgn.getCache().close();
-      basicGetSystem().getLogWriter().info(
-          "<ExpectedException action=remove>sec have elapsed while waiting for replies</ExpectedException>");
-      basicGetSystem().disconnect();
-
-      vm1.invoke(new SerializableRunnable("wait for forced disconnect") {
-        public void run() {
-          // wait a while for the DS to finish disconnecting
-          WaitCriterion ev = new WaitCriterion() {
-            public boolean done() {
-              return !basicGetSystem().isConnected();
-            }
-
-            public String description() {
-              return null;
-            }
-          };
-          // if this fails it means the sick member wasn't kicked out and something is wrong
-          Wait.waitForCriterion(ev, 60 * 1000, 200, true);
-
-          ev = new WaitCriterion() {
-            public boolean done() {
-              return myCache.isClosed();
-            }
-
-            public String description() {
-              return null;
-            }
-          };
-          Wait.waitForCriterion(ev, 20 * 1000, 200, false);
-
-          if (!myCache.isClosed()) {
-            if (basicGetSystem().isConnected()) {
-              basicGetSystem().disconnect();
-            }
-            myCache = null;
-            throw new RuntimeException("Test Failed - vm1's cache is not closed");
-          }
-          if (basicGetSystem().isConnected()) {
-            basicGetSystem().disconnect();
-            throw new RuntimeException("Test Failed - vm1's system should have been disconnected");
-          }
-
-          WaitCriterion wc = new WaitCriterion() {
-            public boolean done() {
-              return regionDestroyedInvoked;
-            }
-
-            public String description() {
-              return "vm1's listener should have received afterRegionDestroyed notification";
-            }
-          };
-          Wait.waitForCriterion(wc, 30 * 1000, 1000, true);
-
-        }
-      });
-
-    } finally {
-      if (oldAckWait != null) {
-        System.setProperty(DistributionConfig.GEMFIRE_PREFIX + ACK_WAIT_THRESHOLD, oldAckWait);
-      }
-    }
-  }
-
-  /**
-   * test use of a bad bind-address for bug #32565
-   */
-  @Test
-  public void testBadBindAddress() throws Exception {
-    disconnectAllFromDS();
-
-    final Properties props = getDistributedSystemProperties();
-    props.setProperty(MCAST_PORT, "0"); // loner
-    // use a valid address that's not proper for this machine
-    props.setProperty(BIND_ADDRESS, "www.yahoo.com");
-    props.setProperty(ACK_WAIT_THRESHOLD, "5");
-    props.setProperty(ACK_SEVERE_ALERT_THRESHOLD, "5");
-    try {
-      getSystem(props);
-    } catch (IllegalArgumentException e) {
-      org.apache.geode.test.dunit.LogWriterUtils.getLogWriter()
-          .info("caught expected exception (1)", e);
-    }
-    // use an invalid address
-    props.setProperty(BIND_ADDRESS, "bruce.schuchardt");
-    try {
-      getSystem(props);
-    } catch (IllegalArgumentException e) {
-      org.apache.geode.test.dunit.LogWriterUtils.getLogWriter()
-          .info("caught expected exception (2_", e);
-    }
-    // use a valid bind address
-    props.setProperty(BIND_ADDRESS, InetAddress.getLocalHost().getCanonicalHostName());
-    getSystem().disconnect();
-  }
-
-  /**
-   * install a new view and show that waitForViewInstallation works as expected
-   */
-  @Test
-  public void testWaitForViewInstallation() {
-    getSystem(new Properties());
-
-    MembershipManager mgr = basicGetSystem().getDM().getMembershipManager();
-
-    final NetView v = mgr.getView();
-
-    final boolean[] passed = new boolean[1];
-    Thread t = new Thread("wait for view installation") {
-      public void run() {
-        try {
-          ((ClusterDistributionManager) basicGetSystem().getDM())
-              .waitForViewInstallation(v.getViewId() + 1);
-          synchronized (passed) {
-            passed[0] = true;
-          }
-        } catch (InterruptedException e) {
-          // failed
-        }
-      }
-    };
-    t.setDaemon(true);
-    t.start();
-
-    Wait.pause(2000);
-
-    NetView newView = new NetView(v, v.getViewId() + 1);
-    ((Manager) mgr).installView(newView);
-
-    Wait.pause(2000);
-
-    synchronized (passed) {
-      Assert.assertTrue(passed[0]);
-    }
-  }
-}

-- 
To stop receiving notification emails like this one, please contact
"commits@geode.apache.org" <co...@geode.apache.org>.

[geode] 01/03: GEODE-3965: rename DistributionManager classes

Posted by kl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

klund pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 75c04138a42bed3edda1f6391b71c45280fe3ef7
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Thu Jan 4 11:58:16 2018 -0800

    GEODE-3965: rename DistributionManager classes
    
    * DistributionManager -> ClusterDistributionManager
    * DM -> DistributionManager
    * LonerDistributionManager remains LonerDistributionManager
---
 .../geode/modules/util/BootstrappingFunction.java  |    5 +-
 .../apache/geode/admin/GemFireMemberStatus.java    |    4 +-
 .../admin/internal/AbstractHealthEvaluator.java    |    4 +-
 .../admin/internal/AdminDistributedSystemImpl.java |   35 +-
 .../geode/admin/internal/CacheHealthEvaluator.java |    8 +-
 .../geode/admin/internal/CacheServerImpl.java      |    7 +-
 .../internal/DistributedSystemHealthEvaluator.java |   14 +-
 .../admin/internal/DistributionLocatorImpl.java    |    5 +-
 .../admin/internal/GemFireHealthEvaluator.java     |    4 +-
 .../admin/internal/MemberHealthEvaluator.java      |    4 +-
 .../internal/SystemMemberCacheEventProcessor.java  |    4 +-
 .../apache/geode/admin/jmx/internal/AgentImpl.java |    6 +-
 .../geode/admin/jmx/internal/AgentLauncher.java    |    4 +-
 .../client/internal/CacheServerLoadMessage.java    |    4 +-
 .../geode/distributed/DistributedSystem.java       |    6 +-
 .../internal/ClusterConfigurationService.java      |    2 +-
 ...anager.java => ClusterDistributionManager.java} |  114 +-
 .../internal/CollectingReplyProcessor.java         |    2 +-
 .../org/apache/geode/distributed/internal/DM.java  |  489 ---
 .../distributed/internal/DirectReplyProcessor.java |   23 +-
 .../distributed/internal/DistributionAdvisee.java  |    4 +-
 .../distributed/internal/DistributionAdvisor.java  |    6 +-
 .../distributed/internal/DistributionManager.java  | 4539 ++------------------
 .../distributed/internal/DistributionMessage.java  |   18 +-
 .../internal/DistributionMessageObserver.java      |    6 +-
 .../distributed/internal/HealthMonitorImpl.java    |    4 +-
 .../internal/HighPriorityAckedMessage.java         |    6 +-
 .../internal/HighPriorityDistributionMessage.java  |    2 +-
 .../internal/InternalDistributedSystem.java        |   19 +-
 .../distributed/internal/InternalLocator.java      |    6 +-
 .../internal/LonerDistributionManager.java         |    4 +-
 .../distributed/internal/MembershipListener.java   |    2 +-
 .../internal/PooledDistributionMessage.java        |    2 +-
 .../geode/distributed/internal/ProductUseLog.java  |    2 +-
 .../internal/ReliableReplyProcessor21.java         |    4 +-
 .../geode/distributed/internal/ReplyMessage.java   |    6 +-
 .../distributed/internal/ReplyProcessor21.java     |   39 +-
 .../geode/distributed/internal/ReplySender.java    |    4 +-
 .../distributed/internal/SerialAckedMessage.java   |    6 +-
 .../internal/SerialDistributionMessage.java        |    2 +-
 .../geode/distributed/internal/ServerLocator.java  |    2 +-
 .../distributed/internal/ShutdownMessage.java      |    2 +-
 .../geode/distributed/internal/StartupMessage.java |    9 +-
 .../internal/StartupMessageReplyProcessor.java     |    6 +-
 .../distributed/internal/StartupOperation.java     |    4 +-
 .../internal/StartupResponseMessage.java           |    4 +-
 .../StartupResponseWithVersionMessage.java         |    5 +-
 .../internal/WaitForViewInstallation.java          |    6 +-
 .../distributed/internal/direct/DirectChannel.java |   10 +-
 .../internal/direct/DirectChannelListener.java     |    4 +-
 .../distributed/internal/locks/DLockGrantor.java   |    8 +-
 .../internal/locks/DLockQueryProcessor.java        |   16 +-
 .../locks/DLockRecoverGrantorProcessor.java        |   23 +-
 .../internal/locks/DLockReleaseProcessor.java      |   16 +-
 .../internal/locks/DLockRequestProcessor.java      |   33 +-
 .../distributed/internal/locks/DLockService.java   |   14 +-
 .../distributed/internal/locks/DLockToken.java     |    8 +-
 .../internal/locks/DeposeGrantorProcessor.java     |   14 +-
 .../internal/locks/DistributedMemberLock.java      |    4 +-
 .../internal/locks/ElderInitProcessor.java         |   15 +-
 .../distributed/internal/locks/ElderState.java     |    8 +-
 .../internal/locks/GrantorRequestProcessor.java    |   20 +-
 .../distributed/internal/locks/LockGrantorId.java  |    8 +-
 .../locks/NonGrantorDestroyedProcessor.java        |   20 +-
 .../membership/DistributedMembershipListener.java  |    1 -
 .../membership/InternalDistributedMember.java      |   39 +-
 .../internal/membership/InternalRole.java          |    6 +-
 .../internal/membership/MemberAttributes.java      |    4 +-
 .../distributed/internal/membership/NetView.java   |   24 +-
 .../gms/locator/FindCoordinatorRequest.java        |    5 +-
 .../gms/locator/FindCoordinatorResponse.java       |    4 +-
 .../membership/gms/locator/GMSLocator.java         |    4 +-
 .../membership/gms/membership/GMSJoinLeave.java    |    9 +-
 .../gms/messages/FinalCheckPassedMessage.java      |    4 +-
 .../membership/gms/messages/HeartbeatMessage.java  |    4 +-
 .../gms/messages/HeartbeatRequestMessage.java      |    4 +-
 .../gms/messages/InstallViewMessage.java           |    8 +-
 .../gms/messages/JoinRequestMessage.java           |    4 +-
 .../gms/messages/JoinResponseMessage.java          |    8 +-
 .../gms/messages/LeaveRequestMessage.java          |    4 +-
 .../gms/messages/NetworkPartitionMessage.java      |    4 +-
 .../gms/messages/RemoveMemberMessage.java          |    5 +-
 .../gms/messages/SuspectMembersMessage.java        |    4 +-
 .../membership/gms/messages/ViewAckMessage.java    |    4 +-
 .../membership/gms/messenger/JGroupsMessenger.java |   12 +-
 .../membership/gms/mgr/GMSMembershipManager.java   |   12 +-
 .../membership/gms/mgr/LocalViewMessage.java       |    6 +-
 .../internal/streaming/StreamingOperation.java     |   17 +-
 .../geode/internal/InternalDataSerializer.java     |    4 +-
 .../geode/internal/InternalInstantiator.java       |    6 +-
 .../internal/admin/ClientMembershipMessage.java    |    6 +-
 .../geode/internal/admin/GfManagerAgent.java       |    4 +-
 .../geode/internal/admin/StatAlertsManager.java    |   10 +-
 .../admin/remote/AddHealthListenerRequest.java     |    2 +-
 .../admin/remote/AddHealthListenerResponse.java    |    4 +-
 .../admin/remote/AddStatListenerRequest.java       |    2 +-
 .../admin/remote/AddStatListenerResponse.java      |    4 +-
 .../remote/AdminConsoleDisconnectMessage.java      |    4 +-
 .../internal/admin/remote/AdminConsoleMessage.java |    9 +-
 .../admin/remote/AdminMultipleReplyProcessor.java  |    4 +-
 .../geode/internal/admin/remote/AdminRequest.java  |    8 +-
 .../geode/internal/admin/remote/AdminResponse.java |    2 +-
 .../geode/internal/admin/remote/AdminWaiters.java  |    6 +-
 .../admin/remote/AlertLevelChangeMessage.java      |    4 +-
 .../admin/remote/AlertListenerMessage.java         |    2 +-
 .../admin/remote/AlertsNotificationMessage.java    |    4 +-
 .../admin/remote/AppCacheSnapshotMessage.java      |    2 +-
 .../internal/admin/remote/BridgeServerRequest.java |    4 +-
 .../admin/remote/BridgeServerResponse.java         |    4 +-
 .../internal/admin/remote/CacheConfigRequest.java  |    2 +-
 .../internal/admin/remote/CacheConfigResponse.java |    6 +-
 .../internal/admin/remote/CacheInfoRequest.java    |    2 +-
 .../internal/admin/remote/CacheInfoResponse.java   |    5 +-
 .../admin/remote/CancelStatListenerRequest.java    |    2 +-
 .../admin/remote/CancelStatListenerResponse.java   |    4 +-
 .../internal/admin/remote/CancellationMessage.java |    2 +-
 .../admin/remote/ChangeRefreshIntervalMessage.java |    4 +-
 .../internal/admin/remote/CliLegacyMessage.java    |    4 +-
 .../internal/admin/remote/CompactRequest.java      |   12 +-
 .../internal/admin/remote/DestroyEntryMessage.java |    4 +-
 .../admin/remote/DestroyRegionMessage.java         |    4 +-
 .../admin/remote/DurableClientInfoRequest.java     |    2 +-
 .../admin/remote/DurableClientInfoResponse.java    |    6 +-
 .../admin/remote/FetchDistLockInfoRequest.java     |    2 +-
 .../admin/remote/FetchDistLockInfoResponse.java    |    3 +-
 .../admin/remote/FetchHealthDiagnosisRequest.java  |    2 +-
 .../admin/remote/FetchHealthDiagnosisResponse.java |    4 +-
 .../internal/admin/remote/FetchHostRequest.java    |    2 +-
 .../internal/admin/remote/FetchHostResponse.java   |    5 +-
 .../remote/FetchResourceAttributesRequest.java     |    2 +-
 .../remote/FetchResourceAttributesResponse.java    |    6 +-
 .../internal/admin/remote/FetchStatsRequest.java   |    2 +-
 .../internal/admin/remote/FetchStatsResponse.java  |    6 +-
 .../internal/admin/remote/FetchSysCfgRequest.java  |    2 +-
 .../internal/admin/remote/FetchSysCfgResponse.java |    3 +-
 .../admin/remote/FlushAppCacheSnapshotMessage.java |    2 +-
 .../admin/remote/HealthListenerMessage.java        |    2 +-
 .../internal/admin/remote/LicenseInfoRequest.java  |    2 +-
 .../internal/admin/remote/LicenseInfoResponse.java |    4 +-
 .../admin/remote/MissingPersistentIDsRequest.java  |   10 +-
 .../admin/remote/MissingPersistentIDsResponse.java |    5 +-
 .../admin/remote/ObjectDetailsRequest.java         |    2 +-
 .../admin/remote/ObjectDetailsResponse.java        |    3 +-
 .../internal/admin/remote/ObjectNamesRequest.java  |    2 +-
 .../internal/admin/remote/ObjectNamesResponse.java |    3 +-
 .../remote/PrepareRevokePersistentIDRequest.java   |   13 +-
 .../admin/remote/RefreshMemberSnapshotRequest.java |    2 +-
 .../remote/RefreshMemberSnapshotResponse.java      |    5 +-
 .../admin/remote/RegionAttributesRequest.java      |    2 +-
 .../admin/remote/RegionAttributesResponse.java     |    4 +-
 .../geode/internal/admin/remote/RegionRequest.java |    2 +-
 .../internal/admin/remote/RegionResponse.java      |    2 +-
 .../internal/admin/remote/RegionSizeRequest.java   |    2 +-
 .../internal/admin/remote/RegionSizeResponse.java  |    3 +-
 .../admin/remote/RegionStatisticsRequest.java      |    2 +-
 .../admin/remote/RegionStatisticsResponse.java     |    4 +-
 .../admin/remote/RegionSubRegionSizeRequest.java   |    4 +-
 .../admin/remote/RegionSubRegionsSizeResponse.java |    7 +-
 .../admin/remote/RemoteGfManagerAgent.java         |   40 +-
 .../admin/remote/RemoveHealthListenerRequest.java  |    2 +-
 .../admin/remote/RemoveHealthListenerResponse.java |    4 +-
 .../admin/remote/ResetHealthStatusRequest.java     |    2 +-
 .../admin/remote/ResetHealthStatusResponse.java    |    4 +-
 .../admin/remote/RevokePersistentIDRequest.java    |    8 +-
 .../internal/admin/remote/RootRegionRequest.java   |    2 +-
 .../internal/admin/remote/RootRegionResponse.java  |    5 +-
 .../remote/ShutdownAllGatewayHubsRequest.java      |    6 +-
 .../internal/admin/remote/ShutdownAllRequest.java  |   13 +-
 .../admin/remote/SnapshotResultMessage.java        |    2 +-
 .../remote/StatAlertsManagerAssignMessage.java     |    6 +-
 .../internal/admin/remote/StatListenerMessage.java |    2 +-
 .../internal/admin/remote/StoreSysCfgRequest.java  |    2 +-
 .../internal/admin/remote/StoreSysCfgResponse.java |    3 +-
 .../internal/admin/remote/SubRegionRequest.java    |    2 +-
 .../internal/admin/remote/SubRegionResponse.java   |    3 +-
 .../internal/admin/remote/TailLogRequest.java      |    2 +-
 .../internal/admin/remote/TailLogResponse.java     |    5 +-
 .../admin/remote/UpdateAlertDefinitionMessage.java |    4 +-
 .../internal/admin/remote/VersionInfoRequest.java  |    2 +-
 .../internal/admin/remote/VersionInfoResponse.java |    3 +-
 .../geode/internal/cache/AbstractCacheServer.java  |    4 +-
 .../geode/internal/cache/AbstractRegion.java       |    4 +-
 .../internal/cache/AbstractUpdateOperation.java    |   10 +-
 .../cache/AddCacheServerProfileMessage.java        |    4 +-
 .../apache/geode/internal/cache/BucketAdvisor.java |    4 +-
 .../internal/cache/CacheDistributionAdvisor.java   |    6 +-
 .../geode/internal/cache/CacheServerAdvisor.java   |    6 +-
 .../geode/internal/cache/CacheServerImpl.java      |    5 +-
 .../geode/internal/cache/CloseCacheMessage.java    |    4 +-
 .../geode/internal/cache/ControllerAdvisor.java    |    7 +-
 .../internal/cache/CreateRegionProcessor.java      |    5 +-
 .../geode/internal/cache/DestroyOperation.java     |    6 +-
 .../cache/DestroyPartitionedRegionMessage.java     |    4 +-
 .../internal/cache/DestroyRegionOperation.java     |   12 +-
 .../geode/internal/cache/DirectReplyMessage.java   |    7 +-
 .../geode/internal/cache/DistPeerTXStateStub.java  |   10 +-
 .../internal/cache/DistTXAdjunctCommitMessage.java |    4 +-
 .../geode/internal/cache/DistTXCommitMessage.java  |    9 +-
 .../internal/cache/DistTXCoordinatorInterface.java |    9 +-
 .../internal/cache/DistTXPrecommitMessage.java     |    9 +-
 .../internal/cache/DistTXRollbackMessage.java      |   10 +-
 .../internal/cache/DistTXStateOnCoordinator.java   |    8 +-
 .../cache/DistTXStateProxyImplOnCoordinator.java   |   14 +-
 .../internal/cache/DistributedCacheOperation.java  |   21 +-
 .../internal/cache/DistributedClearOperation.java  |    8 +-
 .../geode/internal/cache/DistributedRegion.java    |   11 +-
 .../DistributedRegionFunctionStreamingMessage.java |   26 +-
 .../cache/DistributedTombstoneOperation.java       |    8 +-
 .../apache/geode/internal/cache/EntriesMap.java    |    5 +-
 .../apache/geode/internal/cache/FilterProfile.java |    6 +-
 .../internal/cache/FindDurableQueueProcessor.java  |   10 +-
 .../geode/internal/cache/FindRemoteTXMessage.java  |    9 +-
 .../internal/cache/FindVersionTagOperation.java    |   10 +-
 .../FunctionStreamingOrderedReplyMessage.java      |    7 +-
 .../cache/FunctionStreamingReplyMessage.java       |    5 +-
 .../geode/internal/cache/GemFireCacheImpl.java     |   26 +-
 .../internal/cache/InitialImageFlowControl.java    |   16 +-
 .../internal/cache/InitialImageOperation.java      |   71 +-
 .../apache/geode/internal/cache/InternalCache.java |    4 +-
 .../geode/internal/cache/InternalRegion.java       |    4 +-
 .../geode/internal/cache/InvalidateOperation.java  |    6 +-
 .../cache/InvalidatePartitionedRegionMessage.java  |    4 +-
 .../internal/cache/InvalidateRegionOperation.java  |    4 +-
 .../internal/cache/JtaAfterCompletionMessage.java  |    9 +-
 .../internal/cache/JtaBeforeCompletionMessage.java |    4 +-
 .../cache/LatestLastAccessTimeMessage.java         |    4 +-
 .../cache/LatestLastAccessTimeOperation.java       |    4 +-
 .../cache/LatestLastAccessTimeReplyProcessor.java  |    5 +-
 .../apache/geode/internal/cache/LocalRegion.java   |    4 +-
 .../cache/MemberFunctionStreamingMessage.java      |   23 +-
 .../geode/internal/cache/NonLocalRegionEntry.java  |    4 +-
 .../org/apache/geode/internal/cache/Oplog.java     |    4 +-
 .../internal/cache/PRHARedundancyProvider.java     |    6 +-
 .../geode/internal/cache/PartitionedRegion.java    |   20 +-
 .../internal/cache/PartitionedRegionDataStore.java |    6 +-
 .../internal/cache/PartitionedRegionHelper.java    |    4 +-
 .../geode/internal/cache/ProxyBucketRegion.java    |    4 +-
 .../geode/internal/cache/ProxyRegionMap.java       |    4 +-
 .../apache/geode/internal/cache/RegionEntry.java   |    4 +-
 .../internal/cache/ReleaseClearLockMessage.java    |    8 +-
 .../cache/RemoteContainsKeyValueMessage.java       |    6 +-
 .../geode/internal/cache/RemoteDestroyMessage.java |   15 +-
 .../internal/cache/RemoteFetchEntryMessage.java    |    8 +-
 .../internal/cache/RemoteFetchVersionMessage.java  |   10 +-
 .../geode/internal/cache/RemoteGetMessage.java     |   10 +-
 .../internal/cache/RemoteInvalidateMessage.java    |   22 +-
 .../internal/cache/RemoteOperationMessage.java     |   21 +-
 .../RemoteOperationMessageWithDirectReply.java     |    6 +-
 .../geode/internal/cache/RemotePutAllMessage.java  |   12 +-
 .../geode/internal/cache/RemotePutMessage.java     |   18 +-
 .../internal/cache/RemoteRegionOperation.java      |    6 +-
 .../internal/cache/RemoteRemoveAllMessage.java     |   12 +-
 .../cache/SearchLoadAndWriteProcessor.java         |   53 +-
 .../geode/internal/cache/ServerPingMessage.java    |    6 +-
 .../geode/internal/cache/StateFlushOperation.java  |   24 +-
 .../geode/internal/cache/TXCommitMessage.java      |   37 +-
 .../geode/internal/cache/TXFarSideCMTracker.java   |   11 +-
 .../apache/geode/internal/cache/TXManagerImpl.java |   10 +-
 .../org/apache/geode/internal/cache/TXMessage.java |    8 +-
 .../internal/cache/TXRemoteCommitMessage.java      |    9 +-
 .../internal/cache/TXRemoteRollbackMessage.java    |    4 +-
 .../internal/cache/UpdateAttributesProcessor.java  |   13 +-
 .../cache/UpdateEntryVersionOperation.java         |    6 +-
 .../geode/internal/cache/UpdateOperation.java      |   12 +-
 .../geode/internal/cache/ValidatingDiskRegion.java |    4 +-
 .../cache/backup/BackupDataStoreHelper.java        |   12 +-
 .../geode/internal/cache/backup/BackupManager.java |    4 +-
 .../internal/cache/backup/BackupOperation.java     |    8 +-
 .../cache/backup/BackupReplyProcessor.java         |    4 +-
 .../geode/internal/cache/backup/BackupUtil.java    |    6 +-
 .../internal/cache/backup/FinishBackupFactory.java |    6 +-
 .../cache/backup/FinishBackupOperation.java        |    8 +-
 .../internal/cache/backup/FinishBackupRequest.java |    4 +-
 .../internal/cache/backup/FlushToDiskFactory.java  |    5 +-
 .../cache/backup/FlushToDiskOperation.java         |    9 +-
 .../cache/backup/FlushToDiskProcessor.java         |    4 +-
 .../internal/cache/backup/FlushToDiskRequest.java  |    4 +-
 .../cache/backup/PrepareBackupFactory.java         |    6 +-
 .../cache/backup/PrepareBackupOperation.java       |    7 +-
 .../cache/backup/PrepareBackupRequest.java         |    4 +-
 .../internal/cache/control/ResourceAdvisor.java    |   12 +-
 .../entries/AbstractOplogDiskRegionEntry.java      |    6 +-
 .../cache/entries/AbstractRegionEntry.java         |    6 +-
 .../geode/internal/cache/entries/DiskEntry.java    |    5 +-
 .../internal/cache/execute/AbstractExecution.java  |   18 +-
 .../DistributedRegionFunctionResultSender.java     |   12 +-
 .../cache/execute/MemberFunctionExecutor.java      |    4 +-
 .../cache/execute/MemberFunctionResultSender.java  |   11 +-
 .../cache/execute/MultiRegionFunctionExecutor.java |    4 +-
 .../PartitionedRegionFunctionResultSender.java     |   16 +-
 .../geode/internal/cache/ha/HARegionQueue.java     |    4 +-
 .../internal/cache/ha/QueueRemovalMessage.java     |    5 +-
 .../cache/locks/TXLessorDepartureHandler.java      |    4 +-
 .../locks/TXLockUpdateParticipantsMessage.java     |    4 +-
 .../cache/locks/TXOriginatorRecoveryProcessor.java |   12 +-
 .../locks/TXRecoverGrantorMessageProcessor.java    |    8 +-
 .../AllBucketProfilesUpdateMessage.java            |    8 +-
 .../partitioned/BecomePrimaryBucketMessage.java    |   14 +-
 .../cache/partitioned/BucketBackupMessage.java     |    6 +-
 .../partitioned/BucketProfileUpdateMessage.java    |   10 +-
 .../cache/partitioned/BucketSizeMessage.java       |   12 +-
 .../cache/partitioned/ContainsKeyValueMessage.java |    6 +-
 .../cache/partitioned/CreateBucketMessage.java     |   12 +-
 .../partitioned/DeposePrimaryBucketMessage.java    |   12 +-
 .../internal/cache/partitioned/DestroyMessage.java |   12 +-
 .../DestroyRegionOnDataStoreMessage.java           |   14 +-
 .../cache/partitioned/DumpAllPRConfigMessage.java  |    4 +-
 .../internal/cache/partitioned/DumpB2NRegion.java  |   12 +-
 .../cache/partitioned/DumpBucketsMessage.java      |    4 +-
 .../partitioned/EndBucketCreationMessage.java      |    8 +-
 .../cache/partitioned/FetchBulkEntriesMessage.java |   12 +-
 .../cache/partitioned/FetchEntriesMessage.java     |   13 +-
 .../cache/partitioned/FetchEntryMessage.java       |    8 +-
 .../cache/partitioned/FetchKeysMessage.java        |   14 +-
 .../partitioned/FetchPartitionDetailsMessage.java  |   12 +-
 .../internal/cache/partitioned/FlushMessage.java   |    4 +-
 .../internal/cache/partitioned/GetMessage.java     |   14 +-
 .../cache/partitioned/IdentityRequestMessage.java  |   11 +-
 .../cache/partitioned/IdentityUpdateMessage.java   |    6 +-
 .../cache/partitioned/IndexCreationMsg.java        |   20 +-
 .../cache/partitioned/InterestEventMessage.java    |   14 +-
 .../cache/partitioned/InvalidateMessage.java       |   10 +-
 .../partitioned/ManageBackupBucketMessage.java     |   16 +-
 .../cache/partitioned/ManageBucketMessage.java     |   16 +-
 .../cache/partitioned/MoveBucketMessage.java       |   12 +-
 .../cache/partitioned/PRSanityCheckMessage.java    |   10 +-
 .../cache/partitioned/PRTombstoneMessage.java      |    6 +-
 .../partitioned/PRUpdateEntryVersionMessage.java   |    4 +-
 .../cache/partitioned/PartitionMessage.java        |   22 +-
 .../PartitionMessageWithDirectReply.java           |    3 +-
 .../PartitionedRegionFunctionStreamingMessage.java |   16 +-
 .../partitioned/PartitionedRegionRebalanceOp.java  |    4 +-
 .../cache/partitioned/PrimaryRequestMessage.java   |    8 +-
 .../cache/partitioned/PutAllPRMessage.java         |   12 +-
 .../internal/cache/partitioned/PutMessage.java     |   14 +-
 .../internal/cache/partitioned/QueryMessage.java   |    8 +-
 .../cache/partitioned/RemoteFetchKeysMessage.java  |   14 +-
 .../cache/partitioned/RemoteSizeMessage.java       |    9 +-
 .../cache/partitioned/RemoveAllPRMessage.java      |   12 +-
 .../cache/partitioned/RemoveBucketMessage.java     |   12 +-
 .../cache/partitioned/RemoveIndexesMessage.java    |   17 +-
 .../internal/cache/partitioned/SizeMessage.java    |   10 +-
 .../partitioned/StreamingPartitionOperation.java   |   10 +-
 .../cache/persistence/MembershipFlushRequest.java  |    9 +-
 .../cache/persistence/MembershipViewRequest.java   |   27 +-
 .../cache/persistence/PersistenceAdvisorImpl.java  |    4 +-
 .../cache/persistence/PersistentMemberManager.java |    8 +-
 .../persistence/PersistentStateQueryMessage.java   |   14 +-
 .../PrepareNewPersistentMemberMessage.java         |   10 +-
 .../persistence/RemovePersistentMemberMessage.java |   11 +-
 .../internal/cache/snapshot/FlowController.java    |   27 +-
 .../internal/cache/tier/sockets/AcceptorImpl.java  |    4 +-
 .../cache/tier/sockets/CacheClientNotifier.java    |    7 +-
 .../cache/tier/sockets/CacheClientProxy.java       |    4 +-
 .../cache/tier/sockets/CacheClientUpdater.java     |    7 +-
 .../tier/sockets/ClientBlacklistProcessor.java     |   12 +-
 .../internal/cache/tier/sockets/HandShake.java     |    5 +-
 .../sockets/RemoveClientFromBlacklistMessage.java  |    5 +-
 .../sockets/ServerInterestRegistrationMessage.java |    6 +-
 .../cache/tier/sockets/VersionedObjectList.java    |    5 +-
 .../tier/sockets/command/ExecuteFunction66.java    |   11 +-
 .../tier/sockets/command/TXFailoverCommand.java    |    4 +-
 .../geode/internal/cache/tx/ClientTXStateStub.java |    4 +-
 .../internal/cache/tx/DistClientTXStateStub.java   |    9 +-
 .../internal/cache/tx/DistributedTXRegionStub.java |   19 +-
 .../cache/versions/RegionVersionVector.java        |    7 +-
 .../internal/cache/versions/VMVersionTag.java      |    4 +-
 .../geode/internal/cache/versions/VersionTag.java  |    4 +-
 .../internal/cache/wan/AbstractGatewaySender.java  |    4 +-
 .../internal/cache/wan/GatewaySenderAdvisor.java   |    7 +-
 ...aySenderQueueEntrySynchronizationOperation.java |    8 +-
 .../wan/parallel/ParallelGatewaySenderQueue.java   |    5 +-
 .../parallel/ParallelQueueBatchRemovalMessage.java |    8 +-
 .../wan/parallel/ParallelQueueRemovalMessage.java  |    5 +-
 ...tilParallelGatewaySenderFlushedCoordinator.java |    3 +-
 .../cache/wan/serial/BatchDestroyOperation.java    |    6 +-
 .../internal/cache/xmlcache/CacheCreation.java     |    4 +-
 .../xmlcache/ParallelAsyncEventQueueCreation.java  |    4 +-
 .../xmlcache/ParallelGatewaySenderCreation.java    |    4 +-
 .../xmlcache/SerialAsyncEventQueueCreation.java    |    4 +-
 .../xmlcache/SerialGatewaySenderCreation.java      |    4 +-
 .../geode/internal/jta/GlobalTransaction.java      |    4 +-
 .../internal/logging/log4j/AlertAppender.java      |    4 +-
 .../org/apache/geode/internal/tcp/Connection.java  |   12 +-
 .../apache/geode/internal/tcp/ConnectionTable.java |    6 +-
 .../org/apache/geode/internal/tcp/TCPConduit.java  |   10 +-
 .../management/internal/JmxManagerAdvisee.java     |    4 +-
 .../management/internal/JmxManagerAdvisor.java     |   11 +-
 .../geode/management/internal/ManagementAgent.java |    6 +-
 .../management/internal/ManagerStartupMessage.java |    7 +-
 .../geode/management/internal/MemberMessenger.java |    8 +-
 .../internal/beans/DistributedSystemBridge.java    |   10 +-
 .../internal/beans/ManagementAdapter.java          |    4 +-
 .../internal/beans/MemberMBeanBridge.java          |    9 +-
 .../cli/commands/BackupDiskStoreCommand.java       |    4 +-
 .../callbacks/ConfigurationChangeListener.java     |    4 -
 .../internal/messages/CompactRequest.java          |   12 +-
 .../geode/pdx/internal/CheckTypeRegistryState.java |    7 +-
 .../org/apache/geode/admin/AdminTestHelper.java    |   12 +-
 .../geode/cache/ConnectionPoolDUnitTest.java       |    6 +-
 .../geode/cache30/ClientServerCCEDUnitTest.java    |    4 +-
 .../cache30/DistributedAckRegionCCEDUnitTest.java  |    7 +-
 .../geode/cache30/RegionReliabilityTestCase.java   |    4 +-
 .../org/apache/geode/cache30/SlowRecDUnitTest.java |   14 +-
 .../DistributedLockServiceDUnitTest.java           |    7 +-
 .../distributed/DistributedMemberDUnitTest.java    |   15 +-
 .../distributed/DistributedSystemDUnitTest.java    |   17 +-
 .../geode/distributed/HostedLocatorsDUnitTest.java |   28 +-
 .../apache/geode/distributed/LocatorDUnitTest.java |   10 +-
 .../apache/geode/distributed/RoleDUnitTest.java    |    8 +-
 .../geode/distributed/SystemAdminDUnitTest.java    |    4 +-
 .../ConsoleDistributionManagerDUnitTest.java       |   14 +-
 .../geode/distributed/internal/DateMessage.java    |    2 +-
 .../internal/DistributionAdvisorDUnitTest.java     |    2 +-
 .../internal/DistributionManagerDUnitTest.java     |   16 +-
 .../internal/DistributionManagerTest.java          |    2 +-
 .../InternalDistributedSystemJUnitTest.java        |    2 +-
 .../org/apache/geode/distributed/internal/LDM.java |    4 +-
 .../distributed/internal/ProduceDateMessages.java  |    2 +-
 .../internal/membership/MembershipJUnitTest.java   |   18 +-
 .../internal/membership/NetViewJUnitTest.java      |   24 +-
 .../membership/gms/MembershipManagerHelper.java    |    5 +-
 .../gms/fd/GMSHealthMonitorJUnitTest.java          |    5 +-
 .../gms/locator/GMSLocatorRecoveryJUnitTest.java   |    4 +-
 .../gms/membership/GMSJoinLeaveJUnitTest.java      |    7 +-
 .../gms/membership/GMSJoinLeaveTestHelper.java     |    4 +-
 .../gms/membership/StatRecorderJUnitTest.java      |    6 +-
 .../gms/messenger/GMSEncryptJUnitTest.java         |    4 +-
 .../gms/messenger/GMSQuorumCheckerJUnitTest.java   |   27 +-
 .../gms/messenger/JGroupsMessengerJUnitTest.java   |   12 +-
 .../gms/mgr/GMSMembershipManagerJUnitTest.java     |    6 +-
 .../geode/internal/cache/Bug41091DUnitTest.java    |    5 +-
 .../geode/internal/cache/Bug41733DUnitTest.java    |    8 +-
 .../internal/cache/CacheOperationMessageTest.java  |    4 +-
 .../internal/cache/DistributedCacheTestCase.java   |   10 +-
 ...tributedRegionFunctionStreamingMessageTest.java |    4 +-
 .../geode/internal/cache/GIIDeltaDUnitTest.java    |   10 +-
 .../internal/cache/GIIFlowControlDUnitTest.java    |    6 +-
 .../cache/InterruptClientServerDUnitTest.java      |    8 +-
 .../geode/internal/cache/InterruptsDUnitTest.java  |    8 +-
 ...FOEvictionAlgoMemoryEnabledRegionJUnitTest.java |    5 +-
 .../cache/NetSearchMessagingDUnitTest.java         |    8 +-
 .../cache/PartitionedRegionQueryDUnitTest.java     |    5 +-
 .../cache/RemoteFetchEntryMessageTest.java         |    4 +-
 .../internal/cache/RemoteOperationMessageTest.java |    6 +-
 .../cache/SearchLoadAndWriteProcessorTest.java     |    4 +-
 .../internal/cache/SystemFailureDUnitTest.java     |    4 +-
 .../geode/internal/cache/TXManagerImplTest.java    |    7 +-
 .../cache/backup/BackupDistributedTest.java        |   10 +-
 .../backup/BackupPrepareAndFinishMsgDUnitTest.java |    6 +-
 .../cache/backup/BackupReplyProcessorTest.java     |    6 +-
 .../cache/backup/FinishBackupFactoryTest.java      |    6 +-
 .../cache/backup/FinishBackupOperationTest.java    |    6 +-
 .../cache/backup/FinishBackupRequestTest.java      |    6 +-
 .../cache/backup/FlushToDiskFactoryTest.java       |    6 +-
 .../cache/backup/FlushToDiskOperationTest.java     |    6 +-
 .../cache/backup/FlushToDiskRequestTest.java       |    6 +-
 .../backup/IncrementalBackupDistributedTest.java   |    8 +-
 .../cache/backup/PrepareBackupFactoryTest.java     |    6 +-
 .../cache/backup/PrepareBackupOperationTest.java   |    6 +-
 .../cache/backup/PrepareBackupRequestTest.java     |    6 +-
 .../internal/cache/eviction/LRUTestEntry.java      |    5 +-
 ...istributedRegionFunctionExecutionDUnitTest.java |    4 +-
 .../execute/MemberFunctionExecutionDUnitTest.java  |    4 +-
 .../execute/PRFunctionExecutionDUnitTest.java      |    4 +-
 .../cache/locks/TXLockServiceDUnitTest.java        |    6 +-
 .../cache/partitioned/Bug39356DUnitTest.java       |    6 +-
 .../DeposePrimaryBucketMessageTest.java            |    4 +-
 .../cache/partitioned/FetchEntryMessageTest.java   |    4 +-
 .../FetchPartitionDetailsMessageTest.java          |    4 +-
 .../cache/partitioned/MoveBucketMessageTest.java   |    4 +-
 .../cache/partitioned/PartitionMessageTest.java    |    6 +-
 ...sistentColocatedPartitionedRegionDUnitTest.java |    8 +-
 .../PersistentPartitionedRegionDUnitTest.java      |   19 +-
 .../cache/partitioned/RemoveBucketMessageTest.java |    4 +-
 .../PersistentRVVRecoveryDUnitTest.java            |    6 +-
 .../PersistentRecoveryOrderDUnitTest.java          |   41 +-
 .../ParallelQueueRemovalMessageJUnitTest.java      |    4 +-
 ...nectingOutOfOffHeapMemoryListenerJUnitTest.java |    4 +-
 .../offheap/OutOfOffHeapMemoryDUnitTest.java       |    6 +-
 .../geode/internal/tcp/ConnectionJUnitTest.java    |    4 +-
 .../geode/internal/tcp/ConnectionTableTest.java    |    4 +-
 .../geode/management/ManagementTestRule.java       |    4 +-
 .../management/RegionManagementDUnitTest.java      |    4 +-
 .../beans/DistributedSystemBridgeJUnitTest.java    |    6 +-
 .../pdx/ClientsWithVersioningRetryDUnitTest.java   |    7 +-
 .../geode/pdx/DistributedSystemIdDUnitTest.java    |    6 +-
 .../apache/geode/pdx/PdxSerializableDUnitTest.java |    4 +-
 .../security/ClusterConfigNotEnabledDUnitTest.java |    4 +-
 .../java/org/apache/geode/test/fake/Fakes.java     |    4 +-
 .../lucene/internal/DestroyLuceneIndexMessage.java |    5 +-
 .../lucene/internal/LuceneBucketListener.java      |    6 +-
 .../internal/LuceneIndexForPartitionedRegion.java  |    6 +-
 .../web/controllers/AbstractBaseController.java    |    4 +-
 494 files changed, 2230 insertions(+), 6781 deletions(-)

diff --git a/extensions/geode-modules/src/main/java/org/apache/geode/modules/util/BootstrappingFunction.java b/extensions/geode-modules/src/main/java/org/apache/geode/modules/util/BootstrappingFunction.java
index 88a5a7b..82ca3e3 100644
--- a/extensions/geode-modules/src/main/java/org/apache/geode/modules/util/BootstrappingFunction.java
+++ b/extensions/geode-modules/src/main/java/org/apache/geode/modules/util/BootstrappingFunction.java
@@ -28,7 +28,7 @@ import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.MembershipListener;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
@@ -88,7 +88,8 @@ public class BootstrappingFunction implements Function, MembershipListener, Data
   }
 
   private void registerAsMembershipListener(Cache cache) {
-    DM dm = ((InternalDistributedSystem) cache.getDistributedSystem()).getDistributionManager();
+    DistributionManager dm =
+        ((InternalDistributedSystem) cache.getDistributedSystem()).getDistributionManager();
     dm.addMembershipListener(this);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/admin/GemFireMemberStatus.java b/geode-core/src/main/java/org/apache/geode/admin/GemFireMemberStatus.java
index 1e01b40..8d0c6a5 100755
--- a/geode-core/src/main/java/org/apache/geode/admin/GemFireMemberStatus.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/GemFireMemberStatus.java
@@ -30,8 +30,8 @@ import org.apache.geode.cache.client.PoolManager;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.admin.ClientHealthMonitoringRegion;
@@ -635,7 +635,7 @@ public class GemFireMemberStatus implements Serializable {
 
   protected void initializePeers(DistributedSystem distributedSystem) {
     InternalDistributedSystem ids = (InternalDistributedSystem) distributedSystem;
-    DM dm = ids.getDistributionManager();
+    DistributionManager dm = ids.getDistributionManager();
     Set connections = dm.getOtherNormalDistributionManagerIds();
     Set connectionsIDs = new HashSet(connections.size());
     for (Iterator iter = connections.iterator(); iter.hasNext();) {
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/AbstractHealthEvaluator.java b/geode-core/src/main/java/org/apache/geode/admin/internal/AbstractHealthEvaluator.java
index 14235b9..ef5b1b1 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/AbstractHealthEvaluator.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/AbstractHealthEvaluator.java
@@ -20,7 +20,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.admin.GemFireHealth;
 import org.apache.geode.admin.GemFireHealthConfig;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
@@ -58,7 +58,7 @@ public abstract class AbstractHealthEvaluator {
    * processed before the <code>InternalDistributedSystem</code>'s <code>DistributionManager</code>
    * is set.
    */
-  protected AbstractHealthEvaluator(GemFireHealthConfig config, DM dm) {
+  protected AbstractHealthEvaluator(GemFireHealthConfig config, DistributionManager dm) {
     this.numEvaluations = 0;
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/AdminDistributedSystemImpl.java b/geode-core/src/main/java/org/apache/geode/admin/internal/AdminDistributedSystemImpl.java
index 8e23a31..82bcc97 100755
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/AdminDistributedSystemImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/AdminDistributedSystemImpl.java
@@ -615,7 +615,7 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
   }
 
   protected void checkCancellation() {
-    DM dm = this.getDistributionManager();
+    DistributionManager dm = this.getDistributionManager();
     // TODO does dm == null mean we're dead?
     if (dm != null) {
       dm.getCancelCriterion().checkCancelInProgress(null);
@@ -901,7 +901,7 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
    * Returns the DistributionManager this implementation is using to connect to the distributed
    * system.
    */
-  public DM getDistributionManager() {
+  public DistributionManager getDistributionManager() {
     if (this.gfManagerAgent == null) {
       return null;
     }
@@ -1647,7 +1647,7 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
   private GfManagerAgentConfig buildAgentConfig(InternalLogWriter logWriter) {
     RemoteTransportConfig conf = new RemoteTransportConfig(isMcastEnabled(), getDisableTcp(),
         getDisableAutoReconnect(), getBindAddress(), buildSSLConfig(), parseLocators(),
-        getMembershipPortRange(), getTcpPort(), DistributionManager.ADMIN_ONLY_DM_TYPE);
+        getMembershipPortRange(), getTcpPort(), ClusterDistributionManager.ADMIN_ONLY_DM_TYPE);
     return new GfManagerAgentConfig(getSystemName(), conf, logWriter, this.alertLevel.getSeverity(),
         this, this);
   }
@@ -2171,7 +2171,7 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
 
   public Set<PersistentID> getMissingPersistentMembers() throws AdminException {
     connectAdminDS();
-    DM dm = getDistributionManager();
+    DistributionManager dm = getDistributionManager();
     if (dm == null) {
       throw new IllegalStateException(
           LocalizedStrings.AdminDistributedSystemImpl_CONNECT_HAS_NOT_BEEN_INVOKED_ON_THIS_ADMINDISTRIBUTEDSYSTEM
@@ -2180,13 +2180,13 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
     return getMissingPersistentMembers(dm);
   }
 
-  public static Set<PersistentID> getMissingPersistentMembers(DM dm) {
+  public static Set<PersistentID> getMissingPersistentMembers(DistributionManager dm) {
     return MissingPersistentIDsRequest.send(dm);
   }
 
   public void revokePersistentMember(InetAddress host, String directory) throws AdminException {
     connectAdminDS();
-    DM dm = getDistributionManager();
+    DistributionManager dm = getDistributionManager();
     if (dm == null) {
       throw new IllegalStateException(
           LocalizedStrings.AdminDistributedSystemImpl_CONNECT_HAS_NOT_BEEN_INVOKED_ON_THIS_ADMINDISTRIBUTEDSYSTEM
@@ -2198,7 +2198,7 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
 
   public void revokePersistentMember(UUID diskStoreID) throws AdminException {
     connectAdminDS();
-    DM dm = getDistributionManager();
+    DistributionManager dm = getDistributionManager();
     if (dm == null) {
       throw new IllegalStateException(
           LocalizedStrings.AdminDistributedSystemImpl_CONNECT_HAS_NOT_BEEN_INVOKED_ON_THIS_ADMINDISTRIBUTEDSYSTEM
@@ -2208,7 +2208,7 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
 
   }
 
-  public static void revokePersistentMember(DM dm, UUID diskStoreID) {
+  public static void revokePersistentMember(DistributionManager dm, UUID diskStoreID) {
     PersistentMemberPattern pattern = new PersistentMemberPattern(diskStoreID);
     boolean success = false;
     try {
@@ -2246,7 +2246,8 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
    *
    * @deprecated use {@link #revokePersistentMember(UUID)} instead
    */
-  public static void revokePersistentMember(DM dm, InetAddress host, String directory) {
+  public static void revokePersistentMember(DistributionManager dm, InetAddress host,
+      String directory) {
 
     PersistentMemberPattern pattern =
         new PersistentMemberPattern(host, directory, System.currentTimeMillis());
@@ -2273,7 +2274,7 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
 
   public Set shutDownAllMembers(long timeout) throws AdminException {
     connectAdminDS();
-    DM dm = getDistributionManager();
+    DistributionManager dm = getDistributionManager();
     if (dm == null) {
       throw new IllegalStateException(
           LocalizedStrings.AdminDistributedSystemImpl_CONNECT_HAS_NOT_BEEN_INVOKED_ON_THIS_ADMINDISTRIBUTEDSYSTEM
@@ -2291,7 +2292,7 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
    *        timeout is exceeded, persistent recovery after the shutdown may need to do a GII. -1
    *        indicates that the shutdown should wait forever.
    */
-  public static Set shutDownAllMembers(DM dm, long timeout) {
+  public static Set shutDownAllMembers(DistributionManager dm, long timeout) {
     return ShutdownAllRequest.send(dm, timeout);
   }
 
@@ -2301,7 +2302,7 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
 
   public BackupStatus backupAllMembers(File targetDir, File baselineDir) throws AdminException {
     connectAdminDS();
-    DM dm = getDistributionManager();
+    DistributionManager dm = getDistributionManager();
     if (dm == null) {
       throw new IllegalStateException(
           LocalizedStrings.AdminDistributedSystemImpl_CONNECT_HAS_NOT_BEEN_INVOKED_ON_THIS_ADMINDISTRIBUTEDSYSTEM
@@ -2310,14 +2311,14 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
     return backupAllMembers(dm, targetDir, baselineDir);
   }
 
-  public static BackupStatus backupAllMembers(DM dm, File targetDir, File baselineDir)
-      throws AdminException {
+  public static BackupStatus backupAllMembers(DistributionManager dm, File targetDir,
+      File baselineDir) throws AdminException {
     return new BackupStatusImpl(BackupUtil.backupAllMembers(dm, targetDir, baselineDir));
   }
 
   public Map<DistributedMember, Set<PersistentID>> compactAllDiskStores() throws AdminException {
     connectAdminDS();
-    DM dm = getDistributionManager();
+    DistributionManager dm = getDistributionManager();
     if (dm == null) {
       throw new IllegalStateException(
           LocalizedStrings.AdminDistributedSystemImpl_CONNECT_HAS_NOT_BEEN_INVOKED_ON_THIS_ADMINDISTRIBUTEDSYSTEM
@@ -2326,8 +2327,8 @@ public class AdminDistributedSystemImpl implements org.apache.geode.admin.AdminD
     return compactAllDiskStores(dm);
   }
 
-  public static Map<DistributedMember, Set<PersistentID>> compactAllDiskStores(DM dm)
-      throws AdminException {
+  public static Map<DistributedMember, Set<PersistentID>> compactAllDiskStores(
+      DistributionManager dm) throws AdminException {
     return CompactRequest.send(dm);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java b/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java
index c93d0f4..adf03ca 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/CacheHealthEvaluator.java
@@ -22,7 +22,7 @@ import org.apache.geode.CancelException;
 import org.apache.geode.admin.CacheHealthConfig;
 import org.apache.geode.admin.GemFireHealthConfig;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.cache.CacheLifecycleListener;
@@ -72,7 +72,7 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec
   /**
    * Creates a new {@code CacheHealthEvaluator}
    */
-  CacheHealthEvaluator(GemFireHealthConfig config, DM dm) {
+  CacheHealthEvaluator(GemFireHealthConfig config, DistributionManager dm) {
     super(config, dm);
 
     this.config = config;
@@ -97,7 +97,7 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec
   /**
    * Initializes the state of this evaluator based on the given cache instance.
    */
-  private void initialize(InternalCache cache, DM dm) {
+  private void initialize(InternalCache cache, DistributionManager dm) {
     StringBuilder sb = new StringBuilder();
     if (cache != null) {
       this.cacheStats = cache.getCachePerfStats();
@@ -123,7 +123,7 @@ class CacheHealthEvaluator extends AbstractHealthEvaluator implements CacheLifec
   @Override
   public void cacheCreated(InternalCache cache) {
     InternalDistributedSystem system = (InternalDistributedSystem) cache.getDistributedSystem();
-    DM dm = system.getDistributionManager();
+    DistributionManager dm = system.getDistributionManager();
     initialize(cache, dm);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/CacheServerImpl.java b/geode-core/src/main/java/org/apache/geode/admin/internal/CacheServerImpl.java
index 4bfa80b..76a4c44 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/CacheServerImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/CacheServerImpl.java
@@ -17,7 +17,7 @@ package org.apache.geode.admin.internal;
 import static org.apache.geode.distributed.ConfigurationProperties.*;
 
 import org.apache.geode.admin.*;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.internal.admin.GemFireVM;
 import org.apache.geode.internal.admin.remote.RemoteApplicationVM;
@@ -96,7 +96,8 @@ public class CacheServerImpl extends ManagedSystemMemberImpl implements CacheVm,
   }
 
   public boolean isRunning() {
-    DM dm = ((AdminDistributedSystemImpl) getDistributedSystem()).getDistributionManager();
+    DistributionManager dm =
+        ((AdminDistributedSystemImpl) getDistributedSystem()).getDistributionManager();
     if (dm == null) {
       try {
         return this.controller.isRunning(this);
@@ -104,7 +105,7 @@ public class CacheServerImpl extends ManagedSystemMemberImpl implements CacheVm,
         return false;
       }
     }
-    return ((DistributionManager) dm).getDistributionManagerIdsIncludingAdmin()
+    return ((ClusterDistributionManager) dm).getDistributionManagerIdsIncludingAdmin()
         .contains(getDistributedMember());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/DistributedSystemHealthEvaluator.java b/geode-core/src/main/java/org/apache/geode/admin/internal/DistributedSystemHealthEvaluator.java
index 596805c..55cde04 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/DistributedSystemHealthEvaluator.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/DistributedSystemHealthEvaluator.java
@@ -18,7 +18,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.geode.admin.DistributedSystemHealthConfig;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.MembershipListener;
@@ -47,7 +47,7 @@ class DistributedSystemHealthEvaluator extends AbstractHealthEvaluator
   /**
    * The distribution manager with which this MembershipListener is registered
    */
-  private DM dm;
+  private DistributionManager dm;
 
   /** The description of the distributed system being evaluated */
   private String description;
@@ -62,7 +62,7 @@ class DistributedSystemHealthEvaluator extends AbstractHealthEvaluator
   /**
    * Creates a new <code>DistributedSystemHealthEvaluator</code>
    */
-  DistributedSystemHealthEvaluator(DistributedSystemHealthConfig config, DM dm) {
+  DistributedSystemHealthEvaluator(DistributedSystemHealthConfig config, DistributionManager dm) {
     super(null, dm);
 
     this.config = config;
@@ -73,8 +73,8 @@ class DistributedSystemHealthEvaluator extends AbstractHealthEvaluator
     sb.append("Distributed System ");
 
     String desc = null;
-    if (dm instanceof DistributionManager) {
-      desc = ((DistributionManager) dm).getDistributionConfigDescription();
+    if (dm instanceof ClusterDistributionManager) {
+      desc = ((ClusterDistributionManager) dm).getDistributionConfigDescription();
     }
 
     if (desc != null) {
@@ -148,8 +148,8 @@ class DistributedSystemHealthEvaluator extends AbstractHealthEvaluator
     synchronized (this) {
       int kind = id.getVmKind();
       switch (kind) {
-        case DistributionManager.LOCATOR_DM_TYPE:
-        case DistributionManager.NORMAL_DM_TYPE:
+        case ClusterDistributionManager.LOCATOR_DM_TYPE:
+        case ClusterDistributionManager.NORMAL_DM_TYPE:
           this.crashedApplications++;
           break;
         default:
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/DistributionLocatorImpl.java b/geode-core/src/main/java/org/apache/geode/admin/internal/DistributionLocatorImpl.java
index 722bf9f..8b03ef3 100755
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/DistributionLocatorImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/DistributionLocatorImpl.java
@@ -25,8 +25,8 @@ import org.apache.geode.admin.AdminDistributedSystem;
 import org.apache.geode.admin.DistributionLocator;
 import org.apache.geode.admin.DistributionLocatorConfig;
 import org.apache.geode.admin.ManagedEntityConfig;
-import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.admin.remote.DistributionLocatorId;
 import org.apache.geode.internal.i18n.LocalizedStrings;
@@ -174,7 +174,8 @@ public class DistributionLocatorImpl implements DistributionLocator, InternalMan
   }
 
   public boolean isRunning() {
-    DM dm = ((AdminDistributedSystemImpl) getDistributedSystem()).getDistributionManager();
+    DistributionManager dm =
+        ((AdminDistributedSystemImpl) getDistributedSystem()).getDistributionManager();
     if (dm == null) {
       try {
         return this.controller.isRunning(this);
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/GemFireHealthEvaluator.java b/geode-core/src/main/java/org/apache/geode/admin/internal/GemFireHealthEvaluator.java
index 1573ca7..f3cdc56 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/GemFireHealthEvaluator.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/GemFireHealthEvaluator.java
@@ -22,7 +22,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.admin.GemFireHealth;
 import org.apache.geode.admin.GemFireHealthConfig;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
@@ -73,7 +73,7 @@ public class GemFireHealthEvaluator {
    * @param config The configuration that determines whether or GemFire is healthy
    * @param dm The distribution manager
    */
-  public GemFireHealthEvaluator(GemFireHealthConfig config, DistributionManager dm) {
+  public GemFireHealthEvaluator(GemFireHealthConfig config, ClusterDistributionManager dm) {
     if (config == null) {
       throw new NullPointerException(
           LocalizedStrings.GemFireHealthEvaluator_NULL_GEMFIREHEALTHCONFIG.toLocalizedString());
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java b/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
index b257a17..f420f61 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/MemberHealthEvaluator.java
@@ -20,8 +20,8 @@ import org.apache.geode.CancelException;
 import org.apache.geode.admin.GemFireHealthConfig;
 import org.apache.geode.admin.MemberHealthConfig;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DMStats;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.cache.CachePerfStats;
@@ -56,7 +56,7 @@ class MemberHealthEvaluator extends AbstractHealthEvaluator {
   /**
    * Creates a new <code>MemberHealthEvaluator</code>
    */
-  MemberHealthEvaluator(GemFireHealthConfig config, DM dm) {
+  MemberHealthEvaluator(GemFireHealthConfig config, DistributionManager dm) {
     super(config, dm);
 
     this.config = config;
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/SystemMemberCacheEventProcessor.java b/geode-core/src/main/java/org/apache/geode/admin/internal/SystemMemberCacheEventProcessor.java
index f17ef14..d356a68 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/SystemMemberCacheEventProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/internal/SystemMemberCacheEventProcessor.java
@@ -30,7 +30,7 @@ import org.apache.geode.admin.SystemMemberRegionEvent;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Operation;
 import org.apache.geode.cache.Region;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.logging.LogService;
@@ -79,7 +79,7 @@ public class SystemMemberCacheEventProcessor {
     protected Operation op;
 
     @Override
-    protected void process(DistributionManager dm) {
+    protected void process(ClusterDistributionManager dm) {
       AdminDistributedSystemImpl admin = AdminDistributedSystemImpl.getConnectedInstance();
       if (admin == null) {
         if (logger.isDebugEnabled()) {
diff --git a/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentImpl.java b/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentImpl.java
index c635f03..855d522 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentImpl.java
@@ -54,7 +54,7 @@ import org.apache.geode.admin.AdminException;
 import org.apache.geode.admin.jmx.Agent;
 import org.apache.geode.admin.jmx.AgentConfig;
 import org.apache.geode.admin.jmx.AgentFactory;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.i18n.StringId;
 import org.apache.geode.internal.Banner;
 import org.apache.geode.internal.ExitCode;
@@ -411,7 +411,7 @@ public class AgentImpl implements org.apache.geode.admin.jmx.Agent,
           return ((AdminDistributedSystemJmxImpl) this.system).getObjectName();
         }
 
-        DistributionManager.setIsDedicatedAdminVM(true);
+        ClusterDistributionManager.setIsDedicatedAdminVM(true);
 
         AdminDistributedSystemJmxImpl systemJmx = (AdminDistributedSystemJmxImpl) this.system;
         if (systemJmx == null) {
@@ -477,7 +477,7 @@ public class AgentImpl implements org.apache.geode.admin.jmx.Agent,
         logger.warn(e.getMessage(), e);
         throw e;
       } finally {
-        DistributionManager.setIsDedicatedAdminVM(false);
+        ClusterDistributionManager.setIsDedicatedAdminVM(false);
       }
     }
   }
diff --git a/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentLauncher.java b/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentLauncher.java
index 1819655..922dc85 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentLauncher.java
+++ b/geode-core/src/main/java/org/apache/geode/admin/jmx/internal/AgentLauncher.java
@@ -43,7 +43,7 @@ import org.apache.geode.admin.AdminException;
 import org.apache.geode.admin.jmx.Agent;
 import org.apache.geode.admin.jmx.AgentConfig;
 import org.apache.geode.admin.jmx.AgentFactory;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.internal.ExitCode;
 import org.apache.geode.internal.OSProcess;
 import org.apache.geode.internal.PureJavaMode;
@@ -376,7 +376,7 @@ public class AgentLauncher {
   }
 
   private Agent createAgent(final Properties props) throws IOException, AdminException {
-    DistributionManager.setIsDedicatedAdminVM(true);
+    ClusterDistributionManager.setIsDedicatedAdminVM(true);
     SystemFailure.setExitOK(true);
 
     final AgentConfigImpl config = new AgentConfigImpl(props);
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/CacheServerLoadMessage.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/CacheServerLoadMessage.java
index da7b236..5bd4f9b 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/CacheServerLoadMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/CacheServerLoadMessage.java
@@ -23,7 +23,7 @@ import java.util.List;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.cache.server.ServerLoad;
 import org.apache.geode.distributed.Locator;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.InternalLocator;
 import org.apache.geode.distributed.internal.SerialDistributionMessage;
 import org.apache.geode.distributed.internal.ServerLocation;
@@ -55,7 +55,7 @@ public class CacheServerLoadMessage extends SerialDistributionMessage {
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     updateLocalLocators();
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/DistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/DistributedSystem.java
index 8c24214..ccb6ab6 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/DistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/DistributedSystem.java
@@ -35,8 +35,8 @@ import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.ClientCacheFactory;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.ClassPathLoader;
@@ -156,7 +156,7 @@ public abstract class DistributedSystem implements StatisticsFactory {
       config = new Properties();
     }
     synchronized (existingSystemsLock) {
-      if (DistributionManager.isDedicatedAdminVM()) {
+      if (ClusterDistributionManager.isDedicatedAdminVM()) {
         // For a dedicated admin VM, check to see if there is already
         // a connect that will suit our purposes.
         DistributedSystem existingSystem = getConnection(config);
@@ -323,7 +323,7 @@ public abstract class DistributedSystem implements StatisticsFactory {
             LocalizedStrings.DistributedSystem_THIS_VM_ALREADY_HAS_ONE_OR_MORE_DISTRIBUTED_SYSTEM_CONNECTIONS_0
                 .toLocalizedString(existingSystems));
       }
-      DistributionManager.setIsDedicatedAdminVM(adminOnly);
+      ClusterDistributionManager.setIsDedicatedAdminVM(adminOnly);
     }
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java
index 726cde2..37a76a9 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterConfigurationService.java
@@ -387,7 +387,7 @@ public class ClusterConfigurationService {
   public void downloadJarFromOtherLocators(String groupName, String jarName)
       throws IllegalStateException, IOException {
     logger.info("Getting Jar files from other locators");
-    DM dm = this.cache.getDistributionManager();
+    DistributionManager dm = this.cache.getDistributionManager();
     DistributedMember me = this.cache.getMyId();
     List<DistributedMember> locators =
         new ArrayList<>(dm.getAllHostedLocatorsWithSharedConfiguration().keySet());
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
similarity index 97%
copy from geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
copy to geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
index 5a4de3d..7f96a7b 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ClusterDistributionManager.java
@@ -105,7 +105,7 @@ import org.apache.geode.internal.util.concurrent.StoppableReentrantLock;
  * @see DistributionMessage#process
  * @see IgnoredByManager
  */
-public class DistributionManager implements DM {
+public class ClusterDistributionManager implements DistributionManager {
 
   private static final Logger logger = LogService.getLogger();
 
@@ -281,7 +281,7 @@ public class DistributionManager implements DM {
   private boolean waitingForElderChange = false;
 
   /**
-   * @see DM#isAdam()
+   * @see DistributionManager#isAdam()
    */
   private boolean adam = false;
 
@@ -524,9 +524,9 @@ public class DistributionManager implements DM {
    *
    * @param system The distributed system to which this distribution manager will send messages.
    */
-  static DistributionManager create(InternalDistributedSystem system) {
+  static ClusterDistributionManager create(InternalDistributedSystem system) {
 
-    DistributionManager distributionManager = null;
+    ClusterDistributionManager distributionManager = null;
     boolean beforeJoined = true;
 
     try {
@@ -550,7 +550,7 @@ public class DistributionManager implements DM {
 
       long start = System.currentTimeMillis();
 
-      distributionManager = new DistributionManager(system, transport);
+      distributionManager = new ClusterDistributionManager(system, transport);
       distributionManager.assertDistributionManagerType();
 
       beforeJoined = false; // we have now joined the system
@@ -649,7 +649,8 @@ public class DistributionManager implements DM {
    * @param transport The configuration for the communications transport
    *
    */
-  private DistributionManager(RemoteTransportConfig transport, InternalDistributedSystem system) {
+  private ClusterDistributionManager(RemoteTransportConfig transport,
+      InternalDistributedSystem system) {
 
     this.dmType = transport.getVmKind();
     this.system = system;
@@ -706,10 +707,10 @@ public class DistributionManager implements DM {
         }
         ThreadFactory tf = new ThreadFactory() {
           public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incSerialThreadStarts();
+            ClusterDistributionManager.this.stats.incSerialThreadStarts();
             final Runnable r = new Runnable() {
               public void run() {
-                DistributionManager.this.stats.incNumSerialThreads(1);
+                ClusterDistributionManager.this.stats.incNumSerialThreads(1);
                 try {
                   ConnectionTable.threadWantsSharedResources();
                   Connection.makeReaderThread();
@@ -717,7 +718,7 @@ public class DistributionManager implements DM {
                   // command.run();
                 } finally {
                   ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incNumSerialThreads(-1);
+                  ClusterDistributionManager.this.stats.incNumSerialThreads(-1);
                 }
               }
             };
@@ -735,17 +736,17 @@ public class DistributionManager implements DM {
         BlockingQueue q = new LinkedBlockingQueue();
         ThreadFactory tf = new ThreadFactory() {
           public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incViewThreadStarts();
+            ClusterDistributionManager.this.stats.incViewThreadStarts();
             final Runnable r = new Runnable() {
               public void run() {
-                DistributionManager.this.stats.incNumViewThreads(1);
+                ClusterDistributionManager.this.stats.incNumViewThreads(1);
                 try {
                   ConnectionTable.threadWantsSharedResources();
                   Connection.makeReaderThread();
                   runUntilShutdown(command);
                 } finally {
                   ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incNumViewThreads(-1);
+                  ClusterDistributionManager.this.stats.incNumViewThreads(-1);
                 }
               }
             };
@@ -771,17 +772,17 @@ public class DistributionManager implements DM {
           private int next = 0;
 
           public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incProcessingThreadStarts();
+            ClusterDistributionManager.this.stats.incProcessingThreadStarts();
             final Runnable r = new Runnable() {
               public void run() {
-                DistributionManager.this.stats.incNumProcessingThreads(1);
+                ClusterDistributionManager.this.stats.incNumProcessingThreads(1);
                 try {
                   ConnectionTable.threadWantsSharedResources();
                   Connection.makeReaderThread();
                   runUntilShutdown(command);
                 } finally {
                   ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incNumProcessingThreads(-1);
+                  ClusterDistributionManager.this.stats.incNumProcessingThreads(-1);
                 }
               }
             };
@@ -810,17 +811,17 @@ public class DistributionManager implements DM {
           private int next = 0;
 
           public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incHighPriorityThreadStarts();
+            ClusterDistributionManager.this.stats.incHighPriorityThreadStarts();
             final Runnable r = new Runnable() {
               public void run() {
-                DistributionManager.this.stats.incHighPriorityThreads(1);
+                ClusterDistributionManager.this.stats.incHighPriorityThreads(1);
                 try {
                   ConnectionTable.threadWantsSharedResources();
                   Connection.makeReaderThread();
                   runUntilShutdown(command);
                 } finally {
                   ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incHighPriorityThreads(-1);
+                  ClusterDistributionManager.this.stats.incHighPriorityThreads(-1);
                 }
               }
             };
@@ -841,17 +842,17 @@ public class DistributionManager implements DM {
           private int next = 0;
 
           public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incWaitingThreadStarts();
+            ClusterDistributionManager.this.stats.incWaitingThreadStarts();
             final Runnable r = new Runnable() {
               public void run() {
-                DistributionManager.this.stats.incWaitingThreads(1);
+                ClusterDistributionManager.this.stats.incWaitingThreads(1);
                 try {
                   ConnectionTable.threadWantsSharedResources();
                   Connection.makeReaderThread();
                   runUntilShutdown(command);
                 } finally {
                   ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incWaitingThreads(-1);
+                  ClusterDistributionManager.this.stats.incWaitingThreads(-1);
                 }
               }
             };
@@ -878,17 +879,17 @@ public class DistributionManager implements DM {
           private int next = 0;
 
           public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incWaitingThreadStarts();// will it be ok?
+            ClusterDistributionManager.this.stats.incWaitingThreadStarts();// will it be ok?
             final Runnable r = new Runnable() {
               public void run() {
-                DistributionManager.this.stats.incWaitingThreads(1);// will it be ok
+                ClusterDistributionManager.this.stats.incWaitingThreads(1);// will it be ok
                 try {
                   ConnectionTable.threadWantsSharedResources();
                   Connection.makeReaderThread();
                   runUntilShutdown(command);
                 } finally {
                   ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incWaitingThreads(-1);
+                  ClusterDistributionManager.this.stats.incWaitingThreads(-1);
                 }
               }
             };
@@ -917,7 +918,7 @@ public class DistributionManager implements DM {
           private int next = 0;
 
           public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incPartitionedRegionThreadStarts();
+            ClusterDistributionManager.this.stats.incPartitionedRegionThreadStarts();
             final Runnable r = new Runnable() {
               public void run() {
                 stats.incPartitionedRegionThreads(1);
@@ -959,7 +960,7 @@ public class DistributionManager implements DM {
           private int next = 0;
 
           public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incFunctionExecutionThreadStarts();
+            ClusterDistributionManager.this.stats.incFunctionExecutionThreadStarts();
             final Runnable r = new Runnable() {
               public void run() {
                 stats.incFunctionExecutionThreads(1);
@@ -1035,7 +1036,8 @@ public class DistributionManager implements DM {
    *
    * @param system The distributed system to which this distribution manager will send messages.
    */
-  private DistributionManager(InternalDistributedSystem system, RemoteTransportConfig transport) {
+  private ClusterDistributionManager(InternalDistributedSystem system,
+      RemoteTransportConfig transport) {
     this(transport, system);
 
     boolean finishedConstructor = false;
@@ -1081,7 +1083,7 @@ public class DistributionManager implements DM {
   }
 
   public static void setIsDedicatedAdminVM(boolean isDedicatedAdminVM) {
-    DistributionManager.isDedicatedAdminVM = isDedicatedAdminVM;
+    ClusterDistributionManager.isDedicatedAdminVM = isDedicatedAdminVM;
   }
 
   public static Boolean getIsStartupThread() {
@@ -1089,7 +1091,7 @@ public class DistributionManager implements DM {
   }
 
   public static void setIsStartupThread(Boolean isStartup) {
-    DistributionManager.isStartupThread.set(isStartup);
+    ClusterDistributionManager.isStartupThread.set(isStartup);
   }
 
   //////////////////// Instance Methods /////////////////////
@@ -2186,9 +2188,9 @@ public class DistributionManager implements DM {
   /**
    * This thread processes member events as they occur.
    *
-   * @see org.apache.geode.distributed.internal.DistributionManager.MemberCrashedEvent
-   * @see org.apache.geode.distributed.internal.DistributionManager.MemberJoinedEvent
-   * @see org.apache.geode.distributed.internal.DistributionManager.MemberDepartedEvent
+   * @see ClusterDistributionManager.MemberCrashedEvent
+   * @see ClusterDistributionManager.MemberJoinedEvent
+   * @see ClusterDistributionManager.MemberDepartedEvent
    *
    */
   protected class MemberEventInvoker implements Runnable {
@@ -2204,11 +2206,13 @@ public class DistributionManager implements DM {
         // if (getCancelCriterion().isCancelInProgress()) {
         // break; // no message, just quit
         // }
-        if (!DistributionManager.this.system.isConnected && DistributionManager.this.isClosed()) {
+        if (!ClusterDistributionManager.this.system.isConnected
+            && ClusterDistributionManager.this.isClosed()) {
           break;
         }
         try {
-          MemberEvent ev = (MemberEvent) DistributionManager.this.membershipEventQueue.take();
+          MemberEvent ev =
+              (MemberEvent) ClusterDistributionManager.this.membershipEventQueue.take();
           handleMemberEvent(ev);
         } catch (InterruptedException e) {
           if (isCloseInProgress()) {
@@ -2240,7 +2244,7 @@ public class DistributionManager implements DM {
         }
       } // for
       if (logger.isTraceEnabled()) {
-        logger.trace("MemberEventInvoker on {} stopped", DistributionManager.this);
+        logger.trace("MemberEventInvoker on {} stopped", ClusterDistributionManager.this);
       }
     }
   }
@@ -2436,9 +2440,9 @@ public class DistributionManager implements DM {
       throw new IncompatibleSystemException(rejectionMessage);
     }
 
-    boolean isAdminDM = getId().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE
-        || getId().getVmKind() == DistributionManager.LOCATOR_DM_TYPE
-        || DistributionManager.isDedicatedAdminVM()
+    boolean isAdminDM = getId().getVmKind() == ClusterDistributionManager.ADMIN_ONLY_DM_TYPE
+        || getId().getVmKind() == ClusterDistributionManager.LOCATOR_DM_TYPE
+        || ClusterDistributionManager.isDedicatedAdminVM()
         || Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE);
 
     boolean receivedAny = this.receivedStartupResponse;
@@ -2806,7 +2810,7 @@ public class DistributionManager implements DM {
       this.membersAndAdmin = Collections.unmodifiableSet(stmp);
     } // synchronized
 
-    if (theId.getVmKind() != DistributionManager.LOCATOR_DM_TYPE) {
+    if (theId.getVmKind() != ClusterDistributionManager.LOCATOR_DM_TYPE) {
       this.stats.incNodes(1);
     }
     logger.info(LocalizedMessage.create(
@@ -2982,7 +2986,7 @@ public class DistributionManager implements DM {
     removeUnfinishedStartup(theId, true);
 
     if (removeManager(theId, crashed, reason)) {
-      if (theId.getVmKind() != DistributionManager.LOCATOR_DM_TYPE) {
+      if (theId.getVmKind() != ClusterDistributionManager.LOCATOR_DM_TYPE) {
         this.stats.incNodes(-1);
       }
       StringId msg;
@@ -3109,7 +3113,7 @@ public class DistributionManager implements DM {
     long startTime = DistributionStats.getStatTime();
 
     Set result = sendViaMembershipManager(message.getRecipients(), message,
-        DistributionManager.this, this.stats);
+        ClusterDistributionManager.this, this.stats);
     long endTime = 0L;
     if (DistributionStats.enableClockStats) {
       endTime = NanoTimer.getTime();
@@ -3151,7 +3155,7 @@ public class DistributionManager implements DM {
         | InvalidDeltaException | CancelException ex) {
       throw ex;
     } catch (Exception ex) {
-      DistributionManager.this.exceptionInThreads = true;
+      ClusterDistributionManager.this.exceptionInThreads = true;
       String receiver = "NULL";
       if (message != null) {
         receiver = message.getRecipientsDescription();
@@ -3180,7 +3184,7 @@ public class DistributionManager implements DM {
    * @throws NotSerializableException If content cannot be serialized
    */
   private Set sendViaMembershipManager(InternalDistributedMember[] destinations,
-      DistributionMessage content, DistributionManager dm, DistributionStats stats)
+      DistributionMessage content, ClusterDistributionManager dm, DistributionStats stats)
       throws NotSerializableException {
     if (membershipManager == null) {
       logger.warn(LocalizedMessage.create(
@@ -3206,7 +3210,7 @@ public class DistributionManager implements DM {
      * will only be initialized upto the point at which it called startThreads
      */
     waitUntilReadyForMessages();
-    message.schedule(DistributionManager.this);
+    message.schedule(ClusterDistributionManager.this);
   }
 
   @Override
@@ -3964,9 +3968,9 @@ public class DistributionManager implements DM {
    *
    */
   private class DMListener implements DistributedMembershipListener {
-    DistributionManager dm;
+    ClusterDistributionManager dm;
 
-    public DMListener(DistributionManager dm) {
+    public DMListener(ClusterDistributionManager dm) {
       this.dm = dm;
     }
 
@@ -3978,7 +3982,7 @@ public class DistributionManager implements DM {
     @Override
     public void membershipFailure(String reason, Throwable t) {
       exceptionInThreads = true;
-      DistributionManager.this.rootCause = t;
+      ClusterDistributionManager.this.rootCause = t;
       getSystem().disconnect(reason, t, true);
     }
 
@@ -4036,7 +4040,7 @@ public class DistributionManager implements DM {
     }
 
     @Override
-    public DistributionManager getDM() {
+    public ClusterDistributionManager getDM() {
       return dm;
     }
 
@@ -4064,14 +4068,14 @@ public class DistributionManager implements DM {
       return this.id;
     }
 
-    void handleEvent(DistributionManager manager) {
+    void handleEvent(ClusterDistributionManager manager) {
       handleEvent(manager, manager.membershipListeners.keySet());
       handleEvent(manager, manager.allMembershipListeners);
     }
 
     protected abstract void handleEvent(MembershipListener listener);
 
-    private void handleEvent(DistributionManager manager,
+    private void handleEvent(ClusterDistributionManager manager,
         Set<MembershipListener> membershipListeners) {
       for (MembershipListener listener : membershipListeners) {
         try {
@@ -4227,7 +4231,7 @@ public class DistributionManager implements DM {
     }
 
     @Override
-    public void handleEvent(DistributionManager manager) {
+    public void handleEvent(ClusterDistributionManager manager) {
       manager.handleViewInstalledEvent(this);
     }
 
@@ -4421,7 +4425,7 @@ public class DistributionManager implements DM {
     synchronized (this.membersLock) {
       HashSet<InternalDistributedMember> result = new HashSet<InternalDistributedMember>();
       for (InternalDistributedMember m : this.members.keySet()) {
-        if (m.getVmKind() != DistributionManager.LOCATOR_DM_TYPE) {
+        if (m.getVmKind() != ClusterDistributionManager.LOCATOR_DM_TYPE) {
           result.add(m);
         }
       }
@@ -4436,7 +4440,7 @@ public class DistributionManager implements DM {
     synchronized (this.membersLock) {
       HashSet<InternalDistributedMember> result = new HashSet<InternalDistributedMember>();
       for (InternalDistributedMember m : this.members.keySet()) {
-        if (m.getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
+        if (m.getVmKind() == ClusterDistributionManager.LOCATOR_DM_TYPE) {
           result.add(m);
         }
       }
@@ -4471,9 +4475,9 @@ public class DistributionManager implements DM {
 
 
   private static class Stopper extends CancelCriterion {
-    private DistributionManager dm;
+    private ClusterDistributionManager dm;
 
-    Stopper(DistributionManager dm) {
+    Stopper(ClusterDistributionManager dm) {
       this.dm = dm;
     }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/CollectingReplyProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/CollectingReplyProcessor.java
index f724ada..7580f4c 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/CollectingReplyProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/CollectingReplyProcessor.java
@@ -30,7 +30,7 @@ public class CollectingReplyProcessor<T> extends ReplyProcessor21 {
 
   private Map<InternalDistributedMember, T> results = new HashMap<InternalDistributedMember, T>();
 
-  public CollectingReplyProcessor(DM dm, Collection initMembers) {
+  public CollectingReplyProcessor(DistributionManager dm, Collection initMembers) {
     super(dm, initMembers);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DM.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DM.java
deleted file mode 100644
index 44ac87d..0000000
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DM.java
+++ /dev/null
@@ -1,489 +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;
-
-import java.io.NotSerializableException;
-import java.net.InetAddress;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.geode.CancelCriterion;
-import org.apache.geode.admin.GemFireHealthConfig;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheClosedException;
-import org.apache.geode.distributed.DistributedMember;
-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.MembershipManager;
-import org.apache.geode.internal.Version;
-import org.apache.geode.internal.cache.InternalCache;
-
-/**
- * This interface defines the services provided by any class that is a distribution manager.
- */
-public interface DM extends ReplySender {
-
-  boolean shutdownInProgress();
-
-  /**
-   * Returns the current "cache time" in milliseconds since the epoch. The "cache time" takes into
-   * account skew among the local clocks on the various machines involved in the cache.
-   */
-  long cacheTimeMillis();
-
-  /**
-   * Returns the id of this distribution manager.
-   */
-  InternalDistributedMember getDistributionManagerId();
-
-  /**
-   * Get a set of all other members (both admin ones and normal).
-   *
-   * @since GemFire 5.7
-   */
-  Set getAllOtherMembers();
-
-  /**
-   * Returns the ID in the membership view that is equal to the argument. If the ID is not in the
-   * view, the argument is returned.
-   */
-  InternalDistributedMember getCanonicalId(DistributedMember id);
-
-  /**
-   * removes members that have older versions from the given collection, typically a Set from a
-   * distribution advisor
-   *
-   * @since GemFire 8.0
-   */
-  void retainMembersWithSameOrNewerVersion(Collection<InternalDistributedMember> members,
-      Version version);
-
-  /**
-   * removes members that have the given version or later from the given collection, typically a Set
-   * from a distribution advisor
-   *
-   * @since GemFire 8.0
-   */
-  void removeMembersWithSameOrNewerVersion(Collection<InternalDistributedMember> members,
-      Version version);
-
-  /**
-   * Returns an unmodifiable set containing the identities of all of the known distribution
-   * managers. As of 7.0 this includes locators since they have a cache.
-   */
-  Set getDistributionManagerIds();
-
-  /**
-   * Returns an unmodifiable set containing the identities of all of the known "normal" distribution
-   * managers. This does not include locators or admin members.
-   */
-  Set getNormalDistributionManagerIds();
-
-  /**
-   * Returns an unmodifiable set containing the identities of all of the known distribution managers
-   * including admin members.
-   *
-   * @since GemFire 5.7
-   */
-  Set getDistributionManagerIdsIncludingAdmin();
-
-  /**
-   * Returns a private-memory list containing getDistributionManagerIds() minus our id.
-   */
-  Set getOtherDistributionManagerIds();
-
-  /**
-   * Returns a private-memory list containing getNormalDistributionManagerIds() minus our id.
-   */
-  Set getOtherNormalDistributionManagerIds();
-
-  /**
-   * Add a membership listener and return other DistribtionManagerIds as an atomic operation
-   */
-  Set addMembershipListenerAndGetDistributionManagerIds(MembershipListener l);
-
-  /**
-   * Add a membership listener for all members and return other DistribtionManagerIds as an atomic
-   * operation
-   *
-   * @since GemFire 5.7
-   */
-  Set addAllMembershipListenerAndGetAllIds(MembershipListener l);
-
-  /**
-   * Returns the identity of this <code>DistributionManager</code>
-   */
-  InternalDistributedMember getId();
-
-  /**
-   * Return true if no other distribution manager was in this group when he joined.
-   *
-   * @since GemFire 4.0
-   */
-  boolean isAdam();
-
-  /**
-   * Returns the identity of the oldest DM in this group.
-   *
-   * Note that this method may return null (no valid elders exist).
-   *
-   * @return the elder member, possibly null
-   * @since GemFire 4.0
-   */
-  InternalDistributedMember getElderId();
-
-  /**
-   * Return true if this is the oldest DM in this group.
-   *
-   * @since GemFire 5.0
-   */
-  boolean isElder();
-
-  /**
-   * Return true if this DM is a loner that is not part of a real distributed system.
-   */
-  boolean isLoner();
-
-  /**
-   * Returns the elder state or null if this DM is not the elder.
-   * <p>
-   * If useTryLock is true, then it will attempt to get a try-lock and throw IllegalStateException
-   * if another thread already holds the try-lock.
-   *
-   * @param force if true then this DM must become the elder.
-   * @param useTryLock if true then a try-lock will be used
-   * @throws IllegalStateException if elder try lock fails
-   * @since GemFire 4.0
-   */
-  ElderState getElderState(boolean force, boolean useTryLock);
-
-  /**
-   * Returns the membership port of the underlying distribution manager used for communication.
-   *
-   * @since GemFire 3.0
-   */
-  long getMembershipPort();
-
-  /**
-   * Sends a message
-   *
-   * @param msg
-   * @return recipients who did not receive the message
-   */
-  Set putOutgoing(DistributionMessage msg);
-
-  /**
-   * Returns the distributed system to which this distribution manager is connected.
-   */
-  InternalDistributedSystem getSystem();
-
-  /**
-   * Adds a <code>MembershipListener</code> to this distribution manager.
-   */
-  void addMembershipListener(MembershipListener l);
-
-  /**
-   * Removes a <code>MembershipListener</code> from this distribution manager.
-   *
-   * @throws IllegalArgumentException <code>l</code> was not registered on this distribution manager
-   */
-  void removeMembershipListener(MembershipListener l);
-
-  /**
-   * Removes a <code>MembershipListener</code> listening for all members from this distribution
-   * manager.
-   *
-   * @throws IllegalArgumentException <code>l</code> was not registered on this distribution manager
-   * @since GemFire 5.7
-   */
-  void removeAllMembershipListener(MembershipListener l);
-
-  /**
-   * Makes note of a new administration console (admin-only member).
-   *
-   * @Deprecated admin members are deprecated
-   */
-  void addAdminConsole(InternalDistributedMember id);
-
-  DMStats getStats();
-
-  /**
-   * Used to get the DistributionConfig so that Connection can figure out if it is configured for
-   * async comms.
-   *
-   * @since GemFire 4.2.1
-   */
-  DistributionConfig getConfig();
-
-  /**
-   * Makes note of a distribution manager that has shut down. Invokes the appropriate listeners.
-   *
-   * @param theId The id of the distribution manager starting up
-   *
-   * @see ShutdownMessage#process
-   */
-  void handleManagerDeparture(InternalDistributedMember theId, boolean crashed, String reason);
-
-  /**
-   * getThreadPool gets this distribution manager's message-processing thread pool
-   */
-  ExecutorService getThreadPool();
-
-  /**
-   * Return the high-priority message-processing executor
-   */
-  ExecutorService getHighPriorityThreadPool();
-
-  /**
-   * Return the waiting message-processing executor
-   */
-  ExecutorService getWaitingThreadPool();
-
-  /**
-   * Return the special waiting message-processing executor
-   */
-  ExecutorService getPrMetaDataCleanupThreadPool();
-
-  /**
-   * Return the executor used for function processing
-   */
-  Executor getFunctionExcecutor();
-
-  /**
-   * gets this distribution manager's message-processing executor for ordered (i.e. serialized)
-   * message processing
-   */
-  // public Executor getSerialExecutor();
-
-  void close();
-
-  /**
-   * Returns the ordered list of current DistributionManagers in oldest-to-youngest order. Added for
-   * DLockGrantor
-   */
-  List<InternalDistributedMember> getViewMembers();
-
-  /**
-   * Returns the oldest member in the given set of distribution managers. The current implementation
-   * may use n*n/2 comparisons, so use this judiciously
-   *
-   * @return the oldest member of the given collection
-   * @throws NoSuchElementException when none of the given members is actually a member of the
-   *         distributed system.
-   */
-  DistributedMember getOldestMember(Collection members) throws NoSuchElementException;
-
-  /**
-   * @return Set of Admin VM nodes
-   */
-  Set<InternalDistributedMember> getAdminMemberSet();
-
-  /** Throws ShutdownException if closeInProgress returns true. */
-  void throwIfDistributionStopped();
-
-  /** Returns count of members filling the specified role */
-  int getRoleCount(Role role);
-
-  /** Returns true if at least one member is filling the specified role */
-  boolean isRolePresent(Role role);
-
-  /** Returns a set of all roles currently in the distributed system. */
-  Set getAllRoles();
-
-  /** Returns true if id is a current member of the distributed system */
-  boolean isCurrentMember(InternalDistributedMember id);
-
-  /**
-   * Remove given member from list of members who are pending a startup reply
-   *
-   * @param m the member
-   * @param departed true if we're removing them due to membership
-   */
-  void removeUnfinishedStartup(InternalDistributedMember m, boolean departed);
-
-  void setUnfinishedStartups(Collection s);
-
-  /**
-   * Return the CancelCriterion for this DM.
-   *
-   * @return CancelCriterion for this DM
-   */
-  CancelCriterion getCancelCriterion();
-
-  /**
-   * Return the membership manager for this DM
-   *
-   * @return the membership manager
-   */
-  MembershipManager getMembershipManager();
-
-  /**
-   * Set the root cause for DM failure
-   *
-   * @param t the underlying failure
-   */
-  void setRootCause(Throwable t);
-
-  /**
-   * Return the underlying root cause for DM failure, possibly null
-   *
-   * @return the underlying root cause
-   */
-  Throwable getRootCause();
-
-  /**
-   * Return all members that are on the the this host
-   *
-   * @return set of {@link InternalDistributedMember} including this VM
-   * @since GemFire 5.9
-   */
-  Set<InternalDistributedMember> getMembersInThisZone();
-
-  /**
-   * Acquire a permit to request a GII from another member
-   */
-  void acquireGIIPermitUninterruptibly();
-
-  /**
-   * Release a permit to request a GII from another member.
-   */
-  void releaseGIIPermit();
-
-  int getDistributedSystemId();
-
-  boolean enforceUniqueZone();
-
-  Set<InternalDistributedMember> getMembersInSameZone(InternalDistributedMember acceptedMember);
-
-  boolean areInSameZone(InternalDistributedMember member1, InternalDistributedMember member2);
-
-  /**
-   * Returns true is the two members are on the same equivalent host machine based on overlapping IP
-   * addresses collected for all NICs on each member's machine.
-   *
-   * @param member1 First member
-   * @param member2 Second member
-   */
-  boolean areOnEquivalentHost(InternalDistributedMember member1, InternalDistributedMember member2);
-
-  Set<InetAddress> getEquivalents(InetAddress in);
-
-  Set<DistributedMember> getGroupMembers(String group);
-
-  /**
-   * Adds the entry in hostedLocators for a member with one or more hosted locators. The value is a
-   * collection of host[port] strings. If a bind-address was used for a locator then the form is
-   * bind-addr[port].
-   * <p>
-   * This currently only tracks stand-alone/dedicated locators, not embedded locators.
-   *
-   * @param isSharedConfigurationEnabled flag to determine if the locator has enabled shared
-   *        configuration
-   *
-   * @since GemFire 6.6.3
-   */
-  void addHostedLocators(InternalDistributedMember member, Collection<String> locators,
-      boolean isSharedConfigurationEnabled);
-
-
-  /**
-   * Gets the value in hostedLocators for a member with one or more hosted locators. The value is a
-   * collection of host[port] strings. If a bind-address was used for a locator then the form is
-   * bind-addr[port].
-   * <p>
-   * This currently only tracks stand-alone/dedicated locators, not embedded locators.
-   *
-   * @since GemFire 6.6.3
-   */
-  Collection<String> getHostedLocators(InternalDistributedMember member);
-
-  /**
-   * Gets the map of all members hosting locators. The key is the member, and the value is a
-   * collection of host[port] strings. If a bind-address was used for a locator then the form is
-   * bind-addr[port].
-   *
-   *
-   * <p>
-   * This currently only tracks stand-alone/dedicated locators, not embedded locators.
-   *
-   * @since GemFire 6.6.3
-   */
-  Map<InternalDistributedMember, Collection<String>> getAllHostedLocators();
-
-  /**
-   * Gets the map of all members hosting locators with shared configuration. The key is the member,
-   * and the value is a collection of host[port] strings. If a bind-address was used for a locator
-   * then the form is bind-addr[port].
-   * <p>
-   * This currently only tracks stand-alone/dedicated locators, not embedded locators.
-   *
-   * @since GemFire 8.0
-   */
-  Map<InternalDistributedMember, Collection<String>> getAllHostedLocatorsWithSharedConfiguration();
-
-  /**
-   * Forces use of UDP for communications in the current thread. UDP is connectionless, so no tcp/ip
-   * connections will be created or used for messaging until this setting is released with
-   * releaseUDPMessagingForCurrentThread.
-   */
-  void forceUDPMessagingForCurrentThread();
-
-  /**
-   * Releases use of UDP for all communications in the current thread, as established by
-   * forceUDPMessagingForCurrentThread.
-   */
-  void releaseUDPMessagingForCurrentThread();
-
-  /**
-   * returns the type of node
-   *
-   * @see DistributionManager#NORMAL_DM_TYPE
-   * @see DistributionManager#LONER_DM_TYPE
-   * @see DistributionManager#LOCATOR_DM_TYPE
-   * @see DistributionManager#ADMIN_ONLY_DM_TYPE
-   */
-  int getDMType();
-
-  /**
-   * The returned cache will be null if the cache does not yet exist. Note that the returned cache
-   * may be one that is already closed. Callers of GemFireCacheImpl.getInstance() should try to use
-   * this method.
-   */
-  InternalCache getCache();
-
-  /**
-   * Returns an existing non-closed cache associated with this DM. Callers of
-   * CacheFactory.getAnyInstance(), CacheFactory.getInstance(DistributedSystem) or
-   * GemFireCacheImpl.getExisting() should try to use this method.
-   *
-   * @throws CacheClosedException if a cache has not yet been associated with this DM or it has been
-   *         {@link Cache#isClosed closed}.
-   */
-  InternalCache getExistingCache();
-
-  void setCache(InternalCache instance);
-
-  HealthMonitor getHealthMonitor(InternalDistributedMember owner);
-
-  void removeHealthMonitor(InternalDistributedMember owner, int theId);
-
-  void createHealthMonitor(InternalDistributedMember owner, GemFireHealthConfig cfg);
-}
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DirectReplyProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DirectReplyProcessor.java
index 17dce55..860e79d 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DirectReplyProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DirectReplyProcessor.java
@@ -59,26 +59,28 @@ public class DirectReplyProcessor extends ReplyProcessor21 {
    * @param dm the DistributionManager to use for messaging and membership
    * @param member the member this processor wants a reply from
    */
-  public DirectReplyProcessor(DM dm, InternalDistributedMember member) {
+  public DirectReplyProcessor(DistributionManager dm, InternalDistributedMember member) {
     this(dm, Collections.singleton(member));
   }
 
   /**
    * Creates a new <code>ReplyProcessor</code> that wants replies from some number of members of a
-   * distributed system. Call this method with {@link DistributionManager#getDistributionManagerIds}
-   * if you want replies from all DMs including the one hosted in this VM.
+   * distributed system. Call this method with
+   * {@link ClusterDistributionManager#getDistributionManagerIds} if you want replies from all DMs
+   * including the one hosted in this VM.
    *
    * @param dm the DistributionManager to use for messaging and membership
    * @param initMembers the Set of members this processor wants replies from
    */
-  public DirectReplyProcessor(DM dm, Collection initMembers) {
+  public DirectReplyProcessor(DistributionManager dm, Collection initMembers) {
     this(dm, dm.getSystem(), initMembers, null);
   }
 
   /**
    * Creates a new <code>ReplyProcessor</code> that wants replies from some number of members of a
-   * distributed system. Call this method with {@link DistributionManager#getDistributionManagerIds}
-   * if you want replies from all DMs including the one hosted in this VM.
+   * distributed system. Call this method with
+   * {@link ClusterDistributionManager#getDistributionManagerIds} if you want replies from all DMs
+   * including the one hosted in this VM.
    *
    * @param system the DistributedSystem connection
    * @param initMembers the Set of members this processor wants replies from
@@ -89,8 +91,9 @@ public class DirectReplyProcessor extends ReplyProcessor21 {
 
   /**
    * Creates a new <code>ReplyProcessor</code> that wants replies from some number of members of a
-   * distributed system. Call this method with {@link DistributionManager#getDistributionManagerIds}
-   * if you want replies from all DMs including the one hosted in this VM.
+   * distributed system. Call this method with
+   * {@link ClusterDistributionManager#getDistributionManagerIds} if you want replies from all DMs
+   * including the one hosted in this VM.
    *
    * @param system the DistributedSystem connection
    * @param initMembers the Set of members this processor wants replies from
@@ -108,8 +111,8 @@ public class DirectReplyProcessor extends ReplyProcessor21 {
    * @param initMembers
    * @param cancelCriterion
    */
-  public DirectReplyProcessor(DM dm, InternalDistributedSystem system, Collection initMembers,
-      CancelCriterion cancelCriterion) {
+  public DirectReplyProcessor(DistributionManager dm, InternalDistributedSystem system,
+      Collection initMembers, CancelCriterion cancelCriterion) {
     super(dm, system, initMembers, cancelCriterion, false);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionAdvisee.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionAdvisee.java
index ae3b1a7..756ddea 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionAdvisee.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionAdvisee.java
@@ -15,9 +15,7 @@
 package org.apache.geode.distributed.internal;
 
 import org.apache.geode.CancelCriterion;
-import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionAdvisor.Profile;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
 
 /**
  * Resource which uses a {@link DistributionAdvisor}.
@@ -31,7 +29,7 @@ public interface DistributionAdvisee {
    *
    * @return the underlying <code>DistributionManager</code>
    */
-  public DM getDistributionManager();
+  public DistributionManager getDistributionManager();
 
   /**
    * @return the cancellation object for the advisee
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionAdvisor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionAdvisor.java
index e48d4df..7df44aa 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionAdvisor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionAdvisor.java
@@ -374,7 +374,7 @@ public class DistributionAdvisor {
     }
   }
 
-  public DM getDistributionManager() {
+  public DistributionManager getDistributionManager() {
     return getAdvisee().getDistributionManager();
   }
 
@@ -1611,8 +1611,8 @@ public class DistributionAdvisor {
     /**
      * Process add/remove/update of an incoming profile.
      */
-    public void processIncoming(DistributionManager dm, String adviseePath, boolean removeProfile,
-        boolean exchangeProfiles, final List<Profile> replyProfiles) {
+    public void processIncoming(ClusterDistributionManager dm, String adviseePath,
+        boolean removeProfile, boolean exchangeProfiles, final List<Profile> replyProfiles) {
       // nothing by default; just log that nothing was done
       if (logger.isDebugEnabled()) {
         logger.debug("While processing UpdateAttributes message ignored incoming profile: {}",
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
index 5a4de3d..94c9d52 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionManager.java
@@ -14,4530 +14,475 @@
  */
 package org.apache.geode.distributed.internal;
 
-import java.io.NotSerializableException;
 import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Executor;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelCriterion;
-import org.apache.geode.CancelException;
-import org.apache.geode.ForcedDisconnectException;
-import org.apache.geode.IncompatibleSystemException;
-import org.apache.geode.InternalGemFireError;
-import org.apache.geode.InternalGemFireException;
-import org.apache.geode.InvalidDeltaException;
-import org.apache.geode.SystemConnectException;
-import org.apache.geode.SystemFailure;
-import org.apache.geode.ToDataException;
 import org.apache.geode.admin.GemFireHealthConfig;
+import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.DistributedSystemDisconnectedException;
-import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.Role;
 import org.apache.geode.distributed.internal.locks.ElderState;
-import org.apache.geode.distributed.internal.membership.DistributedMembershipListener;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.membership.MemberFactory;
 import org.apache.geode.distributed.internal.membership.MembershipManager;
-import org.apache.geode.distributed.internal.membership.NetView;
-import org.apache.geode.i18n.StringId;
-import org.apache.geode.internal.Assert;
-import org.apache.geode.internal.NanoTimer;
-import org.apache.geode.internal.OSProcess;
-import org.apache.geode.internal.SetUtils;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.admin.remote.AdminConsoleDisconnectMessage;
-import org.apache.geode.internal.admin.remote.RemoteGfManagerAgent;
-import org.apache.geode.internal.admin.remote.RemoteTransportConfig;
-import org.apache.geode.internal.cache.InitialImageOperation;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.LoggingThreadGroup;
-import org.apache.geode.internal.logging.log4j.AlertAppender;
-import org.apache.geode.internal.logging.log4j.LocalizedMessage;
-import org.apache.geode.internal.logging.log4j.LogMarker;
-import org.apache.geode.internal.net.SocketCreator;
-import org.apache.geode.internal.sequencelog.MembershipLogger;
-import org.apache.geode.internal.tcp.Connection;
-import org.apache.geode.internal.tcp.ConnectionTable;
-import org.apache.geode.internal.tcp.ReenteredConnectException;
-import org.apache.geode.internal.util.concurrent.StoppableReentrantLock;
 
 /**
- * The <code>DistributionManager</code> uses a {@link MembershipManager} to distribute
- * {@link DistributionMessage messages}. It also reports on who is currently in the distributed
- * system and tracks the elder member for the distributed lock service. You may also register a
- * membership listener with the DistributionManager to receive notification of changes in
- * membership.
- *
- * <P>
- *
- * Code that wishes to send a {@link DistributionMessage} must get the
- * <code>DistributionManager</code> and invoke {@link #putOutgoing}.
- *
- * <P>
- *
- * @see DistributionMessage#process
- * @see IgnoredByManager
+ * This interface defines the services provided by any class that is a distribution manager.
  */
-public class DistributionManager implements DM {
-
-  private static final Logger logger = LogService.getLogger();
+public interface DistributionManager extends ReplySender {
 
-  private static final int STARTUP_TIMEOUT =
-      Integer.getInteger("DistributionManager.STARTUP_TIMEOUT", 15000).intValue();
-
-  public static final boolean DEBUG_NO_ACKNOWLEDGEMENTS =
-      Boolean.getBoolean("DistributionManager.DEBUG_NO_ACKNOWLEDGEMENTS");
+  boolean shutdownInProgress();
 
   /**
-   * maximum time, in milliseconds, to wait for all threads to exit
+   * Returns the current "cache time" in milliseconds since the epoch. The "cache time" takes into
+   * account skew among the local clocks on the various machines involved in the cache.
    */
-  private static final int MAX_STOP_TIME = 20000;
+  long cacheTimeMillis();
 
   /**
-   * Time to sleep, in milliseconds, while polling to see if threads have finished
+   * Returns the id of this distribution manager.
    */
-  private static final int STOP_PAUSE_TIME = 1000;
+  InternalDistributedMember getDistributionManagerId();
 
   /**
-   * Maximum number of interrupt attempts to stop a thread
+   * Get a set of all other members (both admin ones and normal).
+   *
+   * @since GemFire 5.7
    */
-  private static final int MAX_STOP_ATTEMPTS = 10;
-
-
-
-  private static final boolean SYNC_EVENTS = Boolean.getBoolean("DistributionManager.syncEvents");
+  Set getAllOtherMembers();
 
   /**
-   * Flag indicating whether to use single Serial-Executor thread or Multiple Serial-executor
-   * thread,
+   * Returns the ID in the membership view that is equal to the argument. If the ID is not in the
+   * view, the argument is returned.
    */
-  private static final boolean MULTI_SERIAL_EXECUTORS =
-      !Boolean.getBoolean("DistributionManager.singleSerialExecutor");
-
-  private static final int MAX_WAITING_THREADS =
-      Integer.getInteger("DistributionManager.MAX_WAITING_THREADS", Integer.MAX_VALUE).intValue();
-
-  private static final int MAX_PR_META_DATA_CLEANUP_THREADS =
-      Integer.getInteger("DistributionManager.MAX_PR_META_DATA_CLEANUP_THREADS", 1).intValue();
-
-  public static final int MAX_THREADS =
-      Integer.getInteger("DistributionManager.MAX_THREADS", 100).intValue();
-
-  private static final int MAX_PR_THREADS = Integer.getInteger("DistributionManager.MAX_PR_THREADS",
-      Math.max(Runtime.getRuntime().availableProcessors() * 4, 16)).intValue();
-
-  private static final int MAX_FE_THREADS = Integer.getInteger("DistributionManager.MAX_FE_THREADS",
-      Math.max(Runtime.getRuntime().availableProcessors() * 4, 16)).intValue();
-
-
-
-  private static final int INCOMING_QUEUE_LIMIT =
-      Integer.getInteger("DistributionManager.INCOMING_QUEUE_LIMIT", 80000).intValue();
-
-  /** Throttling based on the Queue byte size */
-  private static final double THROTTLE_PERCENT = (double) (Integer
-      .getInteger("DistributionManager.SERIAL_QUEUE_THROTTLE_PERCENT", 75).intValue()) / 100;
-
-  static final int SERIAL_QUEUE_BYTE_LIMIT = Integer
-      .getInteger("DistributionManager.SERIAL_QUEUE_BYTE_LIMIT", (40 * (1024 * 1024))).intValue();
-
-  static final int SERIAL_QUEUE_THROTTLE =
-      Integer.getInteger("DistributionManager.SERIAL_QUEUE_THROTTLE",
-          (int) (SERIAL_QUEUE_BYTE_LIMIT * THROTTLE_PERCENT)).intValue();
-
-  static final int TOTAL_SERIAL_QUEUE_BYTE_LIMIT =
-      Integer.getInteger("DistributionManager.TOTAL_SERIAL_QUEUE_BYTE_LIMIT", (80 * (1024 * 1024)))
-          .intValue();
-
-  static final int TOTAL_SERIAL_QUEUE_THROTTLE =
-      Integer.getInteger("DistributionManager.TOTAL_SERIAL_QUEUE_THROTTLE",
-          (int) (SERIAL_QUEUE_BYTE_LIMIT * THROTTLE_PERCENT)).intValue();
-
-  /** Throttling based on the Queue item size */
-  static final int SERIAL_QUEUE_SIZE_LIMIT =
-      Integer.getInteger("DistributionManager.SERIAL_QUEUE_SIZE_LIMIT", 20000).intValue();
-
-  static final int SERIAL_QUEUE_SIZE_THROTTLE =
-      Integer.getInteger("DistributionManager.SERIAL_QUEUE_SIZE_THROTTLE",
-          (int) (SERIAL_QUEUE_SIZE_LIMIT * THROTTLE_PERCENT)).intValue();
-
-  /** Max number of serial Queue executors, in case of multi-serial-queue executor */
-  static final int MAX_SERIAL_QUEUE_THREAD =
-      Integer.getInteger("DistributionManager.MAX_SERIAL_QUEUE_THREAD", 20).intValue();
-
-
-
-  /** The DM type for regular distribution managers */
-  public static final int NORMAL_DM_TYPE = 10;
-
-  /** The DM type for locator distribution managers */
-  public static final int LOCATOR_DM_TYPE = 11;
-
-  /** The DM type for Console (admin-only) distribution managers */
-  public static final int ADMIN_ONLY_DM_TYPE = 12;
-
-  /** The DM type for stand-alone members */
-  public static final int LONER_DM_TYPE = 13;
-
-
+  InternalDistributedMember getCanonicalId(DistributedMember id);
 
   /**
-   * @see org.apache.geode.distributed.internal.PooledDistributionMessage
+   * removes members that have older versions from the given collection, typically a Set from a
+   * distribution advisor
+   *
+   * @since GemFire 8.0
    */
-  public static final int STANDARD_EXECUTOR = 73;
+  void retainMembersWithSameOrNewerVersion(Collection<InternalDistributedMember> members,
+      Version version);
 
   /**
-   * @see org.apache.geode.distributed.internal.SerialDistributionMessage
+   * removes members that have the given version or later from the given collection, typically a Set
+   * from a distribution advisor
+   *
+   * @since GemFire 8.0
    */
-  public static final int SERIAL_EXECUTOR = 74;
+  void removeMembersWithSameOrNewerVersion(Collection<InternalDistributedMember> members,
+      Version version);
 
   /**
-   * @see org.apache.geode.distributed.internal.HighPriorityDistributionMessage
+   * Returns an unmodifiable set containing the identities of all of the known distribution
+   * managers. As of 7.0 this includes locators since they have a cache.
    */
-  public static final int HIGH_PRIORITY_EXECUTOR = 75;
-
-  // 76 not in use
+  Set getDistributionManagerIds();
 
   /**
-   * @see org.apache.geode.internal.cache.InitialImageOperation
+   * Returns an unmodifiable set containing the identities of all of the known "normal" distribution
+   * managers. This does not include locators or admin members.
    */
-  public static final int WAITING_POOL_EXECUTOR = 77;
+  Set getNormalDistributionManagerIds();
 
   /**
-   * @see org.apache.geode.internal.cache.InitialImageOperation
+   * Returns an unmodifiable set containing the identities of all of the known distribution managers
+   * including admin members.
+   *
+   * @since GemFire 5.7
    */
-  public static final int PARTITIONED_REGION_EXECUTOR = 78;
-
+  Set getDistributionManagerIdsIncludingAdmin();
 
   /**
-   * Executor for view related messages
-   *
-   * @see org.apache.geode.distributed.internal.membership.gms.messages.ViewAckMessage
+   * Returns a private-memory list containing getDistributionManagerIds() minus our id.
    */
-  public static final int VIEW_EXECUTOR = 79;
-
-
-  public static final int REGION_FUNCTION_EXECUTION_EXECUTOR = 80;
-
-
-
-  /** Is this node running an AdminDistributedSystem? */
-  private static volatile boolean isDedicatedAdminVM = false;
-
-  private static ThreadLocal<Boolean> isStartupThread = new ThreadLocal();
+  Set getOtherDistributionManagerIds();
 
   /**
-   * Identifier for function execution threads and any of their children
+   * Returns a private-memory list containing getNormalDistributionManagerIds() minus our id.
    */
-  private static final InheritableThreadLocal<Boolean> isFunctionExecutionThread =
-      new InheritableThreadLocal<Boolean>() {
-        @Override
-        protected Boolean initialValue() {
-          return Boolean.FALSE;
-        }
-      };
-
-
-
-  ///////////////////// Instance Fields //////////////////////
+  Set getOtherNormalDistributionManagerIds();
 
   /**
-   * Mutex to control access to {@link #waitingForElderChange} or {@link #elder}.
+   * Add a membership listener and return other DistribtionManagerIds as an atomic operation
    */
-  private final Object elderMonitor = new Object();
+  Set addMembershipListenerAndGetDistributionManagerIds(MembershipListener l);
 
   /**
-   * Must be read/written while holding {@link #elderMonitor}
+   * Add a membership listener for all members and return other DistribtionManagerIds as an atomic
+   * operation
    *
-   * @see #elderChangeWait()
+   * @since GemFire 5.7
    */
-  private boolean waitingForElderChange = false;
+  Set addAllMembershipListenerAndGetAllIds(MembershipListener l);
 
   /**
-   * @see DM#isAdam()
+   * Returns the identity of this <code>DistributionManager</code>
    */
-  private boolean adam = false;
+  InternalDistributedMember getId();
 
   /**
-   * This is the "elder" member of the distributed system, responsible for certain types of
-   * arbitration.
-   *
-   * Must hold {@link #elderMonitor} in order to change this.
+   * Return true if no other distribution manager was in this group when he joined.
    *
-   * @see #getElderId()
+   * @since GemFire 4.0
    */
-  protected volatile InternalDistributedMember elder = null;
-
-  /** The id of this distribution manager */
-  protected final InternalDistributedMember localAddress;
+  boolean isAdam();
 
   /**
-   * The distribution manager type of this dm; set in its constructor.
+   * Returns the identity of the oldest DM in this group.
+   *
+   * Note that this method may return null (no valid elders exist).
+   *
+   * @return the elder member, possibly null
+   * @since GemFire 4.0
    */
-  private final int dmType;
+  InternalDistributedMember getElderId();
 
   /**
-   * The <code>MembershipListener</code>s that are registered on this manager.
+   * Return true if this is the oldest DM in this group.
+   *
+   * @since GemFire 5.0
    */
-  private final ConcurrentMap<MembershipListener, Boolean> membershipListeners;
+  boolean isElder();
 
   /**
-   * The <code>MembershipListener</code>s that are registered on this manager for ALL members.
-   *
-   * @since GemFire 5.7
+   * Return true if this DM is a loner that is not part of a real distributed system.
    */
-  protected volatile Set<MembershipListener> allMembershipListeners = Collections.emptySet();
+  boolean isLoner();
 
   /**
-   * A lock to hold while adding and removing all membership listeners.
+   * Returns the elder state or null if this DM is not the elder.
+   * <p>
+   * If useTryLock is true, then it will attempt to get a try-lock and throw IllegalStateException
+   * if another thread already holds the try-lock.
    *
-   * @since GemFire 5.7
+   * @param force if true then this DM must become the elder.
+   * @param useTryLock if true then a try-lock will be used
+   * @throws IllegalStateException if elder try lock fails
+   * @since GemFire 4.0
    */
-  private final Object allMembershipListenersLock = new MembershipListenersLock();
-
-  /** A queue of MemberEvent instances */
-  private final BlockingQueue<MemberEvent> membershipEventQueue = new LinkedBlockingQueue();
-
-  /** Used to invoke registered membership listeners in the background. */
-  private Thread memberEventThread;
-
-
-  /** A brief description of this DistributionManager */
-  protected final String description;
-
-  /** Statistics about distribution */
-  protected DistributionStats stats;
-
-  /** Did an exception occur in one of the DM threads? */
-  private boolean exceptionInThreads;
-
-  private volatile boolean shutdownMsgSent = false;
-
-  /** Set to true when this manager is being shutdown */
-  private volatile boolean closeInProgress = false;
-
-  private volatile boolean receivedStartupResponse = false;
+  ElderState getElderState(boolean force, boolean useTryLock);
 
-  private volatile String rejectionMessage = null;
-
-  private MembershipManager membershipManager;
-
-  /**
-   * The (non-admin-only) members of the distributed system. This is a map of memberid->memberid for
-   * fast access to canonical ID references. All accesses to this field must be synchronized on
-   * {@link #membersLock}.
-   */
-  private Map<InternalDistributedMember, InternalDistributedMember> members =
-      Collections.emptyMap();
   /**
-   * All (admin and non-admin) members of the distributed system. All accesses to this field must be
-   * synchronized on {@link #membersLock}.
-   */
-  private Set<InternalDistributedMember> membersAndAdmin = Collections.emptySet();
-  /**
-   * Map of all locator members of the distributed system. The value is a collection of locator
-   * strings that are hosted in that member. All accesses to this field must be synchronized on
-   * {@link #membersLock}.
+   * Returns the membership port of the underlying distribution manager used for communication.
+   *
+   * @since GemFire 3.0
    */
-  private Map<InternalDistributedMember, Collection<String>> hostedLocatorsAll =
-      Collections.emptyMap();
+  long getMembershipPort();
 
   /**
-   * Map of all locator members of the distributed system which have the shared configuration. The
-   * value is a collection of locator strings that are hosted in that member. All accesses to this
-   * field must be synchronized on {@link #membersLock}.
+   * Sends a message
+   *
+   * @param msg
+   * @return recipients who did not receive the message
    */
-  private Map<InternalDistributedMember, Collection<String>> hostedLocatorsWithSharedConfiguration =
-      Collections.emptyMap();
+  Set putOutgoing(DistributionMessage msg);
 
   /**
-   * The lock held while accessing the field references to the following:<br>
-   * 1) {@link #members}<br>
-   * 2) {@link #membersAndAdmin}<br>
-   * 3) {@link #hostedLocatorsAll}<br>
-   * 4) {@link #hostedLocatorsWithSharedConfiguration}<br>
+   * Returns the distributed system to which this distribution manager is connected.
    */
-  private final Object membersLock = new MembersLock();
+  InternalDistributedSystem getSystem();
 
   /**
-   * The lock held while writing {@link #adminConsoles}.
-   */
-  private final Object adminConsolesLock = new Object();
-  /**
-   * The ids of all known admin consoles Uses Copy on Write. Writers must sync on adminConsolesLock.
-   * Readers don't need to sync.
+   * Adds a <code>MembershipListener</code> to this distribution manager.
    */
-  private volatile Set<InternalDistributedMember> adminConsoles = Collections.emptySet();
-
-  /** The group of distribution manager threads */
-  protected LoggingThreadGroup threadGroup;
-
-  /** Message processing thread pool */
-  private ThreadPoolExecutor threadPool;
+  void addMembershipListener(MembershipListener l);
 
   /**
-   * High Priority processing thread pool, used for initializing messages such as UpdateAttributes
-   * and CreateRegion messages
+   * Removes a <code>MembershipListener</code> from this distribution manager.
+   *
+   * @throws IllegalArgumentException <code>l</code> was not registered on this distribution manager
    */
-  private ThreadPoolExecutor highPriorityPool;
+  void removeMembershipListener(MembershipListener l);
 
   /**
-   * Waiting Pool, used for messages that may have to wait on something. Use this separate pool with
-   * an unbounded queue so that waiting runnables don't get in the way of other processing threads.
-   * Used for threads that will most likely have to wait for a region to be finished initializing
-   * before it can proceed
+   * Removes a <code>MembershipListener</code> listening for all members from this distribution
+   * manager.
+   *
+   * @throws IllegalArgumentException <code>l</code> was not registered on this distribution manager
+   * @since GemFire 5.7
    */
-  private ThreadPoolExecutor waitingPool;
-
-  private ThreadPoolExecutor prMetaDataCleanupThreadPool;
+  void removeAllMembershipListener(MembershipListener l);
 
   /**
-   * Thread used to decouple {@link org.apache.geode.internal.cache.partitioned.PartitionMessage}s
-   * from {@link org.apache.geode.internal.cache.DistributedCacheOperation}s </b>
+   * Makes note of a new administration console (admin-only member).
    *
-   * @see #SERIAL_EXECUTOR
+   * @Deprecated admin members are deprecated
    */
-  private ThreadPoolExecutor partitionedRegionThread;
-  private ThreadPoolExecutor partitionedRegionPool;
+  void addAdminConsole(InternalDistributedMember id);
 
-  /** Function Execution executors */
-  private ThreadPoolExecutor functionExecutionThread;
-  private ThreadPoolExecutor functionExecutionPool;
-
-  /** Message processing executor for serial, ordered, messages. */
-  private ThreadPoolExecutor serialThread;
+  DMStats getStats();
 
   /**
-   * Message processing executor for view messages
+   * Used to get the DistributionConfig so that Connection can figure out if it is configured for
+   * async comms.
    *
-   * @see org.apache.geode.distributed.internal.membership.gms.messages.ViewAckMessage
-   */
-  private ThreadPoolExecutor viewThread;
-
-  /**
-   * If using a throttling queue for the serialThread, we cache the queue here so we can see if
-   * delivery would block
+   * @since GemFire 4.2.1
    */
-  private ThrottlingMemLinkedQueueWithDMStats serialQueue;
-
-  /** a map keyed on InternalDistributedMember, to direct channels to other systems */
-  // protected final Map channelMap = CFactory.createCM();
-
-  private volatile boolean readyForMessages = false;
+  DistributionConfig getConfig();
 
   /**
-   * Set to true once this DM is ready to send messages. Note that it is always ready to send the
-   * startup message.
+   * Makes note of a distribution manager that has shut down. Invokes the appropriate listeners.
+   *
+   * @param theId The id of the distribution manager starting up
+   *
+   * @see ShutdownMessage#process
    */
-  private volatile boolean readyToSendMsgs = false;
-  private final Object readyToSendMsgsLock = new Object();
-
-  /** Is this distribution manager closed? */
-  private volatile boolean closed = false;
+  void handleManagerDeparture(InternalDistributedMember theId, boolean crashed, String reason);
 
   /**
-   * The distributed system to which this distribution manager is connected.
+   * getThreadPool gets this distribution manager's message-processing thread pool
    */
-  private InternalDistributedSystem system;
-
-  /** The remote transport configuration for this dm */
-  private RemoteTransportConfig transport;
+  ExecutorService getThreadPool();
 
   /**
-   * The administration agent associated with this distribution manager.
+   * Return the high-priority message-processing executor
    */
-  private volatile RemoteGfManagerAgent agent;
-
-  private SerialQueuedExecutorPool serialQueuedExecutorPool;
+  ExecutorService getHighPriorityThreadPool();
 
   /**
-   * TODO why does the distribution manager arbitrate GII operations? That should be a Cache
-   * function
+   * Return the waiting message-processing executor
    */
-  private final Semaphore parallelGIIs = new Semaphore(InitialImageOperation.MAX_PARALLEL_GIIS);
+  ExecutorService getWaitingThreadPool();
 
   /**
-   * Map of InetAddress to HashSets of InetAddress, to define equivalences between network interface
-   * cards and hosts.
+   * Return the special waiting message-processing executor
    */
-  private final HashMap<InetAddress, Set<InetAddress>> equivalentHosts = new HashMap<>();
-
-  private int distributedSystemId = DistributionConfig.DEFAULT_DISTRIBUTED_SYSTEM_ID;
-
-
-  private final Map<InternalDistributedMember, String> redundancyZones =
-      Collections.synchronizedMap(new HashMap<InternalDistributedMember, String>());
-
-  private boolean enforceUniqueZone = false;
+  ExecutorService getPrMetaDataCleanupThreadPool();
 
   /**
-   * root cause of forcibly shutting down the distribution manager
+   * Return the executor used for function processing
    */
-  volatile Throwable rootCause = null;
+  Executor getFunctionExcecutor();
 
   /**
-   * @see #closeInProgress
+   * gets this distribution manager's message-processing executor for ordered (i.e. serialized)
+   * message processing
    */
-  private final Object shutdownMutex = new Object();
-
-
+  // public Executor getSerialExecutor();
 
-  ////////////////////// Static Methods //////////////////////
+  void close();
 
   /**
-   * Is the current thread used for executing Functions?
+   * Returns the ordered list of current DistributionManagers in oldest-to-youngest order. Added for
+   * DLockGrantor
    */
-  public static Boolean isFunctionExecutionThread() {
-    return isFunctionExecutionThread.get();
-  }
+  List<InternalDistributedMember> getViewMembers();
 
   /**
-   * Creates a new distribution manager and discovers the other members of the distributed system.
-   * Note that it does not check to see whether or not this VM already has a distribution manager.
+   * Returns the oldest member in the given set of distribution managers. The current implementation
+   * may use n*n/2 comparisons, so use this judiciously
    *
-   * @param system The distributed system to which this distribution manager will send messages.
+   * @return the oldest member of the given collection
+   * @throws NoSuchElementException when none of the given members is actually a member of the
+   *         distributed system.
    */
-  static DistributionManager create(InternalDistributedSystem system) {
-
-    DistributionManager distributionManager = null;
-    boolean beforeJoined = true;
-
-    try {
-
-      int vmKind;
-
-      if (Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE)) {
-        // if this DM is starting for a locator, set it to be a locator DM
-        vmKind = LOCATOR_DM_TYPE;
-
-      } else if (isDedicatedAdminVM()) {
-        vmKind = ADMIN_ONLY_DM_TYPE;
-
-      } else {
-        vmKind = NORMAL_DM_TYPE;
-      }
-
-      RemoteTransportConfig transport = new RemoteTransportConfig(system.getConfig(), vmKind);
-      transport.setIsReconnectingDS(system.isReconnectingDS());
-      transport.setOldDSMembershipInfo(system.oldDSMembershipInfo());
-
-      long start = System.currentTimeMillis();
-
-      distributionManager = new DistributionManager(system, transport);
-      distributionManager.assertDistributionManagerType();
-
-      beforeJoined = false; // we have now joined the system
-
-      {
-        InternalDistributedMember id = distributionManager.getDistributionManagerId();
-        if (!"".equals(id.getName())) {
-          for (InternalDistributedMember m : (List<InternalDistributedMember>) distributionManager
-              .getViewMembers()) {
-            if (m.equals(id)) {
-              // I'm counting on the members returned by getViewMembers being ordered such that
-              // members that joined before us will precede us AND members that join after us
-              // will succeed us.
-              // SO once we find ourself break out of this loop.
-              break;
-            }
-            if (id.getName().equals(m.getName())) {
-              if (distributionManager.getMembershipManager().verifyMember(m,
-                  "member is using the name of " + id)) {
-                throw new IncompatibleSystemException("Member " + id
-                    + " could not join this distributed system because the existing member " + m
-                    + " used the same name. Set the \"name\" gemfire property to a unique value.");
-              }
-            }
-          }
-        }
-        distributionManager.addNewMember(id); // add ourselves
-        distributionManager.selectElder(); // ShutdownException could be thrown here
-      }
-
-      // Send out a StartupMessage to the other members.
-      StartupOperation op = new StartupOperation(distributionManager, transport);
-
-      try {
-        if (!distributionManager.sendStartupMessage(op)) {
-          // We'll we didn't hear back from anyone else. We assume that
-          // we're the first one.
-          if (distributionManager.getOtherDistributionManagerIds().size() == 0) {
-            logger.info(LocalizedMessage.create(
-                LocalizedStrings.DistributionManager_DIDNT_HEAR_BACK_FROM_ANY_OTHER_SYSTEM_I_AM_THE_FIRST_ONE));
-          } else if (transport.isMcastEnabled()) {
-            // perform a multicast ping test
-            if (!distributionManager.testMulticast()) {
-              logger.warn(LocalizedMessage.create(
-                  LocalizedStrings.DistributionManager_RECEIVED_NO_STARTUP_RESPONSES_BUT_OTHER_MEMBERS_EXIST_MULTICAST_IS_NOT_RESPONSIVE));
-            }
-          }
-        }
-      } catch (InterruptedException ex) {
-        Thread.currentThread().interrupt();
-        // This is ALWAYS bad; don't consult a CancelCriterion.
-        throw new InternalGemFireException(
-            LocalizedStrings.DistributionManager_INTERRUPTED_WHILE_WAITING_FOR_FIRST_STARTUPRESPONSEMESSAGE
-                .toLocalizedString(),
-            ex);
-      } catch (IncompatibleSystemException ex) {
-        logger.fatal(ex.getMessage(), ex);
-        throw ex;
-      } finally {
-        distributionManager.readyToSendMsgs();
-      }
-
-      if (logger.isInfoEnabled()) {
-        long delta = System.currentTimeMillis() - start;
-        Object[] logArgs = new Object[] {distributionManager.getDistributionManagerId(), transport,
-            Integer.valueOf(distributionManager.getOtherDistributionManagerIds().size()),
-            distributionManager.getOtherDistributionManagerIds(),
-            (logger.isInfoEnabled(LogMarker.DM) ? " (VERBOSE, took " + delta + " ms)" : ""),
-            ((distributionManager.getDMType() == ADMIN_ONLY_DM_TYPE) ? " (admin only)"
-                : (distributionManager.getDMType() == LOCATOR_DM_TYPE) ? " (locator)" : "")};
-        logger.info(LogMarker.DM,
-            LocalizedMessage.create(
-                LocalizedStrings.DistributionManager_DISTRIBUTIONMANAGER_0_STARTED_ON_1_THERE_WERE_2_OTHER_DMS_3_4_5,
-                logArgs));
-
-        MembershipLogger.logStartup(distributionManager.getDistributionManagerId());
-      }
-      return distributionManager;
-    } catch (RuntimeException r) {
-      if (distributionManager != null) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("cleaning up incompletely started DistributionManager due to exception", r);
-        }
-        distributionManager.uncleanShutdown(beforeJoined);
-      }
-      throw r;
-    }
-  }
-
-  /////////////////////// Constructors ///////////////////////
+  DistributedMember getOldestMember(Collection members) throws NoSuchElementException;
 
   /**
-   * Creates a new <code>DistributionManager</code> by initializing itself, creating the membership
-   * manager and executors
-   *
-   * @param transport The configuration for the communications transport
-   *
+   * @return Set of Admin VM nodes
    */
-  private DistributionManager(RemoteTransportConfig transport, InternalDistributedSystem system) {
-
-    this.dmType = transport.getVmKind();
-    this.system = system;
-    this.elderLock = new StoppableReentrantLock(stopper);
-    this.transport = transport;
-
-    this.membershipListeners = new ConcurrentHashMap();
-    this.distributedSystemId = system.getConfig().getDistributedSystemId();
-    {
-      long statId = OSProcess.getId();
-      /*
-       * deadcoded since we don't know the channel id yet. if (statId == 0 || statId == -1) { statId
-       * = getMembershipPort(); }
-       */
-      this.stats = new DistributionStats(system, statId);
-      DistributionStats.enableClockStats = system.getConfig().getEnableTimeStatistics();
-    }
-
-    this.exceptionInThreads = false;
-
-    // Start the processing threads
-    final LoggingThreadGroup group =
-        LoggingThreadGroup.createThreadGroup("DistributionManager Threads", logger);
-    this.threadGroup = group;
-
-    boolean finishedConstructor = false;
-    try {
-
-      if (MULTI_SERIAL_EXECUTORS) {
-        if (logger.isInfoEnabled(LogMarker.DM)) {
-          logger.info(LogMarker.DM, "Serial Queue info :" + " THROTTLE_PERCENT: " + THROTTLE_PERCENT
-              + " SERIAL_QUEUE_BYTE_LIMIT :" + SERIAL_QUEUE_BYTE_LIMIT + " SERIAL_QUEUE_THROTTLE :"
-              + SERIAL_QUEUE_THROTTLE + " TOTAL_SERIAL_QUEUE_BYTE_LIMIT :"
-              + TOTAL_SERIAL_QUEUE_BYTE_LIMIT + " TOTAL_SERIAL_QUEUE_THROTTLE :"
-              + TOTAL_SERIAL_QUEUE_THROTTLE + " SERIAL_QUEUE_SIZE_LIMIT :" + SERIAL_QUEUE_SIZE_LIMIT
-              + " SERIAL_QUEUE_SIZE_THROTTLE :" + SERIAL_QUEUE_SIZE_THROTTLE);
-        }
-        // when TCP/IP is disabled we can't throttle the serial queue or we run the risk of
-        // distributed deadlock when we block the UDP reader thread
-        boolean throttlingDisabled = system.getConfig().getDisableTcp();
-        this.serialQueuedExecutorPool =
-            new SerialQueuedExecutorPool(this.threadGroup, this.stats, throttlingDisabled);
-      }
-
-      {
-        BlockingQueue poolQueue;
-        if (SERIAL_QUEUE_BYTE_LIMIT == 0) {
-          poolQueue = new OverflowQueueWithDMStats(this.stats.getSerialQueueHelper());
-        } else {
-          this.serialQueue = new ThrottlingMemLinkedQueueWithDMStats(TOTAL_SERIAL_QUEUE_BYTE_LIMIT,
-              TOTAL_SERIAL_QUEUE_THROTTLE, SERIAL_QUEUE_SIZE_LIMIT, SERIAL_QUEUE_SIZE_THROTTLE,
-              this.stats.getSerialQueueHelper());
-          poolQueue = this.serialQueue;
-        }
-        ThreadFactory tf = new ThreadFactory() {
-          public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incSerialThreadStarts();
-            final Runnable r = new Runnable() {
-              public void run() {
-                DistributionManager.this.stats.incNumSerialThreads(1);
-                try {
-                  ConnectionTable.threadWantsSharedResources();
-                  Connection.makeReaderThread();
-                  runUntilShutdown(command);
-                  // command.run();
-                } finally {
-                  ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incNumSerialThreads(-1);
-                }
-              }
-            };
-            Thread thread = new Thread(group, r,
-                LocalizedStrings.DistributionManager_SERIAL_MESSAGE_PROCESSOR.toLocalizedString());
-            thread.setDaemon(true);
-            return thread;
-          }
-        };
-        SerialQueuedExecutorWithDMStats executor = new SerialQueuedExecutorWithDMStats(poolQueue,
-            this.stats.getSerialProcessorHelper(), tf);
-        this.serialThread = executor;
-      }
-      {
-        BlockingQueue q = new LinkedBlockingQueue();
-        ThreadFactory tf = new ThreadFactory() {
-          public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incViewThreadStarts();
-            final Runnable r = new Runnable() {
-              public void run() {
-                DistributionManager.this.stats.incNumViewThreads(1);
-                try {
-                  ConnectionTable.threadWantsSharedResources();
-                  Connection.makeReaderThread();
-                  runUntilShutdown(command);
-                } finally {
-                  ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incNumViewThreads(-1);
-                }
-              }
-            };
-            Thread thread = new Thread(group, r,
-                LocalizedStrings.DistributionManager_VIEW_MESSAGE_PROCESSOR.toLocalizedString());
-            thread.setDaemon(true);
-            return thread;
-          }
-        };
-        this.viewThread =
-            new SerialQueuedExecutorWithDMStats(q, this.stats.getViewProcessorHelper(), tf);
-      }
-
-      {
-        BlockingQueue poolQueue;
-        if (INCOMING_QUEUE_LIMIT == 0) {
-          poolQueue = new OverflowQueueWithDMStats(this.stats.getOverflowQueueHelper());
-        } else {
-          poolQueue = new OverflowQueueWithDMStats(INCOMING_QUEUE_LIMIT,
-              this.stats.getOverflowQueueHelper());
-        }
-        ThreadFactory tf = new ThreadFactory() {
-          private int next = 0;
-
-          public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incProcessingThreadStarts();
-            final Runnable r = new Runnable() {
-              public void run() {
-                DistributionManager.this.stats.incNumProcessingThreads(1);
-                try {
-                  ConnectionTable.threadWantsSharedResources();
-                  Connection.makeReaderThread();
-                  runUntilShutdown(command);
-                } finally {
-                  ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incNumProcessingThreads(-1);
-                }
-              }
-            };
-            Thread thread = new Thread(group, r,
-                LocalizedStrings.DistributionManager_POOLED_MESSAGE_PROCESSOR.toLocalizedString()
-                    + (next++));
-            thread.setDaemon(true);
-            return thread;
-          }
-        };
-        ThreadPoolExecutor pool = new PooledExecutorWithDMStats(poolQueue, MAX_THREADS,
-            this.stats.getNormalPoolHelper(), tf);
-        this.threadPool = pool;
-      }
-
-
-      {
-        BlockingQueue poolQueue;
-        if (INCOMING_QUEUE_LIMIT == 0) {
-          poolQueue = new OverflowQueueWithDMStats(this.stats.getHighPriorityQueueHelper());
-        } else {
-          poolQueue = new OverflowQueueWithDMStats(INCOMING_QUEUE_LIMIT,
-              this.stats.getHighPriorityQueueHelper());
-        }
-        ThreadFactory tf = new ThreadFactory() {
-          private int next = 0;
-
-          public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incHighPriorityThreadStarts();
-            final Runnable r = new Runnable() {
-              public void run() {
-                DistributionManager.this.stats.incHighPriorityThreads(1);
-                try {
-                  ConnectionTable.threadWantsSharedResources();
-                  Connection.makeReaderThread();
-                  runUntilShutdown(command);
-                } finally {
-                  ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incHighPriorityThreads(-1);
-                }
-              }
-            };
-            Thread thread = new Thread(group, r,
-                LocalizedStrings.DistributionManager_POOLED_HIGH_PRIORITY_MESSAGE_PROCESSOR
-                    .toLocalizedString() + (next++));
-            thread.setDaemon(true);
-            return thread;
-          }
-        };
-        this.highPriorityPool = new PooledExecutorWithDMStats(poolQueue, MAX_THREADS,
-            this.stats.getHighPriorityPoolHelper(), tf);
-      }
-
-
-      {
-        ThreadFactory tf = new ThreadFactory() {
-          private int next = 0;
-
-          public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incWaitingThreadStarts();
-            final Runnable r = new Runnable() {
-              public void run() {
-                DistributionManager.this.stats.incWaitingThreads(1);
-                try {
-                  ConnectionTable.threadWantsSharedResources();
-                  Connection.makeReaderThread();
-                  runUntilShutdown(command);
-                } finally {
-                  ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incWaitingThreads(-1);
-                }
-              }
-            };
-            Thread thread = new Thread(group, r,
-                LocalizedStrings.DistributionManager_POOLED_WAITING_MESSAGE_PROCESSOR
-                    .toLocalizedString() + (next++));
-            thread.setDaemon(true);
-            return thread;
-          }
-        };
-        BlockingQueue poolQueue;
-        if (MAX_WAITING_THREADS == Integer.MAX_VALUE) {
-          // no need for a queue since we have infinite threads
-          poolQueue = new SynchronousQueue();
-        } else {
-          poolQueue = new OverflowQueueWithDMStats(this.stats.getWaitingQueueHelper());
-        }
-        this.waitingPool = new PooledExecutorWithDMStats(poolQueue, MAX_WAITING_THREADS,
-            this.stats.getWaitingPoolHelper(), tf);
-      }
-
-      {
-        ThreadFactory tf = new ThreadFactory() {
-          private int next = 0;
-
-          public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incWaitingThreadStarts();// will it be ok?
-            final Runnable r = new Runnable() {
-              public void run() {
-                DistributionManager.this.stats.incWaitingThreads(1);// will it be ok
-                try {
-                  ConnectionTable.threadWantsSharedResources();
-                  Connection.makeReaderThread();
-                  runUntilShutdown(command);
-                } finally {
-                  ConnectionTable.releaseThreadsSockets();
-                  DistributionManager.this.stats.incWaitingThreads(-1);
-                }
-              }
-            };
-            Thread thread = new Thread(group, r,
-                LocalizedStrings.DistributionManager_PR_META_DATA_CLEANUP_MESSAGE_PROCESSOR
-                    .toLocalizedString() + (next++));
-            thread.setDaemon(true);
-            return thread;
-          }
-        };
-        BlockingQueue poolQueue;
-        poolQueue = new OverflowQueueWithDMStats(this.stats.getWaitingQueueHelper());
-        this.prMetaDataCleanupThreadPool = new PooledExecutorWithDMStats(poolQueue,
-            MAX_PR_META_DATA_CLEANUP_THREADS, this.stats.getWaitingPoolHelper(), tf);
-      }
-
-      {
-        BlockingQueue poolQueue;
-        if (INCOMING_QUEUE_LIMIT == 0) {
-          poolQueue = new OverflowQueueWithDMStats(this.stats.getPartitionedRegionQueueHelper());
-        } else {
-          poolQueue = new OverflowQueueWithDMStats(INCOMING_QUEUE_LIMIT,
-              this.stats.getPartitionedRegionQueueHelper());
-        }
-        ThreadFactory tf = new ThreadFactory() {
-          private int next = 0;
-
-          public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incPartitionedRegionThreadStarts();
-            final Runnable r = new Runnable() {
-              public void run() {
-                stats.incPartitionedRegionThreads(1);
-                try {
-                  ConnectionTable.threadWantsSharedResources();
-                  Connection.makeReaderThread();
-                  runUntilShutdown(command);
-                } finally {
-                  ConnectionTable.releaseThreadsSockets();
-                  stats.incPartitionedRegionThreads(-1);
-                }
-              }
-            };
-            Thread thread = new Thread(group, r, "PartitionedRegion Message Processor" + (next++));
-            thread.setDaemon(true);
-            return thread;
-          }
-        };
-        if (MAX_PR_THREADS > 1) {
-          this.partitionedRegionPool = new PooledExecutorWithDMStats(poolQueue, MAX_PR_THREADS,
-              this.stats.getPartitionedRegionPoolHelper(), tf);
-        } else {
-          SerialQueuedExecutorWithDMStats executor = new SerialQueuedExecutorWithDMStats(poolQueue,
-              this.stats.getPartitionedRegionPoolHelper(), tf);
-          this.partitionedRegionThread = executor;
-        }
-
-      }
-
-      {
-        BlockingQueue poolQueue;
-        if (INCOMING_QUEUE_LIMIT == 0) {
-          poolQueue = new OverflowQueueWithDMStats(this.stats.getFunctionExecutionQueueHelper());
-        } else {
-          poolQueue = new OverflowQueueWithDMStats(INCOMING_QUEUE_LIMIT,
-              this.stats.getFunctionExecutionQueueHelper());
-        }
-        ThreadFactory tf = new ThreadFactory() {
-          private int next = 0;
-
-          public Thread newThread(final Runnable command) {
-            DistributionManager.this.stats.incFunctionExecutionThreadStarts();
-            final Runnable r = new Runnable() {
-              public void run() {
-                stats.incFunctionExecutionThreads(1);
-                isFunctionExecutionThread.set(Boolean.TRUE);
-                try {
-                  ConnectionTable.threadWantsSharedResources();
-                  Connection.makeReaderThread();
-                  runUntilShutdown(command);
-                } finally {
-                  ConnectionTable.releaseThreadsSockets();
-                  stats.incFunctionExecutionThreads(-1);
-                }
-              }
-            };
-            Thread thread = new Thread(group, r, "Function Execution Processor" + (next++));
-            thread.setDaemon(true);
-            return thread;
-          }
-        };
-
-        if (MAX_FE_THREADS > 1) {
-          this.functionExecutionPool =
-              new FunctionExecutionPooledExecutor(poolQueue, MAX_FE_THREADS,
-                  this.stats.getFunctionExecutionPoolHelper(), tf, true /* for fn exec */);
-        } else {
-          SerialQueuedExecutorWithDMStats executor = new SerialQueuedExecutorWithDMStats(poolQueue,
-              this.stats.getFunctionExecutionPoolHelper(), tf);
-          this.functionExecutionThread = executor;
-        }
-
-      }
-
-      if (!SYNC_EVENTS) {
-        this.memberEventThread =
-            new Thread(group, new MemberEventInvoker(), "DM-MemberEventInvoker");
-        this.memberEventThread.setDaemon(true);
-      }
-
-      StringBuffer sb = new StringBuffer(" (took ");
+  Set<InternalDistributedMember> getAdminMemberSet();
 
-      // connect to the cluster
-      long start = System.currentTimeMillis();
+  /** Throws ShutdownException if closeInProgress returns true. */
+  void throwIfDistributionStopped();
 
-      DMListener l = new DMListener(this);
-      membershipManager = MemberFactory.newMembershipManager(l, system.getConfig(), transport,
-          stats, system.getSecurityService());
-
-      sb.append(System.currentTimeMillis() - start);
-
-      this.localAddress = membershipManager.getLocalMember();
-
-      membershipManager.postConnect();
-
-      sb.append(" ms)");
+  /** Returns count of members filling the specified role */
+  int getRoleCount(Role role);
 
-      logger.info(LocalizedMessage.create(
-          LocalizedStrings.DistributionManager_STARTING_DISTRIBUTIONMANAGER_0_1, new Object[] {
-              this.localAddress, (logger.isInfoEnabled(LogMarker.DM) ? sb.toString() : "")}));
+  /** Returns true if at least one member is filling the specified role */
+  boolean isRolePresent(Role role);
 
-      this.description = "Distribution manager on " + this.localAddress + " started at "
-          + (new Date(System.currentTimeMillis())).toString();
+  /** Returns a set of all roles currently in the distributed system. */
+  Set getAllRoles();
 
-      finishedConstructor = true;
-    } finally {
-      if (!finishedConstructor) {
-        askThreadsToStop(); // fix for bug 42039
-      }
-    }
-  }
+  /** Returns true if id is a current member of the distributed system */
+  boolean isCurrentMember(InternalDistributedMember id);
 
   /**
-   * Creates a new distribution manager
+   * Remove given member from list of members who are pending a startup reply
    *
-   * @param system The distributed system to which this distribution manager will send messages.
+   * @param m the member
+   * @param departed true if we're removing them due to membership
    */
-  private DistributionManager(InternalDistributedSystem system, RemoteTransportConfig transport) {
-    this(transport, system);
-
-    boolean finishedConstructor = false;
-    try {
+  void removeUnfinishedStartup(InternalDistributedMember m, boolean departed);
 
-      setIsStartupThread(Boolean.TRUE);
-
-      startThreads();
-
-      // Allow events to start being processed.
-      membershipManager.startEventProcessing();
-      for (;;) {
-        this.getCancelCriterion().checkCancelInProgress(null);
-        boolean interrupted = Thread.interrupted();
-        try {
-          membershipManager.waitForEventProcessing();
-          break;
-        } catch (InterruptedException e) {
-          interrupted = true;
-        } finally {
-          if (interrupted) {
-            Thread.currentThread().interrupt();
-          }
-        }
-      }
-
-      finishedConstructor = true;
-    } finally {
-      if (!finishedConstructor) {
-        askThreadsToStop(); // fix for bug 42039
-      }
-    }
-  }
+  void setUnfinishedStartups(Collection s);
 
   /**
-   * Is this VM dedicated to administration (like a GUI console or a JMX agent)? If so, then it
-   * creates {@link #ADMIN_ONLY_DM_TYPE} type distribution managers.
+   * Return the CancelCriterion for this DM.
    *
-   * @since GemFire 4.0
+   * @return CancelCriterion for this DM
    */
-  public static boolean isDedicatedAdminVM() {
-    return isDedicatedAdminVM;
-  }
-
-  public static void setIsDedicatedAdminVM(boolean isDedicatedAdminVM) {
-    DistributionManager.isDedicatedAdminVM = isDedicatedAdminVM;
-  }
-
-  public static Boolean getIsStartupThread() {
-    return isStartupThread.get();
-  }
-
-  public static void setIsStartupThread(Boolean isStartup) {
-    DistributionManager.isStartupThread.set(isStartup);
-  }
-
-  //////////////////// Instance Methods /////////////////////
-
-  private void runUntilShutdown(Runnable r) {
-    try {
-      r.run();
-    } catch (CancelException e) {
-      if (logger.isTraceEnabled()) {
-        logger.trace("Caught shutdown exception", e);
-      }
-    } catch (VirtualMachineError err) {
-      SystemFailure.initiateFailure(err);
-      // If this ever returns, rethrow the error. We're poisoned
-      // now, so don't let this thread continue.
-      throw err;
-    } catch (Throwable t) {
-      SystemFailure.checkFailure();
-      if (isCloseInProgress()) {
-        logger.debug("Caught unusual exception during shutdown: {}", t.getMessage(), t);
-      } else {
-        logger.warn(LocalizedMessage
-            .create(LocalizedStrings.DistributionManager_TASK_FAILED_WITH_EXCEPTION), t);
-      }
-    }
-  }
+  CancelCriterion getCancelCriterion();
 
   /**
-   * Returns true if the two members are on the same equivalent host based on overlapping IP
-   * addresses collected for all NICs during exchange of startup messages.
+   * Return the membership manager for this DM
    *
-   * @param member1 First member
-   * @param member2 Second member
+   * @return the membership manager
    */
-  @Override
-  public boolean areOnEquivalentHost(InternalDistributedMember member1,
-      InternalDistributedMember member2) {
-    Set<InetAddress> equivalents1 = getEquivalents(member1.getInetAddress());
-    return equivalents1.contains(member2.getInetAddress());
-  }
+  MembershipManager getMembershipManager();
 
   /**
-   * Set the host equivalencies for a given host. This overrides any previous information in the
-   * tables.
+   * Set the root cause for DM failure
    *
-   * @param equivs list of InetAddress's that all point at same host
+   * @param t the underlying failure
    */
-  void setEquivalentHosts(Set<InetAddress> equivs) {
-    Iterator<InetAddress> it = equivs.iterator();
-    synchronized (equivalentHosts) {
-      while (it.hasNext()) {
-        equivalentHosts.put(it.next(), Collections.unmodifiableSet(equivs));
-      }
-    }
-  }
-
+  void setRootCause(Throwable t);
 
   /**
-   * Return all of the InetAddress's that are equivalent to the given one (same host)
+   * Return the underlying root cause for DM failure, possibly null
    *
-   * @param in host to match up
-   * @return all the addresses thus equivalent
+   * @return the underlying root cause
    */
-  @Override
-  public Set<InetAddress> getEquivalents(InetAddress in) {
-    Set<InetAddress> result;
-    synchronized (equivalentHosts) {
-      result = equivalentHosts.get(in);
-    }
-    // DS 11/25/08 - It appears that when using VPN, the distributed member
-    // id is the vpn address, but that doesn't show up in the equivalents.
-    if (result == null) {
-      result = Collections.singleton(in);
-    }
-    return result;
-  }
-
-  void setRedundancyZone(InternalDistributedMember member, String redundancyZone) {
-    if (redundancyZone != null && !redundancyZone.equals("")) {
-      this.redundancyZones.put(member, redundancyZone);
-    }
-    if (member != getDistributionManagerId()) {
-      String relationship = areInSameZone(getDistributionManagerId(), member) ? "" : "not ";
-      Object[] logArgs = new Object[] {member, relationship};
-      logger.info(LocalizedMessage.create(
-          LocalizedStrings.DistributionManager_DISTRIBUTIONMANAGER_MEMBER_0_IS_1_EQUIVALENT,
-          logArgs));
-    }
-  }
+  Throwable getRootCause();
 
   /**
-   * Set the flag indicating that we should enforce unique zones. If we are already enforcing unique
-   * zones, keep it that way.
+   * Return all members that are on the the this host
+   *
+   * @return set of {@link InternalDistributedMember} including this VM
+   * @since GemFire 5.9
    */
-  void setEnforceUniqueZone(boolean enforceUniqueZone) {
-    this.enforceUniqueZone |= enforceUniqueZone;
-  }
-
-  @Override
-  public boolean enforceUniqueZone() {
-    return enforceUniqueZone;
-  }
-
-  public String getRedundancyZone(InternalDistributedMember member) {
-    return redundancyZones.get(member);
-  }
+  Set<InternalDistributedMember> getMembersInThisZone();
 
   /**
-   * Asserts that distributionManagerType is LOCAL, GEMFIRE, or ADMIN_ONLY. Also asserts that the
-   * distributionManagerId (jgroups DistributedMember) has a VmKind that matches.
+   * Acquire a permit to request a GII from another member
    */
-  private void assertDistributionManagerType() {
-    // Assert that dmType is one of the three DM types...
-    int theDmType = getDMType();
-    switch (theDmType) {
-      case NORMAL_DM_TYPE:
-      case LONER_DM_TYPE:
-      case ADMIN_ONLY_DM_TYPE:
-      case LOCATOR_DM_TYPE:
-        break;
-      default:
-        Assert.assertTrue(false, "unknown distribution manager type");
-    }
-
-    // Assert InternalDistributedMember VmKind matches this DistributionManagerType...
-    final InternalDistributedMember theId = getDistributionManagerId();
-    final int vmKind = theId.getVmKind();
-    if (theDmType != vmKind) {
-      Assert.assertTrue(false,
-          "InternalDistributedMember has a vmKind of " + vmKind + " instead of " + theDmType);
-    }
-  }
-
-  public int getDMType() {
-    return this.dmType;
-  }
-
-  public List<InternalDistributedMember> getViewMembers() {
-    return membershipManager.getView().getMembers();
-  }
-
-  /* implementation of DM.getOldestMember */
-  @Override
-  public DistributedMember getOldestMember(Collection c) throws NoSuchElementException {
-    List<InternalDistributedMember> view = getViewMembers();
-    for (int i = 0; i < view.size(); i++) {
-      Object viewMbr = view.get(i);
-      Iterator it = c.iterator();
-      while (it.hasNext()) {
-        Object nextMbr = it.next();
-        if (viewMbr.equals(nextMbr)) {
-          return (DistributedMember) nextMbr;
-        }
-      }
-    }
-    throw new NoSuchElementException(
-        LocalizedStrings.DistributionManager_NONE_OF_THE_GIVEN_MANAGERS_IS_IN_THE_CURRENT_MEMBERSHIP_VIEW
-            .toLocalizedString());
-  }
-
-  private boolean testMulticast() {
-    return this.membershipManager.testMulticast();
-  }
+  void acquireGIIPermitUninterruptibly();
 
   /**
-   * Need to do this outside the constructor so that the child constructor can finish.
+   * Release a permit to request a GII from another member.
    */
-  private void startThreads() {
-    this.system.setDM(this); // fix for bug 33362
-    if (this.memberEventThread != null)
-      this.memberEventThread.start();
-    try {
-
-      // And the distinguished guests today are...
-      NetView v = membershipManager.getView();
-      logger.info(LocalizedMessage.create(
-          LocalizedStrings.DistributionManager_INITIAL_MEMBERSHIPMANAGER_VIEW___0,
-          String.valueOf(v)));
-
-      // Add them all to our view
-      for (InternalDistributedMember internalDistributedMember : v.getMembers()) {
-        addNewMember(internalDistributedMember);
-      }
+  void releaseGIIPermit();
 
-      // Figure out who the elder is...
-      selectElder(); // ShutdownException could be thrown here
-    } catch (Exception ex) {
-      throw new InternalGemFireException(
-          LocalizedStrings.DistributionManager_COULD_NOT_PROCESS_INITIAL_VIEW.toLocalizedString(),
-          ex);
-    }
-    try {
-      getWaitingThreadPool().execute(new Runnable() {
-        public void run() {
-          // call in background since it might need to send a reply
-          // and we are not ready to send messages until startup is finished
-          setIsStartupThread(Boolean.TRUE);
-          readyForMessages();
-        }
-      });
-    } catch (VirtualMachineError err) {
-      SystemFailure.initiateFailure(err);
-      // If this ever returns, rethrow the error. We're poisoned
-      // now, so don't let this thread continue.
-      throw err;
-    } catch (Throwable t) {
-      // Whenever you catch Error or Throwable, you must also
-      // catch VirtualMachineError (see above). However, there is
-      // _still_ a possibility that you are dealing with a cascading
-      // error condition, so you also need to check to see if the JVM
-      // is still usable:
-      SystemFailure.checkFailure();
-      logger.fatal(LocalizedMessage.create(
-          LocalizedStrings.DistributionManager_UNCAUGHT_EXCEPTION_CALLING_READYFORMESSAGES), t);
-    }
-  }
+  int getDistributedSystemId();
 
-  private void readyForMessages() {
-    synchronized (this) {
-      this.readyForMessages = true;
-      this.notifyAll();
-    }
-    membershipManager.startEventProcessing();
-  }
+  boolean enforceUniqueZone();
 
-  private void waitUntilReadyForMessages() {
-    if (readyForMessages)
-      return;
-    synchronized (this) {
-      for (;;) {
-        if (readyForMessages)
-          break;
-        stopper.checkCancelInProgress(null);
-        boolean interrupted = Thread.interrupted();
-        try {
-          this.wait();
-        } catch (InterruptedException e) {
-          interrupted = true;
-          stopper.checkCancelInProgress(e);
-        } finally {
-          if (interrupted) {
-            Thread.currentThread().interrupt();
-          }
-        }
-      } // for
-    } // synchronized
-  }
+  Set<InternalDistributedMember> getMembersInSameZone(InternalDistributedMember acceptedMember);
 
-  /**
-   * Call when the DM is ready to send messages.
-   */
-  private void readyToSendMsgs() {
-    synchronized (this.readyToSendMsgsLock) {
-      this.readyToSendMsgs = true;
-      this.readyToSendMsgsLock.notifyAll();
-    }
-  }
+  boolean areInSameZone(InternalDistributedMember member1, InternalDistributedMember member2);
 
   /**
-   * Return when DM is ready to send out messages.
+   * Returns true is the two members are on the same equivalent host machine based on overlapping IP
+   * addresses collected for all NICs on each member's machine.
    *
-   * @param msg the messsage that is currently being sent
-   */
-  private void waitUntilReadyToSendMsgs(DistributionMessage msg) {
-    if (this.readyToSendMsgs) {
-      return;
-    }
-    // another process may have been started in the same view, so we need
-    // to be responsive to startup messages and be able to send responses
-    if (msg instanceof StartupMessage || msg instanceof StartupResponseMessage
-        || msg instanceof AdminMessageType) {
-      return;
-    }
-    if (getIsStartupThread() == Boolean.TRUE) {
-      // let the startup thread send messages
-      // the only case I know of that does this is if we happen to log a
-      // message during startup and an alert listener has registered.
-      return;
-    }
-
-    synchronized (this.readyToSendMsgsLock) {
-      for (;;) {
-        if (this.readyToSendMsgs)
-          break;
-        stopper.checkCancelInProgress(null);
-        boolean interrupted = Thread.interrupted();
-        try {
-          this.readyToSendMsgsLock.wait();
-        } catch (InterruptedException e) {
-          interrupted = true;
-          stopper.checkCancelInProgress(e);
-        } finally {
-          if (interrupted) {
-            Thread.currentThread().interrupt();
-          }
-        }
-      } // for
-    } // synchronized
-  }
-
-
-  @Override
-  public void forceUDPMessagingForCurrentThread() {
-    membershipManager.forceUDPMessagingForCurrentThread();
-  }
-
-
-  @Override
-  public void releaseUDPMessagingForCurrentThread() {
-    membershipManager.releaseUDPMessagingForCurrentThread();
-  }
-
-  /**
-   * Did an exception occur in one of the threads launched by this distribution manager?
-   */
-  public boolean exceptionInThreads() {
-    return this.exceptionInThreads || this.threadGroup.getUncaughtExceptionsCount() > 0;
-  }
-
-  /**
-   * Clears the boolean that determines whether or not an exception occurred in one of the worker
-   * threads. This method should be used for testing purposes only!
-   */
-  void clearExceptionInThreads() {
-    this.exceptionInThreads = false;
-    this.threadGroup.clearUncaughtExceptionsCount();
-  }
-
-  /**
-   * Returns the current "cache time" in milliseconds since the epoch. The "cache time" takes into
-   * account skew among the local clocks on the various machines involved in the cache.
+   * @param member1 First member
+   * @param member2 Second member
    */
-  @Override
-  public long cacheTimeMillis() {
-    return this.system.getClock().cacheTimeMillis();
-  }
+  boolean areOnEquivalentHost(InternalDistributedMember member1, InternalDistributedMember member2);
 
+  Set<InetAddress> getEquivalents(InetAddress in);
 
+  Set<DistributedMember> getGroupMembers(String group);
 
   /**
-   * Returns the id of this distribution manager.
-   */
-  @Override
-  public InternalDistributedMember getDistributionManagerId() {
-    return this.localAddress;
-  }
-
-  /**
-   * Returns an unmodifiable set containing the identities of all of the known (non-admin-only)
-   * distribution managers.
-   */
-  @Override
-  public Set getDistributionManagerIds() {
-    // access to members synchronized under membersLock in order to
-    // ensure serialization
-    synchronized (this.membersLock) {
-      return this.members.keySet();
-    }
-  }
-
-  /**
-   * Adds the entry in {@link #hostedLocatorsAll} for a member with one or more hosted locators. The
-   * value is a collection of host[port] strings. If a bind-address was used for a locator then the
-   * form is bind-addr[port].
+   * Adds the entry in hostedLocators for a member with one or more hosted locators. The value is a
+   * collection of host[port] strings. If a bind-address was used for a locator then the form is
+   * bind-addr[port].
+   * <p>
+   * This currently only tracks stand-alone/dedicated locators, not embedded locators.
+   *
+   * @param isSharedConfigurationEnabled flag to determine if the locator has enabled shared
+   *        configuration
    *
    * @since GemFire 6.6.3
    */
-  @Override
-  public void addHostedLocators(InternalDistributedMember member, Collection<String> locators,
-      boolean isSharedConfigurationEnabled) {
-    synchronized (this.membersLock) {
-      if (locators == null || locators.isEmpty()) {
-        throw new IllegalArgumentException("Cannot use empty collection of locators");
-      }
-      if (this.hostedLocatorsAll.isEmpty()) {
-        this.hostedLocatorsAll = new HashMap<InternalDistributedMember, Collection<String>>();
-      }
-      Map<InternalDistributedMember, Collection<String>> tmp =
-          new HashMap<InternalDistributedMember, Collection<String>>(this.hostedLocatorsAll);
-      tmp.remove(member);
-      tmp.put(member, locators);
-      tmp = Collections.unmodifiableMap(tmp);
-      this.hostedLocatorsAll = tmp;
-
-      if (isSharedConfigurationEnabled) {
-        if (locators == null || locators.isEmpty()) {
-          throw new IllegalArgumentException("Cannot use empty collection of locators");
-        }
-        if (this.hostedLocatorsWithSharedConfiguration.isEmpty()) {
-          this.hostedLocatorsWithSharedConfiguration = new HashMap<>();
-        }
-        tmp = new HashMap<>(this.hostedLocatorsWithSharedConfiguration);
-        tmp.remove(member);
-        tmp.put(member, locators);
-        tmp = Collections.unmodifiableMap(tmp);
-        this.hostedLocatorsWithSharedConfiguration = tmp;
-      }
-
-    }
-  }
-
-
-  private void removeHostedLocators(InternalDistributedMember member) {
-    synchronized (this.membersLock) {
-      if (this.hostedLocatorsAll.containsKey(member)) {
-        Map<InternalDistributedMember, Collection<String>> tmp =
-            new HashMap<>(this.hostedLocatorsAll);
-        tmp.remove(member);
-        if (tmp.isEmpty()) {
-          tmp = Collections.emptyMap();
-        } else {
-          tmp = Collections.unmodifiableMap(tmp);
-        }
-        this.hostedLocatorsAll = tmp;
-      }
-      if (this.hostedLocatorsWithSharedConfiguration.containsKey(member)) {
-        Map<InternalDistributedMember, Collection<String>> tmp =
-            new HashMap<InternalDistributedMember, Collection<String>>(
-                this.hostedLocatorsWithSharedConfiguration);
-        tmp.remove(member);
-        if (tmp.isEmpty()) {
-          tmp = Collections.emptyMap();
-        } else {
-          tmp = Collections.unmodifiableMap(tmp);
-        }
-        this.hostedLocatorsWithSharedConfiguration = tmp;
-      }
-    }
-  }
-
+  void addHostedLocators(InternalDistributedMember member, Collection<String> locators,
+      boolean isSharedConfigurationEnabled);
 
 
   /**
-   * Gets the value in {@link #hostedLocatorsAll} for a member with one or more hosted locators. The
-   * value is a collection of host[port] strings. If a bind-address was used for a locator then the
-   * form is bind-addr[port].
+   * Gets the value in hostedLocators for a member with one or more hosted locators. The value is a
+   * collection of host[port] strings. If a bind-address was used for a locator then the form is
+   * bind-addr[port].
+   * <p>
+   * This currently only tracks stand-alone/dedicated locators, not embedded locators.
    *
    * @since GemFire 6.6.3
    */
-  @Override
-  public Collection<String> getHostedLocators(InternalDistributedMember member) {
-    synchronized (this.membersLock) {
-      return this.hostedLocatorsAll.get(member);
-    }
-  }
+  Collection<String> getHostedLocators(InternalDistributedMember member);
 
   /**
-   * Returns a copy of the map of all members hosting locators. The key is the member, and the value
-   * is a collection of host[port] strings. If a bind-address was used for a locator then the form
-   * is bind-addr[port].
-   *
-   * The member is the vm that hosts one or more locator, if another locator starts up linking to
-   * this locator, it will put that member in this map as well, and this member will in the map on
-   * the other locato vm as well.
+   * Gets the map of all members hosting locators. The key is the member, and the value is a
+   * collection of host[port] strings. If a bind-address was used for a locator then the form is
+   * bind-addr[port].
    *
-   * The keyset of the map are the locator vms in this cluster.
    *
-   * the value is a collection of strings in case one vm can have multiple locators ????
+   * <p>
+   * This currently only tracks stand-alone/dedicated locators, not embedded locators.
    *
    * @since GemFire 6.6.3
    */
-  public Map<InternalDistributedMember, Collection<String>> getAllHostedLocators() {
-    synchronized (this.membersLock) {
-      return this.hostedLocatorsAll;
-    }
-  }
+  Map<InternalDistributedMember, Collection<String>> getAllHostedLocators();
 
   /**
-   * Returns a copy of the map of all members hosting locators with shared configuration. The key is
-   * the member, and the value is a collection of host[port] strings. If a bind-address was used for
-   * a locator then the form is bind-addr[port].
+   * Gets the map of all members hosting locators with shared configuration. The key is the member,
+   * and the value is a collection of host[port] strings. If a bind-address was used for a locator
+   * then the form is bind-addr[port].
+   * <p>
+   * This currently only tracks stand-alone/dedicated locators, not embedded locators.
    *
    * @since GemFire 8.0
    */
-  @Override
-  public Map<InternalDistributedMember, Collection<String>> getAllHostedLocatorsWithSharedConfiguration() {
-    synchronized (this.membersLock) {
-      return this.hostedLocatorsWithSharedConfiguration;
-    }
-  }
+  Map<InternalDistributedMember, Collection<String>> getAllHostedLocatorsWithSharedConfiguration();
 
   /**
-   * Returns an unmodifiable set containing the identities of all of the known (including admin)
-   * distribution managers.
+   * Forces use of UDP for communications in the current thread. UDP is connectionless, so no tcp/ip
+   * connections will be created or used for messaging until this setting is released with
+   * releaseUDPMessagingForCurrentThread.
    */
-  @Override
-  public Set getDistributionManagerIdsIncludingAdmin() {
-    // access to members synchronized under membersLock in order to
-    // ensure serialization
-    synchronized (this.membersLock) {
-      return this.membersAndAdmin;
-    }
-  }
-
+  void forceUDPMessagingForCurrentThread();
 
   /**
-   * Returns a private-memory list containing the identities of all the other known distribution
-   * managers not including me.
+   * Releases use of UDP for all communications in the current thread, as established by
+   * forceUDPMessagingForCurrentThread.
    */
-  @Override
-  public Set getOtherDistributionManagerIds() {
-    // We return a modified copy of the list, so
-    // collect the old list and copy under the lock.
-    Set result = new HashSet(getDistributionManagerIds());
-
-    InternalDistributedMember me = getDistributionManagerId();
-    result.remove(me);
-
-    // It's okay for my own id to not be in the list of all ids yet.
-    return result;
-  }
-
-  @Override
-  public Set getOtherNormalDistributionManagerIds() {
-    // We return a modified copy of the list, so
-    // collect the old list and copy under the lock.
-    Set result = new HashSet(getNormalDistributionManagerIds());
-
-    InternalDistributedMember me = getDistributionManagerId();
-    result.remove(me);
-
-    // It's okay for my own id to not be in the list of all ids yet.
-    return result;
-  }
-
-  @Override
-  public InternalDistributedMember getCanonicalId(DistributedMember id) {
-    // the members set is copy-on-write, so it is safe to iterate over it
-    InternalDistributedMember result = this.members.get(id);
-    if (result == null) {
-      return (InternalDistributedMember) id;
-    }
-    return result;
-  }
+  void releaseUDPMessagingForCurrentThread();
 
   /**
-   * Add a membership listener and return other DistributionManagerIds as an atomic operation
+   * returns the type of node
+   *
+   * @see ClusterDistributionManager#NORMAL_DM_TYPE
+   * @see ClusterDistributionManager#LONER_DM_TYPE
+   * @see ClusterDistributionManager#LOCATOR_DM_TYPE
+   * @see ClusterDistributionManager#ADMIN_ONLY_DM_TYPE
    */
-  @Override
-  public Set addMembershipListenerAndGetDistributionManagerIds(MembershipListener l) {
-    // switched sync order to fix bug 30360
-    synchronized (this.membersLock) {
-      // Don't let the members come and go while we are adding this
-      // listener. This ensures that the listener (probably a
-      // ReplyProcessor) gets a consistent view of the members.
-      addMembershipListener(l);
-      // Note it is ok to return the members set
-      // because we will never modify the returned set.
-      return members.keySet();
-    }
-  }
-
-  private void addNewMember(InternalDistributedMember member) {
-    // This is the place to cleanup the zombieMembers
-    int vmType = member.getVmKind();
-    switch (vmType) {
-      case ADMIN_ONLY_DM_TYPE:
-        handleConsoleStartup(member);
-        break;
-      case LOCATOR_DM_TYPE:
-      case NORMAL_DM_TYPE:
-        handleManagerStartup(member);
-        break;
-      default:
-        throw new InternalGemFireError(LocalizedStrings.DistributionManager_UNKNOWN_MEMBER_TYPE_0
-            .toLocalizedString(Integer.valueOf(vmType)));
-    }
-  }
+  int getDMType();
 
   /**
-   * Returns the identity of this <code>DistributionManager</code>
+   * The returned cache will be null if the cache does not yet exist. Note that the returned cache
+   * may be one that is already closed. Callers of GemFireCacheImpl.getInstance() should try to use
+   * this method.
    */
-  @Override
-  public InternalDistributedMember getId() {
-    return this.localAddress;
-  }
-
-  @Override
-  public long getMembershipPort() {
-    return localAddress.getPort();
-  }
-
-  @Override
-  public Set putOutgoing(final DistributionMessage msg) {
-    try {
-      DistributionMessageObserver observer = DistributionMessageObserver.getInstance();
-      if (observer != null) {
-        observer.beforeSendMessage(this, msg);
-      }
-      return sendMessage(msg);
-    } catch (NotSerializableException e) {
-      throw new InternalGemFireException(e);
-    } catch (ToDataException e) {
-      // exception from user code
-      throw e;
-    }
-  }
-
-  @Override
-  public String toString() {
-    return this.description;
-  }
+  InternalCache getCache();
 
   /**
-   * Informs other members that this dm is shutting down. Stops the pusher, puller, and processor
-   * threads and closes the connection to the transport layer.
+   * Returns an existing non-closed cache associated with this DM. Callers of
+   * CacheFactory.getAnyInstance(), CacheFactory.getInstance(DistributedSystem) or
+   * GemFireCacheImpl.getExisting() should try to use this method.
+   *
+   * @throws CacheClosedException if a cache has not yet been associated with this DM or it has been
+   *         {@link Cache#isClosed closed}.
    */
-  protected void shutdown() {
-    // Make sure only one thread initiates shutdown...
-    synchronized (shutdownMutex) {
-      if (closeInProgress) {
-        return;
-      }
-      this.closeInProgress = true;
-    } // synchronized
-
-    // [bruce] log shutdown at info level and with ID to balance the
-    // "Starting" message. recycleConn.conf is hard to debug w/o this
-    final String exceptionStatus = (this.exceptionInThreads()
-        ? LocalizedStrings.DistributionManager_AT_LEAST_ONE_EXCEPTION_OCCURRED.toLocalizedString()
-        : "");
-    logger.info(LocalizedMessage.create(
-        LocalizedStrings.DistributionManager_SHUTTING_DOWN_DISTRIBUTIONMANAGER_0_1,
-        new Object[] {this.localAddress, exceptionStatus}));
-
-    final long start = System.currentTimeMillis();
-    try {
-      if (this.rootCause instanceof ForcedDisconnectException) {
-        if (logger.isDebugEnabled()) {
-          logger.debug(
-              "inhibiting sending of shutdown message to other members due to forced-disconnect");
-        }
-      } else {
-        // Don't block indefinitely trying to send the shutdown message, in
-        // case other VMs in the system are ill-behaved. (bug 34710)
-        final Runnable r = new Runnable() {
-          public void run() {
-            try {
-              ConnectionTable.threadWantsSharedResources();
-              sendShutdownMessage();
-            } catch (final CancelException e) {
-              // We were terminated.
-              logger.debug("Cancelled during shutdown message", e);
-            }
-          }
-        };
-        final Thread t = new Thread(threadGroup, r,
-            LocalizedStrings.DistributionManager_SHUTDOWN_MESSAGE_THREAD_FOR_0
-                .toLocalizedString(this.localAddress));
-        t.start();
-        boolean interrupted = Thread.interrupted();
-        try {
-          t.join(MAX_STOP_TIME / 4);
-        } catch (final InterruptedException e) {
-          interrupted = true;
-          t.interrupt();
-          logger.warn(
-              LocalizedMessage.create(
-                  LocalizedStrings.DistributionManager_INTERRUPTED_SENDING_SHUTDOWN_MESSAGE_TO_PEERS),
-              e);
-        } finally {
-          if (interrupted) {
-            Thread.currentThread().interrupt();
-          }
-        }
-
-        if (t.isAlive()) {
-          t.interrupt();
-          logger.warn(LocalizedMessage.create(
-              LocalizedStrings.DistributionManager_FAILED_SENDING_SHUTDOWN_MESSAGE_TO_PEERS_TIMEOUT));
-        }
-      }
-
-    } finally {
-      this.shutdownMsgSent = true; // in case sendShutdownMessage failed....
-      try {
-        this.uncleanShutdown(false);
-      } finally {
-        final Long delta = Long.valueOf(System.currentTimeMillis() - start);
-        logger.info(LocalizedMessage.create(
-            LocalizedStrings.DistributionManager_DISTRIBUTIONMANAGER_STOPPED_IN_0_MS, delta));
-      }
-    }
-  }
-
-  private void askThreadsToStop() {
-    // Stop executors after they have finished
-    ExecutorService es;
-    es = this.serialThread;
-    if (es != null) {
-      es.shutdown();
-    }
-    es = this.viewThread;
-    if (es != null) {
-      // Hmmm...OK, I'll let any view events currently in the queue be
-      // processed. Not sure it's very important whether they get
-      // handled...
-      es.shutdown();
-    }
-    if (this.serialQueuedExecutorPool != null) {
-      this.serialQueuedExecutorPool.shutdown();
-    }
-    es = this.functionExecutionThread;
-    if (es != null) {
-      es.shutdown();
-    }
-    es = this.functionExecutionPool;
-    if (es != null) {
-      es.shutdown();
-    }
-    es = this.partitionedRegionThread;
-    if (es != null) {
-      es.shutdown();
-    }
-    es = this.partitionedRegionPool;
-    if (es != null) {
-      es.shutdown();
-    }
-    es = this.highPriorityPool;
-    if (es != null) {
-      es.shutdown();
-    }
-    es = this.waitingPool;
-    if (es != null) {
-      es.shutdown();
-    }
-    es = this.prMetaDataCleanupThreadPool;
-    if (es != null) {
-      es.shutdown();
-    }
-    es = this.threadPool;
-    if (es != null) {
-      es.shutdown();
-    }
-
-    Thread th = this.memberEventThread;
-    if (th != null)
-      th.interrupt();
-  }
-
-  private void waitForThreadsToStop(long timeInMillis) throws InterruptedException {
-    long start = System.currentTimeMillis();
-    long remaining = timeInMillis;
-
-    ExecutorService[] allExecutors = new ExecutorService[] {this.serialThread, this.viewThread,
-        this.functionExecutionThread, this.functionExecutionPool, this.partitionedRegionThread,
-        this.partitionedRegionPool, this.highPriorityPool, this.waitingPool,
-        this.prMetaDataCleanupThreadPool, this.threadPool};
-    for (ExecutorService es : allExecutors) {
-      if (es != null) {
-        es.awaitTermination(remaining, TimeUnit.MILLISECONDS);
-      }
-      remaining = timeInMillis - (System.currentTimeMillis() - start);
-      if (remaining <= 0) {
-        return;
-      }
-    }
-
-
-    this.serialQueuedExecutorPool.awaitTermination(remaining, TimeUnit.MILLISECONDS);
-    remaining = timeInMillis - (System.currentTimeMillis() - start);
-    if (remaining <= 0) {
-      return;
-    }
-    Thread th = this.memberEventThread;
-    if (th != null) {
-      th.interrupt(); // bug #43452 - this thread sometimes eats interrupts, so we interrupt it
-                      // again here
-      th.join(remaining);
-    }
-
-  }
-
-  /**
-   * Cheap tool to kill a referenced thread
-   *
-   * @param t the thread to kill
-   */
-  private void clobberThread(Thread t) {
-    if (t == null)
-      return;
-    if (t.isAlive()) {
-      logger.warn(LocalizedMessage
-          .create(LocalizedStrings.DistributionManager_FORCING_THREAD_STOP_ON__0_, t));
-
-      // Start by being nice.
-      t.interrupt();
-
-      // we could be more violent here...
-      // t.stop();
-      try {
-        for (int i = 0; i < MAX_STOP_ATTEMPTS && t.isAlive(); i++) {
-          t.join(STOP_PAUSE_TIME);
-          t.interrupt();
-        }
-      } catch (InterruptedException ex) {
-        logger.debug("Interrupted while attempting to terminate threads.");
-        Thread.currentThread().interrupt();
-        // just keep going
-      }
-
-      if (t.isAlive()) {
-        logger.warn(LocalizedMessage.create(
-            LocalizedStrings.DistributionManager_CLOBBERTHREAD_THREAD_REFUSED_TO_DIE__0, t));
-      }
-    }
-  }
-
-  /**
-   * Cheap tool to examine an executor to see if it is still working
-   *
-   * @param tpe
-   * @return true if executor is still active
-   */
-  private boolean executorAlive(ThreadPoolExecutor tpe, String name) {
-    if (tpe == null) {
-      return false;
-    } else {
-      int ac = tpe.getActiveCount();
-      // boolean result = tpe.getActiveCount() > 0;
-      if (ac > 0) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("Still waiting for {} threads in '{}' pool to exit", ac, name);
-        }
-        return true;
-      } else {
-        return false;
-      }
-    }
-  }
-
-  /**
-   * Wait for the ancillary queues to exit. Kills them if they are still around.
-   *
-   */
-  private void forceThreadsToStop() {
-    long endTime = System.currentTimeMillis() + MAX_STOP_TIME;
-    String culprits = "";
-    for (;;) {
-      boolean stillAlive = false;
-      culprits = "";
-      if (executorAlive(this.serialThread, "serial thread")) {
-        stillAlive = true;
-        culprits = culprits + " serial thread;";
-      }
-      if (executorAlive(this.viewThread, "view thread")) {
-        stillAlive = true;
-        culprits = culprits + " view thread;";
-      }
-      if (executorAlive(this.partitionedRegionThread, "partitioned region thread")) {
-        stillAlive = true;
-        culprits = culprits + " partitioned region thread;";
-      }
-      if (executorAlive(this.partitionedRegionPool, "partitioned region pool")) {
-        stillAlive = true;
-        culprits = culprits + " partitioned region pool;";
-      }
-      if (executorAlive(this.highPriorityPool, "high priority pool")) {
-        stillAlive = true;
-        culprits = culprits + " high priority pool;";
-      }
-      if (executorAlive(this.waitingPool, "waiting pool")) {
-        stillAlive = true;
-        culprits = culprits + " waiting pool;";
-      }
-      if (executorAlive(this.prMetaDataCleanupThreadPool, "prMetaDataCleanupThreadPool")) {
-        stillAlive = true;
-        culprits = culprits + " special waiting pool;";
-      }
-      if (executorAlive(this.threadPool, "thread pool")) {
-        stillAlive = true;
-        culprits = culprits + " thread pool;";
-      }
-
-      if (!stillAlive)
-        return;
-
-      long now = System.currentTimeMillis();
-      if (now >= endTime)
-        break;
-
-      try {
-        Thread.sleep(STOP_PAUSE_TIME);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        // Desperation, the shutdown thread is being killed. Don't
-        // consult a CancelCriterion.
-        logger.warn(LocalizedMessage
-            .create(LocalizedStrings.DistributionManager_INTERRUPTED_DURING_SHUTDOWN), e);
-        break;
-      }
-    } // for
-
-    logger.warn(LocalizedMessage.create(
-        LocalizedStrings.DistributionManager_DAEMON_THREADS_ARE_SLOW_TO_STOP_CULPRITS_INCLUDE_0,
-        culprits));
-
-    // Kill with no mercy
-    if (this.serialThread != null) {
-      this.serialThread.shutdownNow();
-    }
-    if (this.viewThread != null) {
-      this.viewThread.shutdownNow();
-    }
-    if (this.functionExecutionThread != null) {
-      this.functionExecutionThread.shutdownNow();
-    }
-    if (this.functionExecutionPool != null) {
-      this.functionExecutionPool.shutdownNow();
-    }
-    if (this.partitionedRegionThread != null) {
-      this.partitionedRegionThread.shutdownNow();
-    }
-    if (this.partitionedRegionPool != null) {
-      this.partitionedRegionPool.shutdownNow();
-    }
-    if (this.highPriorityPool != null) {
-      this.highPriorityPool.shutdownNow();
-    }
-    if (this.waitingPool != null) {
-      this.waitingPool.shutdownNow();
-    }
-    if (this.prMetaDataCleanupThreadPool != null) {
-      this.prMetaDataCleanupThreadPool.shutdownNow();
-    }
-    if (this.threadPool != null) {
-      this.threadPool.shutdownNow();
-    }
-
-    Thread th = this.memberEventThread;
-    if (th != null) {
-      clobberThread(th);
-    }
-  }
-
-  private volatile boolean shutdownInProgress = false;
-
-  /** guard for membershipViewIdAcknowledged */
-  private final Object membershipViewIdGuard = new Object();
-
-  /** the latest view ID that has been processed by all membership listeners */
-  private long membershipViewIdAcknowledged;
-
-  @Override
-  public boolean shutdownInProgress() {
-    return this.shutdownInProgress;
-  }
-
-  /**
-   * Stops the pusher, puller and processor threads and closes the connection to the transport
-   * layer. This should only be used from shutdown() or from the dm initialization code
-   */
-  private void uncleanShutdown(boolean beforeJoined) {
-    try {
-      this.closeInProgress = true; // set here also to fix bug 36736
-      removeAllHealthMonitors();
-      shutdownInProgress = true;
-      if (membershipManager != null) {
-        membershipManager.setShutdown();
-      }
-
-      askThreadsToStop();
-
-      // wait a moment before asking threads to terminate
-      try {
-        waitForThreadsToStop(1000);
-      } catch (InterruptedException ie) {
-        // No need to reset interrupt bit, we're really trying to quit...
-      }
-      forceThreadsToStop();
-
-    } // try
-    finally {
-      // ABSOLUTELY ESSENTIAL that we close the distribution channel!
-      try {
-        // For safety, but channel close in a finally AFTER this...
-        if (this.stats != null) {
-          this.stats.close();
-          try {
-            Thread.sleep(100);
-          } catch (InterruptedException ie) {
-            // No need to reset interrupt bit, we're really trying to quit...
-          }
-        }
-      } finally {
-        if (this.membershipManager != null) {
-          logger.info(LocalizedMessage.create(
-              LocalizedStrings.DistributionManager_NOW_CLOSING_DISTRIBUTION_FOR__0,
-              this.localAddress));
-          this.membershipManager.disconnect(beforeJoined);
-        }
-      }
-    }
-  }
-
-  @Override
-  public InternalDistributedSystem getSystem() {
-    return this.system;
-  }
-
-  /**
-   * Returns the transport configuration for this distribution manager
-   */
-  RemoteTransportConfig getTransport() {
-    return this.transport;
-  }
-
-
-  @Override
-  public void addMembershipListener(MembershipListener l) {
-    this.membershipListeners.putIfAbsent(l, Boolean.TRUE);
-  }
-
-  @Override
-  public void removeMembershipListener(MembershipListener l) {
-    this.membershipListeners.remove(l);
-  }
-
-  /**
-   * Adds a <code>MembershipListener</code> to this distribution manager.
-   */
-  private void addAllMembershipListener(MembershipListener l) {
-    synchronized (this.allMembershipListenersLock) {
-      Set newAllMembershipListeners = new HashSet(this.allMembershipListeners);
-      newAllMembershipListeners.add(l);
-      this.allMembershipListeners = newAllMembershipListeners;
-    }
-  }
-
-  @Override
-  public void removeAllMembershipListener(MembershipListener l) {
-    synchronized (this.allMembershipListenersLock) {
-      Set newAllMembershipListeners = new HashSet(this.allMembershipListeners);
-      if (!newAllMembershipListeners.remove(l)) {
-        // There seems to be a race condition in which
-        // multiple departure events can be registered
-        // on the same peer. We regard this as benign.
-        // FIXME when membership events become sane again
-        // String s = "MembershipListener was never registered";
-        // throw new IllegalArgumentException(s);
-      }
-      this.allMembershipListeners = newAllMembershipListeners;
-    }
-  }
-
-  /**
-   * Returns true if this DM or the DistributedSystem owned by it is closing or is closed.
-   */
-  private boolean isCloseInProgress() {
-    if (closeInProgress) {
-      return true;
-    }
-    InternalDistributedSystem ds = getSystem();
-    if (ds != null && ds.isDisconnecting()) {
-      return true;
-    }
-    return false;
-  }
-
-  public boolean isShutdownStarted() {
-    return closeInProgress;
-  }
-
-  private void handleViewInstalledEvent(ViewInstalledEvent ev) {
-    synchronized (this.membershipViewIdGuard) {
-      this.membershipViewIdAcknowledged = ev.getViewId();
-      this.membershipViewIdGuard.notifyAll();
-    }
-  }
-
-  /**
-   * This stalls waiting for the current membership view (as seen by the membership manager) to be
-   * acknowledged by all membership listeners
-   */
-  void waitForViewInstallation(long id) throws InterruptedException {
-    if (id <= this.membershipViewIdAcknowledged) {
-      return;
-    }
-    synchronized (this.membershipViewIdGuard) {
-      while (this.membershipViewIdAcknowledged < id && !this.stopper.isCancelInProgress()) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("waiting for view {}.  Current DM view processed by all listeners is {}", id,
-              this.membershipViewIdAcknowledged);
-        }
-        this.membershipViewIdGuard.wait();
-      }
-    }
-  }
-
-  private void handleMemberEvent(MemberEvent ev) {
-    ev.handleEvent(this);
-  }
-
-  /**
-   * This thread processes member events as they occur.
-   *
-   * @see org.apache.geode.distributed.internal.DistributionManager.MemberCrashedEvent
-   * @see org.apache.geode.distributed.internal.DistributionManager.MemberJoinedEvent
-   * @see org.apache.geode.distributed.internal.DistributionManager.MemberDepartedEvent
-   *
-   */
-  protected class MemberEventInvoker implements Runnable {
-
-
-    @SuppressWarnings("synthetic-access")
-    public void run() {
-      for (;;) {
-        SystemFailure.checkFailure();
-        // bug 41539 - member events need to be delivered during shutdown
-        // or reply processors may hang waiting for replies from
-        // departed members
-        // if (getCancelCriterion().isCancelInProgress()) {
-        // break; // no message, just quit
-        // }
-        if (!DistributionManager.this.system.isConnected && DistributionManager.this.isClosed()) {
-          break;
-        }
-        try {
-          MemberEvent ev = (MemberEvent) DistributionManager.this.membershipEventQueue.take();
-          handleMemberEvent(ev);
-        } catch (InterruptedException e) {
-          if (isCloseInProgress()) {
-            if (logger.isTraceEnabled()) {
-              logger.trace("MemberEventInvoker: InterruptedException during shutdown");
-            }
-          } else {
-            logger.warn(LocalizedMessage
-                .create(LocalizedStrings.DistributionManager_UNEXPECTED_INTERRUPTEDEXCEPTION), e);
-          }
-          break;
-        } catch (DistributedSystemDisconnectedException e) {
-          break;
-        } catch (CancelException e) {
-          if (isCloseInProgress()) {
-            if (logger.isTraceEnabled()) {
-              logger.trace("MemberEventInvoker: cancelled");
-            }
-          } else {
-            logger.warn(LocalizedMessage
-                .create(LocalizedStrings.DistributionManager_UNEXPECTED_CANCELLATION), e);
-          }
-          break;
-        } catch (Exception e) {
-          logger.fatal(
-              LocalizedMessage.create(
-                  LocalizedStrings.DistributionManager_UNCAUGHT_EXCEPTION_PROCESSING_MEMBER_EVENT),
-              e);
-        }
-      } // for
-      if (logger.isTraceEnabled()) {
-        logger.trace("MemberEventInvoker on {} stopped", DistributionManager.this);
-      }
-    }
-  }
-
-  private void addMemberEvent(MemberEvent ev) {
-    if (SYNC_EVENTS) {
-      handleMemberEvent(ev);
-    } else {
-      stopper.checkCancelInProgress(null);
-      boolean interrupted = Thread.interrupted();
-      try {
-        this.membershipEventQueue.put(ev);
-      } catch (InterruptedException ex) {
-        interrupted = true;
-        stopper.checkCancelInProgress(ex);
-        handleMemberEvent(ev); // FIXME why???
-      } finally {
-        if (interrupted) {
-          Thread.currentThread().interrupt();
-        }
-      }
-    }
-  }
-
-
-  @Override
-  public void close() {
-    if (!closed) {
-      this.shutdown();
-      logger.info(LocalizedMessage.create(
-          LocalizedStrings.DistributionManager_MARKING_DISTRIBUTIONMANAGER_0_AS_CLOSED,
-          this.localAddress));
-      MembershipLogger.logShutdown(this.localAddress);
-      closed = true;
-    }
-  }
-
-  @Override
-  public void throwIfDistributionStopped() {
-    if (this.shutdownMsgSent) {
-      throw new DistributedSystemDisconnectedException(
-          LocalizedStrings.DistributionManager_MESSAGE_DISTRIBUTION_HAS_TERMINATED
-              .toLocalizedString(),
-          this.getRootCause());
-    }
-  }
-
-  /**
-   * Returns true if this distribution manager has been closed.
-   */
-  public boolean isClosed() {
-    return this.closed;
-  }
-
-
-  @Override
-  public void addAdminConsole(InternalDistributedMember theId) {
-    logger.info(LocalizedMessage.create(
-        LocalizedStrings.DistributionManager_NEW_ADMINISTRATION_MEMBER_DETECTED_AT_0, theId));
-    synchronized (this.adminConsolesLock) {
-      HashSet tmp = new HashSet(this.adminConsoles);
-      tmp.add(theId);
-      this.adminConsoles = Collections.unmodifiableSet(tmp);
-    }
-  }
-
-  @Override
-  public DMStats getStats() {
-    return this.stats;
-  }
-
-  @Override
-  public DistributionConfig getConfig() {
-    DistributionConfig result = null;
-    InternalDistributedSystem sys = getSystem();
-    if (sys != null) {
-      result = system.getConfig();
-    }
-    return result;
-  }
-
-  @Override
-  public Set getAllOtherMembers() {
-    Set result = new HashSet(getDistributionManagerIdsIncludingAdmin());
-    result.remove(getDistributionManagerId());
-    return result;
-  }
-
-  @Override
-  public void retainMembersWithSameOrNewerVersion(Collection<InternalDistributedMember> members,
-      Version version) {
-    for (Iterator<InternalDistributedMember> it = members.iterator(); it.hasNext();) {
-      InternalDistributedMember id = it.next();
-      if (id.getVersionObject().compareTo(version) < 0) {
-        it.remove();
-      }
-    }
-  }
-
-  @Override
-  public void removeMembersWithSameOrNewerVersion(Collection<InternalDistributedMember> members,
-      Version version) {
-    for (Iterator<InternalDistributedMember> it = members.iterator(); it.hasNext();) {
-      InternalDistributedMember id = it.next();
-      if (id.getVersionObject().compareTo(version) >= 0) {
-        it.remove();
-      }
-    }
-  }
-
-  @Override
-  public Set addAllMembershipListenerAndGetAllIds(MembershipListener l) {
-    MembershipManager mgr = membershipManager;
-    mgr.getViewLock().writeLock().lock();
-    try {
-      synchronized (this.membersLock) {
-        // Don't let the members come and go while we are adding this
-        // listener. This ensures that the listener (probably a
-        // ReplyProcessor) gets a consistent view of the members.
-        addAllMembershipListener(l);
-        return getDistributionManagerIdsIncludingAdmin();
-      }
-    } finally {
-      mgr.getViewLock().writeLock().unlock();
-    }
-  }
-
-  /**
-   * Sends a startup message and waits for a response. Returns true if response received; false if
-   * it timed out or there are no peers.
-   */
-  private boolean sendStartupMessage(StartupOperation startupOperation)
-      throws InterruptedException {
-    if (Thread.interrupted())
-      throw new InterruptedException();
-    this.receivedStartupResponse = false;
-    boolean ok = false;
-
-    // Be sure to add ourself to the equivalencies list!
-    Set equivs = StartupMessage.getMyAddresses(this);
-    if (equivs == null || equivs.size() == 0) {
-      // no network interface
-      equivs = new HashSet();
-      try {
-        equivs.add(SocketCreator.getLocalHost());
-      } catch (UnknownHostException e) {
-        // can't even get localhost
-        if (getViewMembers().size() > 1) {
-          throw new SystemConnectException(
-              "Unable to examine network cards and other members exist");
-        }
-      }
-    }
-    setEquivalentHosts(equivs);
-    setEnforceUniqueZone(getConfig().getEnforceUniqueHost());
-    String redundancyZone = getConfig().getRedundancyZone();
-    if (redundancyZone != null && !redundancyZone.equals("")) {
-      setEnforceUniqueZone(true);
-    }
-    setRedundancyZone(getDistributionManagerId(), redundancyZone);
-    if (logger.isDebugEnabled()) {
-      StringBuffer sb = new StringBuffer();
-      sb.append("Equivalent IPs for this host: ");
-      Iterator it = equivs.iterator();
-      while (it.hasNext()) {
-        InetAddress in = (InetAddress) it.next();
-        sb.append(in.toString());
-        if (it.hasNext()) {
-          sb.append(", ");
-        }
-      } // while
-      logger.debug(sb);
-    }
-
-    // we need to send this to everyone else; even admin vm
-    Set allOthers = new HashSet(getViewMembers());
-    allOthers.remove(getDistributionManagerId());
-
-    if (allOthers.isEmpty()) {
-      return false; // no peers, we are alone.
-    }
-
-    try {
-      ok = startupOperation.sendStartupMessage(allOthers, STARTUP_TIMEOUT, equivs, redundancyZone,
-          enforceUniqueZone());
-    } catch (Exception re) {
-      throw new SystemConnectException(
-          LocalizedStrings.DistributionManager_ONE_OR_MORE_PEERS_GENERATED_EXCEPTIONS_DURING_CONNECTION_ATTEMPT
-              .toLocalizedString(),
-          re);
-    }
-    if (this.rejectionMessage != null) {
-      throw new IncompatibleSystemException(rejectionMessage);
-    }
-
-    boolean isAdminDM = getId().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE
-        || getId().getVmKind() == DistributionManager.LOCATOR_DM_TYPE
-        || DistributionManager.isDedicatedAdminVM()
-        || Boolean.getBoolean(InternalLocator.FORCE_LOCATOR_DM_TYPE);
-
-    boolean receivedAny = this.receivedStartupResponse;
-
-    if (!ok) { // someone didn't reply
-      int unresponsiveCount;
-
-      synchronized (unfinishedStartupsLock) {
-        if (unfinishedStartups == null)
-          unresponsiveCount = 0;
-        else
-          unresponsiveCount = unfinishedStartups.size();
-
-        if (unresponsiveCount != 0) {
-          if (Boolean.getBoolean("DistributionManager.requireAllStartupResponses")) {
-            throw new SystemConnectException(
-                LocalizedStrings.DistributionManager_NO_STARTUP_REPLIES_FROM_0
-                    .toLocalizedString(unfinishedStartups));
-          }
-        }
-      } // synchronized
-
-
-      // Bug 35887:
-      // If there are other members, we must receive at least _one_ response
-      if (allOthers.size() != 0) { // there exist others
-        if (!receivedAny) { // and none responded
-          StringBuffer sb = new StringBuffer();
-          Iterator itt = allOthers.iterator();
-          while (itt.hasNext()) {
-            Object m = itt.next();
-            sb.append(m.toString());
-            if (itt.hasNext())
-              sb.append(", ");
-          }
-          if (DEBUG_NO_ACKNOWLEDGEMENTS) {
-            printStacks(allOthers, false);
-          }
-          throw new SystemConnectException(
-              LocalizedStrings.DistributionManager_RECEIVED_NO_CONNECTION_ACKNOWLEDGMENTS_FROM_ANY_OF_THE_0_SENIOR_CACHE_MEMBERS_1
-                  .toLocalizedString(
-                      new Object[] {Integer.toString(allOthers.size()), sb.toString()}));
-        } // and none responded
-      } // there exist others
-
-      InternalDistributedMember e = getElderId();
-      if (e != null) { // an elder exists
-        boolean unresponsiveElder;
-        synchronized (unfinishedStartupsLock) {
-          if (unfinishedStartups == null)
-            unresponsiveElder = false;
-          else
-            unresponsiveElder = unfinishedStartups.contains(e);
-        }
-        if (unresponsiveElder) {
-          logger.warn(LocalizedMessage.create(
-              LocalizedStrings.DistributionManager_FORCING_AN_ELDER_JOIN_EVENT_SINCE_A_STARTUP_RESPONSE_WAS_NOT_RECEIVED_FROM_ELDER__0_,
-              e));
-          handleManagerStartup(e);
-        }
-      } // an elder exists
-    } // someone didn't reply
-    return receivedAny;
-  }
-
-  /**
-   * List of InternalDistributedMember's that we have not received startup replies from. If null, we
-   * have not finished sending the startup message.
-   * <p>
-   * Must be synchronized using {@link #unfinishedStartupsLock}
-   */
-  private Set unfinishedStartups = null;
-
-  /**
-   * Synchronization for {@link #unfinishedStartups}
-   */
-  private final Object unfinishedStartupsLock = new Object();
-
-  @Override
-  public void setUnfinishedStartups(Collection s) {
-    synchronized (unfinishedStartupsLock) {
-      Assert.assertTrue(unfinishedStartups == null, "Set unfinished startups twice");
-      unfinishedStartups = new HashSet(s);
-
-      // OK, I don't _quite_ trust the list to be current, so let's
-      // prune it here.
-      Iterator it = unfinishedStartups.iterator();
-      synchronized (this.membersLock) {
-        while (it.hasNext()) {
-          InternalDistributedMember m = (InternalDistributedMember) it.next();
-          if (!isCurrentMember(m)) {
-            it.remove();
-          }
-        } // while
-      } // synchronized
-    }
-  }
-
-  @Override
-  public void removeUnfinishedStartup(InternalDistributedMember m, boolean departed) {
-    synchronized (unfinishedStartupsLock) {
-      if (logger.isDebugEnabled()) {
-        logger.debug("removeUnfinishedStartup for {} wtih {}", m, unfinishedStartups);
-      }
-      if (unfinishedStartups == null)
-        return; // not yet done with startup
-      if (!unfinishedStartups.remove(m))
-        return;
-      StringId msg = null;
-      if (departed) {
-        msg =
-            LocalizedStrings.DistributionManager_STOPPED_WAITING_FOR_STARTUP_REPLY_FROM_0_BECAUSE_THE_PEER_DEPARTED_THE_VIEW;
-      } else {
-        msg =
-            LocalizedStrings.DistributionManager_STOPPED_WAITING_FOR_STARTUP_REPLY_FROM_0_BECAUSE_THE_REPLY_WAS_FINALLY_RECEIVED;
-      }
-      logger.info(LocalizedMessage.create(msg, m));
-      int numLeft = unfinishedStartups.size();
-      if (numLeft != 0) {
-        logger.info(LocalizedMessage.create(
-            LocalizedStrings.DistributionManager_STILL_AWAITING_0_RESPONSES_FROM_1,
-            new Object[] {Integer.valueOf(numLeft), unfinishedStartups}));
-      }
-    } // synchronized
-  }
-
-  /**
-   * Processes the first startup response.
-   *
-   * @see StartupResponseMessage#process
-   */
-  void processStartupResponse(InternalDistributedMember sender, String theRejectionMessage) {
-    removeUnfinishedStartup(sender, false);
-    synchronized (this) {
-      if (!this.receivedStartupResponse) {
-        // only set the cacheTimeDelta once
-        this.receivedStartupResponse = true;
-      }
-      if (theRejectionMessage != null && this.rejectionMessage == null) {
-        // remember the first non-null rejection. This fixes bug 33266
-        this.rejectionMessage = theRejectionMessage;
-      }
-    }
-  }
-
-  /**
-   * Based on a recent JGroups view, return a member that might be the next elder.
-   *
-   * @return the elder candidate, possibly this VM.
-   */
-  private InternalDistributedMember getElderCandidate() {
-    List<InternalDistributedMember> theMembers = getViewMembers();
-
-    // Assert.assertTrue(!closeInProgress
-    // && theMembers.contains(this.localAddress)); // bug36202?
-
-    int elderCandidates = 0;
-    Iterator<InternalDistributedMember> it;
-
-    // for bug #50510 we need to know if there are any members older than v8.0
-    it = theMembers.iterator();
-    boolean anyPre80Members = false;
-    while (it.hasNext()) {
-      InternalDistributedMember member = it.next();
-      if (member.getVersionObject().compareTo(Version.GFE_80) < 0) {
-        anyPre80Members = true;
-      }
-    }
-
-    // determine number of elder candidates (unless adam)
-    if (!this.adam) {
-      it = theMembers.iterator();
-      while (it.hasNext()) {
-        InternalDistributedMember member = it.next();
-        int managerType = member.getVmKind();
-        if (managerType == ADMIN_ONLY_DM_TYPE)
-          continue;
-
-        if (managerType == LOCATOR_DM_TYPE) {
-          // Fix for #50510 - pre-8.0 members will not let a locator be the elder
-          // so we need to make the same decision here
-          if (anyPre80Members) {
-            continue;
-          }
-        }
-
-        // Fix for #45566. Using a surprise member as the elder can cause a
-        // deadlock.
-        if (getMembershipManager().isSurpriseMember(member)) {
-          continue;
-        }
-
-        elderCandidates++;
-        if (elderCandidates > 1) {
-          // If we have more than one candidate then we are not adam
-          break;
-        }
-      } // while
-    }
-
-    // Second pass over members...
-    it = theMembers.iterator();
-    while (it.hasNext()) {
-      InternalDistributedMember member = it.next();
-      int managerType = member.getVmKind();
-      if (managerType == ADMIN_ONLY_DM_TYPE)
-        continue;
-
-      if (managerType == LOCATOR_DM_TYPE) {
-        // Fix for #50510 - pre-8.0 members will not let a locator be the elder
-        // so we need to make the same decision here
-        if (anyPre80Members) {
-          continue;
-        }
-      }
-
-      // Fix for #45566. Using a surprise member as the elder can cause a
-      // deadlock.
-      if (getMembershipManager().isSurpriseMember(member)) {
-        continue;
-      }
-
-      if (member.equals(this.localAddress)) {
-        if (!this.adam && elderCandidates == 1) {
-          this.adam = true;
-          logger.info(LocalizedMessage.create(
-              LocalizedStrings.DistributionManager_0_IS_THE_ELDER_AND_THE_ONLY_MEMBER,
-              this.localAddress));
-        } else {
-          logger.info(LocalizedMessage.create(LocalizedStrings.DistributionManager_I_0_AM_THE_ELDER,
-              this.localAddress));
-        }
-      }
-      return member;
-    } // while
-    // If we get this far then no elder exists
-    return null;
-  }
-
-  /**
-   * Select a new elder
-   *
-   */
-  private void selectElder() {
-    getSystem().getCancelCriterion().checkCancelInProgress(null); // bug 37884, if DS is
-                                                                  // disconnecting, throw exception
-
-    // Once we are the elder, we're stuck until we leave the view.
-    if (this.localAddress.equals(this.elder)) {
-      return;
-    }
-
-    // Determine who is the elder...
-    InternalDistributedMember candidate = getElderCandidate();
-    if (candidate == null) {
-      changeElder(null);
-      return; // No valid elder in current context
-    }
-
-    // Carefully switch to new elder
-    synchronized (this.elderMonitor) {
-      if (!candidate.equals(this.elder)) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("The elder is: {} (was {})", candidate, this.elder);
-        }
-        changeElder(candidate);
-      }
-    } // synchronized
-  }
-
-  private String prettifyReason(String r) {
-    final String str = "java.io.IOException:";
-    if (r.startsWith(str)) {
-      return r.substring(str.length());
-    }
-    return r;
-  }
-
-  /**
-   * Returns true if id was removed. Returns false if it was not in the list of managers.
-   */
-  private boolean removeManager(InternalDistributedMember theId, boolean crashed, String p_reason) {
-    String reason = p_reason;
-    boolean result = false; // initialization shouldn't be required, but...
-
-    // Test once before acquiring the lock, fault tolerance for potentially
-    // recursive (and deadlock) conditions -- bug33626
-    // Note that it is always safe to _read_ {@link members} without locking
-    if (isCurrentMember(theId)) {
-      // Destroy underlying member's resources
-      reason = prettifyReason(reason);
-      synchronized (this.membersLock) {
-        if (logger.isDebugEnabled()) {
-          logger.debug("DistributionManager: removing member <{}>; crashed {}; reason = {}", theId,
-              crashed, reason);
-        }
-        Map<InternalDistributedMember, InternalDistributedMember> tmp = new HashMap(this.members);
-        if (tmp.remove(theId) != null) {
-          // Note we don't modify in place. This allows reader to get snapshots
-          // without locking.
-          if (tmp.isEmpty()) {
-            tmp = Collections.EMPTY_MAP;
-          } else {
-            tmp = Collections.unmodifiableMap(tmp);
-          }
-          this.members = tmp;
-          result = true;
-
-        } else {
-          result = false;
-          // Don't get upset since this can happen twice due to
-          // an explicit remove followed by an implicit one caused
-          // by a JavaGroup view change
-        }
-        Set tmp2 = new HashSet(this.membersAndAdmin);
-        if (tmp2.remove(theId)) {
-          if (tmp2.isEmpty()) {
-            tmp2 = Collections.EMPTY_SET;
-          } else {
-            tmp2 = Collections.unmodifiableSet(tmp2);
-          }
-          this.membersAndAdmin = tmp2;
-        }
-        this.removeHostedLocators(theId);
-      } // synchronized
-    } // if
-
-    // In any event, make sure that this member is no longer an elder.
-    if (!theId.equals(localAddress) && theId.equals(elder)) {
-      try {
-        selectElder();
-      } catch (DistributedSystemDisconnectedException e) {
-        // ignore
-      }
-    }
-
-    redundancyZones.remove(theId);
-
-    return result;
-  }
-
-  /**
-   * Makes note of a new distribution manager that has started up in the distributed cache. Invokes
-   * the appropriately listeners.
-   *
-   * @param theId The id of the distribution manager starting up
-   *
-   */
-  private void handleManagerStartup(InternalDistributedMember theId) {
-    HashMap<InternalDistributedMember, InternalDistributedMember> tmp = null;
-    synchronized (this.membersLock) {
-      // Note test is under membersLock
-      if (members.containsKey(theId)) {
-        return; // already accounted for
-      }
-
-      // Note we don't modify in place. This allows reader to get snapshots
-      // without locking.
-      tmp = new HashMap(this.members);
-      tmp.put(theId, theId);
-      this.members = Collections.unmodifiableMap(tmp);
-
-      Set stmp = new HashSet(this.membersAndAdmin);
-      stmp.add(theId);
-      this.membersAndAdmin = Collections.unmodifiableSet(stmp);
-    } // synchronized
-
-    if (theId.getVmKind() != DistributionManager.LOCATOR_DM_TYPE) {
-      this.stats.incNodes(1);
-    }
-    logger.info(LocalizedMessage.create(
-        LocalizedStrings.DistributionManager_ADMITTING_MEMBER_0_NOW_THERE_ARE_1_NONADMIN_MEMBERS,
-        new Object[] {theId, Integer.valueOf(tmp.size())}));
-    addMemberEvent(new MemberJoinedEvent(theId));
-  }
-
-  @Override
-  public boolean isCurrentMember(InternalDistributedMember id) {
-    Set m;
-    synchronized (this.membersLock) {
-      // access to members synchronized under membersLock in order to
-      // ensure serialization
-      m = this.membersAndAdmin;
-    }
-    return m.contains(id);
-  }
-
-  /**
-   * Makes note of a new console that has started up in the distributed cache.
-   *
-   */
-  private void handleConsoleStartup(InternalDistributedMember theId) {
-    // if we have an all listener then notify it NOW.
-    HashSet tmp = null;
-    synchronized (this.membersLock) {
-      // Note test is under membersLock
-      if (membersAndAdmin.contains(theId))
-        return; // already accounted for
-
-      // Note we don't modify in place. This allows reader to get snapshots
-      // without locking.
-      tmp = new HashSet(this.membersAndAdmin);
-      tmp.add(theId);
-      this.membersAndAdmin = Collections.unmodifiableSet(tmp);
-    } // synchronized
-
-    for (Iterator iter = allMembershipListeners.iterator(); iter.hasNext();) {
-      MembershipListener listener = (MembershipListener) iter.next();
-      listener.memberJoined(theId);
-    }
-    logger.info(LocalizedMessage.create(
-        LocalizedStrings.DistributionManager_DMMEMBERSHIP_ADMITTING_NEW_ADMINISTRATION_MEMBER__0_,
-        theId));
-    // Note that we don't add the member to the list of admin consoles until
-    // we receive a message from them.
-  }
-
-  /**
-   * Process an incoming distribution message. This includes scheduling it correctly based on the
-   * message's nioPriority (executor type)
-   */
-  private void handleIncomingDMsg(DistributionMessage message) {
-    stats.incReceivedMessages(1L);
-    stats.incReceivedBytes(message.getBytesRead());
-    stats.incMessageChannelTime(message.resetTimestamp());
-
-    if (logger.isDebugEnabled()) {
-      logger.debug("Received message '{}' from <{}>", message, message.getSender());
-    }
-    scheduleIncomingMessage(message);
-  }
-
-  /**
-   * Makes note of a console that has shut down.
-   *
-   * @param theId The id of the console shutting down
-   * @param crashed only true if we detect this id to be gone from a javagroup view
-   *
-   * @see AdminConsoleDisconnectMessage#process
-   */
-  public void handleConsoleShutdown(InternalDistributedMember theId, boolean crashed,
-      String reason) {
-    boolean removedConsole = false;
-    boolean removedMember = false;
-    synchronized (this.membersLock) {
-      // to fix bug 39747 we can only remove this member from
-      // membersAndAdmin if he is not in members.
-      // This happens when we have an admin guy colocated with a normal DS.
-      // In this case we need for the normal DS to shutdown or crash.
-      if (!this.members.containsKey(theId)) {
-        if (logger.isDebugEnabled())
-          logger.debug("DistributionManager: removing admin member <{}>; crashed = {}; reason = {}",
-              theId, crashed, reason);
-        Set tmp = new HashSet(this.membersAndAdmin);
-        if (tmp.remove(theId)) {
-          // Note we don't modify in place. This allows reader to get snapshots
-          // without locking.
-          if (tmp.isEmpty()) {
-            tmp = Collections.EMPTY_SET;
-          } else {
-            tmp = Collections.unmodifiableSet(tmp);
-          }
-          this.membersAndAdmin = tmp;
-          removedMember = true;
-        } else {
-          // Don't get upset since this can happen twice due to
-          // an explicit remove followed by an implicit one caused
-          // by a JavaGroup view change
-        }
-      }
-      removeHostedLocators(theId);
-    }
-    synchronized (this.adminConsolesLock) {
-      if (this.adminConsoles.contains(theId)) {
-        removedConsole = true;
-        Set tmp = new HashSet(this.adminConsoles);
-        tmp.remove(theId);
-        if (tmp.isEmpty()) {
-          tmp = Collections.EMPTY_SET;
-        } else {
-          tmp = Collections.unmodifiableSet(tmp);
-        }
-        this.adminConsoles = tmp;
-      }
-    }
-    if (removedMember) {
-      for (Iterator iter = allMembershipListeners.iterator(); iter.hasNext();) {
-        MembershipListener listener = (MembershipListener) iter.next();
-        listener.memberDeparted(theId, crashed);
-      }
-    }
-    if (removedConsole) {
-      StringId msg = null;
-      if (crashed) {
-        msg = LocalizedStrings.DistributionManager_ADMINISTRATION_MEMBER_AT_0_CRASHED_1;
-      } else {
-        msg = LocalizedStrings.DistributionManager_ADMINISTRATION_MEMBER_AT_0_CLOSED_1;
-      }
-      logger.info(LocalizedMessage.create(msg, new Object[] {theId, reason}));
-    }
-
-    redundancyZones.remove(theId);
-  }
-
-  public void shutdownMessageReceived(InternalDistributedMember theId, String reason) {
-    this.membershipManager.shutdownMessageReceived(theId, reason);
-    handleManagerDeparture(theId, false,
-        LocalizedStrings.ShutdownMessage_SHUTDOWN_MESSAGE_RECEIVED.toLocalizedString());
-  }
-
-  @Override
-  public void handleManagerDeparture(InternalDistributedMember theId, boolean p_crashed,
-      String p_reason) {
-    boolean crashed = p_crashed;
-    String reason = p_reason;
-
-    AlertAppender.getInstance().removeAlertListener(theId);
-
-    // this fixes a race introduced in 5.0.1 by the fact that an explicit
-    // shutdown will cause a member to no longer be in our DM membership
-    // but still in the javagroup view.
-    try {
-      selectElder();
-    } catch (DistributedSystemDisconnectedException e) {
-      // keep going
-    }
-
-
-
-    int vmType = theId.getVmKind();
-    if (vmType == ADMIN_ONLY_DM_TYPE) {
-      removeUnfinishedStartup(theId, true);
-      handleConsoleShutdown(theId, crashed, reason);
-      return;
-    }
-
-    // not an admin VM...
-    if (!isCurrentMember(theId)) {
-      return; // fault tolerance
-    }
-    removeUnfinishedStartup(theId, true);
-
-    if (removeManager(theId, crashed, reason)) {
-      if (theId.getVmKind() != DistributionManager.LOCATOR_DM_TYPE) {
-        this.stats.incNodes(-1);
-      }
-      StringId msg;
-      if (crashed && !isCloseInProgress()) {
-        msg =
-            LocalizedStrings.DistributionManager_MEMBER_AT_0_UNEXPECTEDLY_LEFT_THE_DISTRIBUTED_CACHE_1;
-        addMemberEvent(new MemberCrashedEvent(theId, reason));
-      } else {
-        msg =
-            LocalizedStrings.DistributionManager_MEMBER_AT_0_GRACEFULLY_LEFT_THE_DISTRIBUTED_CACHE_1;
-        addMemberEvent(new MemberDepartedEvent(theId, reason));
-      }
-      logger.info(LocalizedMessage.create(msg, new Object[] {theId, prettifyReason(reason)}));
-
-      // Remove this manager from the serialQueueExecutor.
-      if (this.serialQueuedExecutorPool != null) {
-        serialQueuedExecutorPool.handleMemberDeparture(theId);
-      }
-    }
-  }
-
-  private void handleManagerSuspect(InternalDistributedMember suspect,
-      InternalDistributedMember whoSuspected, String reason) {
-    if (!isCurrentMember(suspect)) {
-      return; // fault tolerance
-    }
-
-    int vmType = suspect.getVmKind();
-    if (vmType == ADMIN_ONLY_DM_TYPE) {
-      return;
-    }
-
-    addMemberEvent(new MemberSuspectEvent(suspect, whoSuspected, reason));
-  }
-
-  private void handleViewInstalled(NetView view) {
-    addMemberEvent(new ViewInstalledEvent(view));
-  }
-
-  private void handleQuorumLost(Set<InternalDistributedMember> failures,
-      List<InternalDistributedMember> remaining) {
-    addMemberEvent(new QuorumLostEvent(failures, remaining));
-  }
-
-  /**
-   * Sends the shutdown message. Not all DistributionManagers need to do this.
-   */
-  private void sendShutdownMessage() {
-    if (getDMType() == ADMIN_ONLY_DM_TYPE && Locator.getLocators().size() == 0) {
-      // [bruce] changed above "if" to have ShutdownMessage sent by locators.
-      // Otherwise the system can hang because an admin member does not trigger
-      // member-left notification unless a new view is received showing the departure.
-      // If two locators are simultaneously shut down this may not occur.
-      return;
-    }
-
-    ShutdownMessage m = new ShutdownMessage();
-    InternalDistributedMember theId = this.getDistributionManagerId();
-    m.setDistributionManagerId(theId);
-    Set allOthers = new HashSet(getViewMembers());
-    allOthers.remove(getDistributionManagerId());
-    m.setRecipients(allOthers);
-
-    // Address recipient = (Address) m.getRecipient();
-    if (logger.isTraceEnabled()) {
-      logger.trace("{} Sending {} to {}", this.getDistributionManagerId(), m,
-          m.getRecipientsDescription());
-    }
-
-    try {
-      // m.resetTimestamp(); // nanotimers across systems don't match
-      long startTime = DistributionStats.getStatTime();
-      sendViaMembershipManager(m.getRecipients(), m, this, stats);
-      this.stats.incSentMessages(1L);
-      if (DistributionStats.enableClockStats) {
-        stats.incSentMessagesTime(DistributionStats.getStatTime() - startTime);
-      }
-    } catch (CancelException e) {
-      logger.debug("CancelException caught sending shutdown: {}", e.getMessage(), e);
-    } catch (Exception ex2) {
-      logger.fatal(LocalizedMessage
-          .create(LocalizedStrings.DistributionManager_WHILE_SENDING_SHUTDOWN_MESSAGE), ex2);
-    } finally {
-      // Even if the message wasn't sent, *lie* about it, so that
-      // everyone believes that message distribution is done.
-      this.shutdownMsgSent = true;
-    }
-  }
-
-  /**
-   * Returns the executor for the given type of processor.
-   */
-  public Executor getExecutor(int processorType, InternalDistributedMember sender) {
-    switch (processorType) {
-      case STANDARD_EXECUTOR:
-        return getThreadPool();
-      case SERIAL_EXECUTOR:
-        return getSerialExecutor(sender);
-      case VIEW_EXECUTOR:
-        return this.viewThread;
-      case HIGH_PRIORITY_EXECUTOR:
-        return getHighPriorityThreadPool();
-      case WAITING_POOL_EXECUTOR:
-        return getWaitingThreadPool();
-      case PARTITIONED_REGION_EXECUTOR:
-        return getPartitionedRegionExcecutor();
-      case REGION_FUNCTION_EXECUTION_EXECUTOR:
-        return getFunctionExcecutor();
-      default:
-        throw new InternalGemFireError(LocalizedStrings.DistributionManager_UNKNOWN_PROCESSOR_TYPE
-            .toLocalizedString(processorType));
-    }
-  }
-
-  /**
-   * Actually does the work of sending a message out over the distribution channel.
-   *
-   * @param message the message to send
-   * @return list of recipients that did not receive the message because they left the view (null if
-   *         all received it or it was sent to {@link DistributionMessage#ALL_RECIPIENTS}.
-   * @throws NotSerializableException If <code>message</code> cannot be serialized
-   */
-  Set sendOutgoing(DistributionMessage message) throws NotSerializableException {
-    long startTime = DistributionStats.getStatTime();
-
-    Set result = sendViaMembershipManager(message.getRecipients(), message,
-        DistributionManager.this, this.stats);
-    long endTime = 0L;
-    if (DistributionStats.enableClockStats) {
-      endTime = NanoTimer.getTime();
-    }
-    boolean sentToAll = message.forAll();
-
-    if (sentToAll) {
-      stats.incBroadcastMessages(1L);
-      if (DistributionStats.enableClockStats) {
-        stats.incBroadcastMessagesTime(endTime - startTime);
-      }
-    }
-    stats.incSentMessages(1L);
-    if (DistributionStats.enableClockStats) {
-      stats.incSentMessagesTime(endTime - startTime);
-      stats.incDistributeMessageTime(endTime - message.getTimestamp());
-    }
-
-    return result;
-  }
-
-
-
-  /**
-   * @return recipients who did not receive the message
-   * @throws NotSerializableException If <codE>message</code> cannot be serialized
-   */
-  Set sendMessage(DistributionMessage message) throws NotSerializableException {
-    Set result = null;
-    try {
-      // Verify we're not too far into the shutdown
-      stopper.checkCancelInProgress(null);
-
-      // avoid race condition during startup
-      waitUntilReadyToSendMsgs(message);
-
-      result = sendOutgoing(message);
-    } catch (NotSerializableException | ToDataException | ReenteredConnectException
-        | InvalidDeltaException | CancelException ex) {
-      throw ex;
-    } catch (Exception ex) {
-      DistributionManager.this.exceptionInThreads = true;
-      String receiver = "NULL";
-      if (message != null) {
-        receiver = message.getRecipientsDescription();
-      }
-
-      logger.fatal(
-          LocalizedMessage.create(LocalizedStrings.DistributionManager_WHILE_PUSHING_MESSAGE_0_TO_1,
-              new Object[] {message, receiver}),
-          ex);
-      if (message == null || message.forAll())
-        return null;
-      result = new HashSet();
-      for (int i = 0; i < message.getRecipients().length; i++)
-        result.add(message.getRecipients()[i]);
-      return result;
-      /*
-       * if (ex instanceof org.apache.geode.GemFireIpcResourceException) { return; }
-       */
-    }
-    return result;
-  }
-
-  /**
-   * @return list of recipients who did not receive the message because they left the view (null if
-   *         all received it or it was sent to {@link DistributionMessage#ALL_RECIPIENTS}).
-   * @throws NotSerializableException If content cannot be serialized
-   */
-  private Set sendViaMembershipManager(InternalDistributedMember[] destinations,
-      DistributionMessage content, DistributionManager dm, DistributionStats stats)
-      throws NotSerializableException {
-    if (membershipManager == null) {
-      logger.warn(LocalizedMessage.create(
-          LocalizedStrings.DistributionChannel_ATTEMPTING_A_SEND_TO_A_DISCONNECTED_DISTRIBUTIONMANAGER));
-      if (destinations.length == 1 && destinations[0] == DistributionMessage.ALL_RECIPIENTS)
-        return null;
-      HashSet result = new HashSet();
-      for (int i = 0; i < destinations.length; i++) {
-        result.add(destinations[i]);
-      }
-      return result;
-    }
-    return membershipManager.send(destinations, content, stats);
-  }
-
-
-  /**
-   * Schedule a given message appropriately, depending upon its executor kind.
-   */
-  void scheduleIncomingMessage(DistributionMessage message) {
-    /*
-     * Potential race condition between starting up and getting other distribution manager ids -- DM
-     * will only be initialized upto the point at which it called startThreads
-     */
-    waitUntilReadyForMessages();
-    message.schedule(DistributionManager.this);
-  }
-
-  @Override
-  public boolean isAdam() {
-    return this.adam;
-  }
-
-  @Override
-  public InternalDistributedMember getElderId() throws DistributedSystemDisconnectedException {
-    if (closeInProgress) {
-      throw new DistributedSystemDisconnectedException(
-          LocalizedStrings.DistributionManager_NO_VALID_ELDER_WHEN_SYSTEM_IS_SHUTTING_DOWN
-              .toLocalizedString(),
-          this.getRootCause());
-    }
-    getSystem().getCancelCriterion().checkCancelInProgress(null);
-
-    // Cache a recent value of the elder
-    InternalDistributedMember result = elder;
-    if (result != null && membershipManager.memberExists(result)) {
-      return result;
-    }
-    logger.info(LocalizedMessage.create(
-        LocalizedStrings.DistributionManager_ELDER__0__IS_NOT_CURRENTLY_AN_ACTIVE_MEMBER_SELECTING_NEW_ELDER,
-        elder));
-
-    selectElder(); // ShutdownException can be thrown here
-    logger.info(LocalizedMessage
-        .create(LocalizedStrings.DistributionManager_NEWLY_SELECTED_ELDER_IS_NOW__0_, elder));
-    return elder;
-  }
-
-  @Override
-  public boolean isElder() {
-    return getId().equals(elder);
-  }
-
-  @Override
-  public boolean isLoner() {
-    return false;
-  }
-
-  private final StoppableReentrantLock elderLock;
-  private ElderState elderState;
-  private volatile boolean elderStateInitialized;
-
-  @Override
-  public ElderState getElderState(boolean force, boolean useTryLock) {
-    if (force) {
-      if (logger.isDebugEnabled()) {
-        if (!this.localAddress.equals(this.elder)) {
-          logger.debug("Forcing myself, {}, to be the elder.", this.localAddress);
-        }
-      }
-      changeElder(this.localAddress);
-    }
-    if (force || this.localAddress.equals(elder)) {
-      // we are the elder
-      if (this.elderStateInitialized) {
-        return this.elderState;
-      }
-      return getElderStateWithTryLock(useTryLock);
-    } else {
-      // we are not the elder so return null
-      return null;
-    }
-  }
-
-  /**
-   * Usage: GrantorRequestProcessor calls getElderState with useTryLock set to true if the
-   * becomeGrantor Collaboration is already acquired.
-   * <p>
-   * This tryLock is attempted and if it fails, an exception is thrown to cause a Doug Lea style
-   * back-off (p. 149). It throws an exception because it needs to back down a couple of packages
-   * and I didn't want to couple this pkg too tightly with the dlock pkg.
-   * <p>
-   * GrantorRequestProcessor catches the exception, releases and reacquires the Collaboration, and
-   * then comes back here to attempt the tryLock again. Currently nothing will stop it from
-   * re-attempting forever. It has to get the ElderState and cannot give up, but it can free up the
-   * Collaboration and then re-enter it. The other thread holding the elder lock will hold it only
-   * briefly. I've added a volatile called elderStateInitialized which should cause this back-off to
-   * occur only once in the life of a vm... once the elder, always the elder.
-   * <p>
-   */
-  private ElderState getElderStateWithTryLock(boolean useTryLock) {
-    boolean locked = false;
-    if (useTryLock) {
-      boolean interrupted = Thread.interrupted();
-      try {
-        locked = this.elderLock.tryLock(2000);
-      } catch (InterruptedException e) {
-        interrupted = true;
-        getCancelCriterion().checkCancelInProgress(e);
-        // one last attempt and then allow it to fail for back-off...
-        locked = this.elderLock.tryLock();
-      } finally {
-        if (interrupted) {
-          Thread.currentThread().interrupt();
-        }
-      }
-    } else {
-      locked = true;
-      this.elderLock.lock();
-    }
-    if (!locked) {
-      // try-lock must have failed
-      throw new IllegalStateException(
-          LocalizedStrings.DistributionManager_POSSIBLE_DEADLOCK_DETECTED.toLocalizedString());
-    }
-    try {
-      if (this.elderState == null) {
-        this.elderState = new ElderState(this);
-      }
-    } finally {
-      this.elderLock.unlock();
-    }
-    this.elderStateInitialized = true;
-    // if (Thread.currentThread().isInterrupted())
-    // throw new RuntimeException("Interrupted");
-    return this.elderState;
-  }
-
-  /**
-   * Waits until elder if newElder or newElder is no longer a member
-   *
-   * @return true if newElder is the elder; false if he is no longer a member or we are the elder.
-   */
-  public boolean waitForElder(final InternalDistributedMember desiredElder) {
-    MembershipListener l = null;
-    try {
-      // Assert.assertTrue(
-      // desiredElder.getVmKind() != DistributionManager.ADMIN_ONLY_DM_TYPE);
-      synchronized (this.elderMonitor) {
-        while (true) {
-          if (closeInProgress)
-            return false;
-          InternalDistributedMember currentElder = this.elder;
-          // Assert.assertTrue(
-          // currentElder.getVmKind() != DistributionManager.ADMIN_ONLY_DM_TYPE);
-          if (desiredElder.equals(currentElder)) {
-            return true;
-          }
-          if (!isCurrentMember(desiredElder)) {
-            return false; // no longer present
-          }
-          if (this.localAddress.equals(currentElder)) {
-            // Once we become the elder we no longer allow anyone else to be the
-            // elder so don't let them wait anymore.
-            return false;
-          }
-          if (l == null) {
-            l = new MembershipListener() {
-              public void memberJoined(InternalDistributedMember theId) {
-                // nothing needed
-              }
-
-              public void memberDeparted(InternalDistributedMember theId, boolean crashed) {
-                if (desiredElder.equals(theId)) {
-                  notifyElderChangeWaiters();
-                }
-              }
-
-              public void memberSuspect(InternalDistributedMember id,
-                  InternalDistributedMember whoSuspected, String reason) {}
-
-              public void viewInstalled(NetView view) {}
-
-              public void quorumLost(Set<InternalDistributedMember> failures,
-                  List<InternalDistributedMember> remaining) {}
-            };
-            addMembershipListener(l);
-          }
-          logger.info(LocalizedMessage.create(
-              LocalizedStrings.DistributionManager_CHANGING_ELDER_FROM_0_TO_1,
-              new Object[] {currentElder, desiredElder}));
-          elderChangeWait();
-        } // while true
-      }
-    } finally {
-      if (l != null) {
-        removeMembershipListener(l);
-      }
-    }
-  }
-
-  /**
-   * Set the elder to newElder and notify anyone waiting for it to change
-   */
-  private void changeElder(InternalDistributedMember newElder) {
-    synchronized (this.elderMonitor) {
-      if (newElder != null && this.localAddress != null && !this.localAddress.equals(newElder)) {
-        if (this.localAddress.equals(this.elder)) {
-          // someone else changed the elder while this thread was off cpu
-          if (logger.isDebugEnabled()) {
-            logger.debug("changeElder found this VM to be the elder and is taking an early out");
-          }
-          return;
-        }
-      }
-      this.elder = newElder;
-      if (this.waitingForElderChange) {
-        this.waitingForElderChange = false;
-        this.elderMonitor.notifyAll();
-      }
-    }
-  }
-
-  /**
-   * Used to wakeup someone in elderChangeWait even though the elder has not changed
-   */
-  private void notifyElderChangeWaiters() {
-    synchronized (this.elderMonitor) {
-      if (this.waitingForElderChange) {
-        this.waitingForElderChange = false;
-        this.elderMonitor.notifyAll();
-      }
-    }
-  }
-
-  /**
-   * Must be called holding {@link #elderMonitor} lock
-   */
-  private void elderChangeWait() {
-    // This is OK since we're holding the elderMonitor lock, so no
-    // new events will come through until the wait() below.
-    this.waitingForElderChange = true;
-
-    while (this.waitingForElderChange) {
-      stopper.checkCancelInProgress(null);
-      boolean interrupted = Thread.interrupted();
-      try {
-        this.elderMonitor.wait();
-        break;
-      } catch (InterruptedException ignore) {
-        interrupted = true;
-      } finally {
-        if (interrupted) {
-          Thread.currentThread().interrupt();
-        }
-      }
-    } // while
-  }
-
-  @Override
-  public ExecutorService getThreadPool() {
-    return this.threadPool;
-  }
-
-  @Override
-  public ExecutorService getHighPriorityThreadPool() {
-    return this.highPriorityPool;
-  }
-
-  @Override
-  public ExecutorService getWaitingThreadPool() {
-    return this.waitingPool;
-  }
-
-  @Override
-  public ExecutorService getPrMetaDataCleanupThreadPool() {
-    return this.prMetaDataCleanupThreadPool;
-  }
-
-  private Executor getPartitionedRegionExcecutor() {
-    if (this.partitionedRegionThread != null) {
-      return this.partitionedRegionThread;
-    } else {
-      return this.partitionedRegionPool;
-    }
-  }
-
-
-  @Override
-  public Executor getFunctionExcecutor() {
-    if (this.functionExecutionThread != null) {
-      return this.functionExecutionThread;
-    } else {
-      return this.functionExecutionPool;
-    }
-  }
-
-  private Executor getSerialExecutor(InternalDistributedMember sender) {
-    if (MULTI_SERIAL_EXECUTORS) {
-      return this.serialQueuedExecutorPool.getThrottledSerialExecutor(sender);
-    } else {
-      return this.serialThread;
-    }
-  }
-
-  /** returns the serialThread's queue if throttling is being used, null if not */
-  public OverflowQueueWithDMStats getSerialQueue(InternalDistributedMember sender) {
-    if (MULTI_SERIAL_EXECUTORS) {
-      return this.serialQueuedExecutorPool.getSerialQueue(sender);
-    } else {
-      return this.serialQueue;
-    }
-  }
-
-  /**
-   * Sets the administration agent associated with this distribution manager.
-   */
-  public void setAgent(RemoteGfManagerAgent agent) {
-    // Don't let the agent be set twice. There should be a one-to-one
-    // correspondence between admin agent and distribution manager.
-    if (agent != null) {
-      if (this.agent != null) {
-        throw new IllegalStateException(
-            LocalizedStrings.DistributionManager_THERE_IS_ALREADY_AN_ADMIN_AGENT_ASSOCIATED_WITH_THIS_DISTRIBUTION_MANAGER
-                .toLocalizedString());
-      }
-
-    } else {
-      if (this.agent == null) {
-        throw new IllegalStateException(
-            LocalizedStrings.DistributionManager_THERE_WAS_NEVER_AN_ADMIN_AGENT_ASSOCIATED_WITH_THIS_DISTRIBUTION_MANAGER
-                .toLocalizedString());
-      }
-    }
-    this.agent = agent;
-  }
-
-  /**
-   * Returns the agent that owns this distribution manager. (in ConsoleDistributionManager)
-   */
-  public RemoteGfManagerAgent getAgent() {
-    return this.agent;
-  }
-
-  /**
-   * Returns a description of the distribution configuration used for this distribution manager. (in
-   * ConsoleDistributionManager)
-   *
-   * @return <code>null</code> if no admin {@linkplain #getAgent agent} is associated with this
-   *         distribution manager
-   */
-  public String getDistributionConfigDescription() {
-    if (this.agent == null) {
-      return null;
-
-    } else {
-      return this.agent.getTransport().toString();
-    }
-  }
-
-  /* -----------------------------Health Monitor------------------------------ */
-  private final ConcurrentMap hmMap = new ConcurrentHashMap();
-
-  private volatile InternalCache cache;
-
-  /**
-   * Returns the health monitor for this distribution manager and owner.
-   *
-   * @param owner the agent that owns the returned monitor
-   * @return the health monitor created by the owner; <code>null</code> if the owner has now created
-   *         a monitor.
-   * @since GemFire 3.5
-   */
-  @Override
-  public HealthMonitor getHealthMonitor(InternalDistributedMember owner) {
-    return (HealthMonitor) this.hmMap.get(owner);
-  }
-
-  /**
-   * Returns the health monitor for this distribution manager.
-   *
-   * @param owner the agent that owns the created monitor
-   * @param cfg the configuration to use when creating the monitor
-   * @since GemFire 3.5
-   */
-  @Override
-  public void createHealthMonitor(InternalDistributedMember owner, GemFireHealthConfig cfg) {
-    if (closeInProgress) {
-      return;
-    }
-    {
-      final HealthMonitor hm = getHealthMonitor(owner);
-      if (hm != null) {
-        hm.stop();
-        this.hmMap.remove(owner);
-      }
-    }
-    {
-      HealthMonitorImpl newHm = new HealthMonitorImpl(owner, cfg, this);
-      newHm.start();
-      this.hmMap.put(owner, newHm);
-    }
-  }
-
-  /**
-   * Remove a monitor that was previously created.
-   *
-   * @param owner the agent that owns the monitor to remove
-   */
-  @Override
-  public void removeHealthMonitor(InternalDistributedMember owner, int theId) {
-    final HealthMonitor hm = getHealthMonitor(owner);
-    if (hm != null && hm.getId() == theId) {
-      hm.stop();
-      this.hmMap.remove(owner);
-    }
-  }
-
-  private void removeAllHealthMonitors() {
-    Iterator it = this.hmMap.values().iterator();
-    while (it.hasNext()) {
-      HealthMonitor hm = (HealthMonitor) it.next();
-      hm.stop();
-      it.remove();
-    }
-  }
-
-  @Override
-  public Set<InternalDistributedMember> getAdminMemberSet() {
-    return this.adminConsoles;
-  }
-
-  /** Returns count of members filling the specified role */
-  @Override
-  public int getRoleCount(Role role) {
-    int count = 0;
-    Set mbrs = getDistributionManagerIds();
-    for (Iterator mbrIter = mbrs.iterator(); mbrIter.hasNext();) {
-      Set roles = ((InternalDistributedMember) mbrIter.next()).getRoles();
-      for (Iterator rolesIter = roles.iterator(); rolesIter.hasNext();) {
-        Role mbrRole = (Role) rolesIter.next();
-        if (mbrRole.equals(role)) {
-          count++;
-          break;
-        }
-      }
-    }
-    return count;
-  }
-
-  /** Returns true if at least one member is filling the specified role */
-  @Override
-  public boolean isRolePresent(Role role) {
-    Set mbrs = getDistributionManagerIds();
-    for (Iterator mbrIter = mbrs.iterator(); mbrIter.hasNext();) {
-      Set roles = ((InternalDistributedMember) mbrIter.next()).getRoles();
-      for (Iterator rolesIter = roles.iterator(); rolesIter.hasNext();) {
-        Role mbrRole = (Role) rolesIter.next();
-        if (mbrRole.equals(role)) {
-          return true;
-        }
-      }
-    }
-    return false;
-  }
-
-  /** Returns a set of all roles currently in the distributed system. */
-  @Override
-  public Set getAllRoles() {
-    Set allRoles = new HashSet();
-    Set mbrs = getDistributionManagerIds();
-    for (Iterator mbrIter = mbrs.iterator(); mbrIter.hasNext();) {
-      Set roles = ((InternalDistributedMember) mbrIter.next()).getRoles();
-      for (Iterator rolesIter = roles.iterator(); rolesIter.hasNext();) {
-        Role mbrRole = (Role) rolesIter.next();
-        allRoles.add(mbrRole);
-      }
-    }
-    return allRoles;
-  }
-
-  /**
-   * Returns the membership manager for this distributed system. The membership manager owns the
-   * membership set and handles all communications. The manager should NOT be used to bypass
-   * DistributionManager to send or receive messages.
-   * <p>
-   * This method was added to allow hydra to obtain thread-local data for transport from one thread
-   * to another.
-   */
-  @Override
-  public MembershipManager getMembershipManager() {
-    // NOTE: do not add cancellation checks here. This method is
-    // used during auto-reconnect after the DS has been closed
-    return membershipManager;
-  }
-
-
-  ////////////////////// Inner Classes //////////////////////
-
-
-  /**
-   * This class is used for DM's multi serial executor. The serial messages are managed/executed by
-   * multiple serial thread. This class takes care of executing messages related to a sender using
-   * the same thread.
-   */
-  private static class SerialQueuedExecutorPool {
-    /** To store the serial threads */
-    ConcurrentMap serialQueuedExecutorMap = new ConcurrentHashMap(MAX_SERIAL_QUEUE_THREAD);
-
-    /** To store the queue associated with thread */
-    Map serialQueuedMap = new HashMap(MAX_SERIAL_QUEUE_THREAD);
-
-    /** Holds mapping between sender to the serial thread-id */
-    Map senderToSerialQueueIdMap = new HashMap();
-
-    /**
-     * Holds info about unused thread, a thread is marked unused when the member associated with it
-     * has left distribution system.
-     */
-    ArrayList threadMarkedForUse = new ArrayList();
-
-    DistributionStats stats;
-    ThreadGroup threadGroup;
-
-    final boolean throttlingDisabled;
-
-    /**
-     * Constructor.
-     *
-     * @param group thread group to which the threads will belog to.
-     * @param stats
-     */
-    SerialQueuedExecutorPool(ThreadGroup group, DistributionStats stats,
-        boolean throttlingDisabled) {
-      this.threadGroup = group;
-      this.stats = stats;
-      this.throttlingDisabled = throttlingDisabled;
-    }
-
-    /*
-     * Returns an id of the thread in serialQueuedExecutorMap, thats mapped to the given seder.
-     *
-     * @param sender
-     *
-     * @param createNew boolean flag to indicate whether to create a new id, if id doesnot exists.
-     */
-    private Integer getQueueId(InternalDistributedMember sender, boolean createNew) {
-      // Create a new Id.
-      Integer queueId;
-
-      synchronized (senderToSerialQueueIdMap) {
-        // Check if there is a executor associated with this sender.
-        queueId = (Integer) senderToSerialQueueIdMap.get(sender);
-
-        if (!createNew || queueId != null) {
-          return queueId;
-        }
-
-        // Create new.
-        // Check if any threads are availabe that is marked for Use.
-        if (!threadMarkedForUse.isEmpty()) {
-          queueId = (Integer) threadMarkedForUse.remove(0);
-        }
-        // If Map is full, use the threads in round-robin fashion.
-        if (queueId == null) {
-          queueId = Integer.valueOf((serialQueuedExecutorMap.size() + 1) % MAX_SERIAL_QUEUE_THREAD);
-        }
-        senderToSerialQueueIdMap.put(sender, queueId);
-      }
-      return queueId;
-    }
-
-    /*
-     * Returns the queue associated with this sender. Used in FlowControl for throttling (based on
-     * queue size).
-     */
-    public OverflowQueueWithDMStats getSerialQueue(InternalDistributedMember sender) {
-      Integer queueId = getQueueId(sender, false);
-      if (queueId == null) {
-        return null;
-      }
-      return (OverflowQueueWithDMStats) serialQueuedMap.get(queueId);
-    }
-
-    /*
-     * Returns the serial queue executor, before returning the thread this applies throttling, based
-     * on the total serial queue size (total - sum of all the serial queue size). The throttling is
-     * applied during put event, this doesnt block the extract operation on the queue.
-     *
-     */
-    public SerialQueuedExecutorWithDMStats getThrottledSerialExecutor(
-        InternalDistributedMember sender) {
-      SerialQueuedExecutorWithDMStats executor = getSerialExecutor(sender);
-
-      // Get the total serial queue size.
-      int totalSerialQueueMemSize = stats.getSerialQueueBytes();
-
-      // for tcp socket reader threads, this code throttles the thread
-      // to keep the sender-side from overwhelming the receiver.
-      // UDP readers are throttled in the FC protocol, which queries
-      // the queue to see if it should throttle
-      if (stats.getSerialQueueBytes() > TOTAL_SERIAL_QUEUE_THROTTLE
-          && !DistributionMessage.isPreciousThread()) {
-        do {
-          boolean interrupted = Thread.interrupted();
-          try {
-            float throttlePercent = (float) (totalSerialQueueMemSize - TOTAL_SERIAL_QUEUE_THROTTLE)
-                / (float) (TOTAL_SERIAL_QUEUE_BYTE_LIMIT - TOTAL_SERIAL_QUEUE_THROTTLE);
-            int sleep = (int) (100.0 * throttlePercent);
-            sleep = Math.max(sleep, 1);
-            Thread.sleep(sleep);
-          } catch (InterruptedException ex) {
-            interrupted = true;
-            // FIXME-InterruptedException
-            // Perhaps we should return null here?
-          } finally {
-            if (interrupted) {
-              Thread.currentThread().interrupt();
-            }
-          }
-          this.stats.getSerialQueueHelper().incThrottleCount();
-        } while (stats.getSerialQueueBytes() >= TOTAL_SERIAL_QUEUE_BYTE_LIMIT);
-      }
-      return executor;
-    }
-
-    /*
-     * Returns the serial queue executor for the given sender.
-     */
-    public SerialQueuedExecutorWithDMStats getSerialExecutor(InternalDistributedMember sender) {
-      SerialQueuedExecutorWithDMStats executor = null;
-      Integer queueId = getQueueId(sender, true);
-      if ((executor =
-          (SerialQueuedExecutorWithDMStats) serialQueuedExecutorMap.get(queueId)) != null) {
-        return executor;
-      }
-      // If executor doesn't exists for this sender, create one.
-      executor = createSerialExecutor(queueId);
-
-      serialQueuedExecutorMap.put(queueId, executor);
-
-      if (logger.isDebugEnabled()) {
-        logger.debug(
-            "Created Serial Queued Executor With queueId {}. Total number of live Serial Threads :{}",
-            queueId, serialQueuedExecutorMap.size());
-      }
-      stats.incSerialPooledThread();
-      return executor;
-    }
-
-    /*
-     * Creates a serial queue executor.
-     */
-    private SerialQueuedExecutorWithDMStats createSerialExecutor(final Integer id) {
-
-      BlockingQueue poolQueue;
-
-      if (SERIAL_QUEUE_BYTE_LIMIT == 0 || this.throttlingDisabled) {
-        poolQueue = new OverflowQueueWithDMStats(stats.getSerialQueueHelper());
-      } else {
-        poolQueue = new ThrottlingMemLinkedQueueWithDMStats(SERIAL_QUEUE_BYTE_LIMIT,
-            SERIAL_QUEUE_THROTTLE, SERIAL_QUEUE_SIZE_LIMIT, SERIAL_QUEUE_SIZE_THROTTLE,
-            this.stats.getSerialQueueHelper());
-      }
-
-      serialQueuedMap.put(id, poolQueue);
-
-      ThreadFactory tf = new ThreadFactory() {
-        public Thread newThread(final Runnable command) {
-          SerialQueuedExecutorPool.this.stats.incSerialPooledThreadStarts();
-          final Runnable r = new Runnable() {
-            public void run() {
-              ConnectionTable.threadWantsSharedResources();
-              Connection.makeReaderThread();
-              try {
-                command.run();
-              } finally {
-                ConnectionTable.releaseThreadsSockets();
-              }
-            }
-          };
-
-          Thread thread = new Thread(threadGroup, r, "Pooled Serial Message Processor " + id);
-          thread.setDaemon(true);
-          return thread;
-        }
-      };
-      return new SerialQueuedExecutorWithDMStats(poolQueue,
-          this.stats.getSerialPooledProcessorHelper(), tf);
-    }
-
-    /*
-     * Does cleanup relating to this member. And marks the serial executor associated with this
-     * member for re-use.
-     */
-    private void handleMemberDeparture(InternalDistributedMember member) {
-      Integer queueId = getQueueId(member, false);
-      if (queueId == null) {
-        return;
-      }
-
-      boolean isUsed = false;
-
-      synchronized (senderToSerialQueueIdMap) {
-        senderToSerialQueueIdMap.remove(member);
-
-        // Check if any other members are using the same executor.
-        for (Iterator iter = senderToSerialQueueIdMap.values().iterator(); iter.hasNext();) {
-          Integer value = (Integer) iter.next();
-          if (value.equals(queueId)) {
-            isUsed = true;
-            break;
-          }
-        }
-
-        // If not used mark this as unused.
-        if (!isUsed) {
-          if (logger.isInfoEnabled(LogMarker.DM))
-            logger.info(LogMarker.DM,
-                LocalizedMessage.create(
-                    LocalizedStrings.DistributionManager_MARKING_THE_SERIALQUEUEDEXECUTOR_WITH_ID__0__USED_BY_THE_MEMBER__1__TO_BE_UNUSED,
-                    new Object[] {queueId, member}));
-
-          threadMarkedForUse.add(queueId);
-        }
-      }
-    }
-
-    private void awaitTermination(long time, TimeUnit unit) throws InterruptedException {
-      long timeNanos = unit.toNanos(time);
-      long remainingNanos = timeNanos;
-      long start = System.nanoTime();
-      for (Iterator iter = serialQueuedExecutorMap.values().iterator(); iter.hasNext();) {
-        ExecutorService executor = (ExecutorService) iter.next();
-        executor.awaitTermination(remainingNanos, TimeUnit.NANOSECONDS);
-        remainingNanos = timeNanos = (System.nanoTime() - start);
-        if (remainingNanos <= 0) {
-          return;
-        }
-      }
-    }
-
-    private void shutdown() {
-      for (Iterator iter = serialQueuedExecutorMap.values().iterator(); iter.hasNext();) {
-        ExecutorService executor = (ExecutorService) iter.next();
-        executor.shutdown();
-      }
-    }
-  }
-
-  /**
-   * A simple class used for locking the list of members of the distributed system. We give this
-   * lock its own class so that it shows up nicely in stack traces.
-   */
-  private static class MembersLock {
-    protected MembersLock() {
-
-    }
-  }
-
-  /**
-   * A simple class used for locking the list of membership listeners. We give this lock its own
-   * class so that it shows up nicely in stack traces.
-   */
-  private static class MembershipListenersLock {
-    protected MembershipListenersLock() {}
-  }
-
-  /**
-   * This is the listener implementation for responding from events from the Membership Manager.
-   *
-   */
-  private class DMListener implements DistributedMembershipListener {
-    DistributionManager dm;
-
-    public DMListener(DistributionManager dm) {
-      this.dm = dm;
-    }
-
-    @Override
-    public boolean isShutdownMsgSent() {
-      return shutdownMsgSent;
-    }
-
-    @Override
-    public void membershipFailure(String reason, Throwable t) {
-      exceptionInThreads = true;
-      DistributionManager.this.rootCause = t;
-      getSystem().disconnect(reason, t, true);
-    }
-
-    @Override
-    public void messageReceived(DistributionMessage message) {
-      handleIncomingDMsg(message);
-    }
-
-    @Override
-    public void newMemberConnected(InternalDistributedMember member) {
-      // Do not elect the elder here as surprise members invoke this callback
-      // without holding the view lock. That can cause a race condition and
-      // subsequent deadlock (#45566). Elder selection is now done when a view
-      // is installed.
-      dm.addNewMember(member);
-    }
-
-    @Override
-    public void memberDeparted(InternalDistributedMember theId, boolean crashed, String reason) {
-      boolean wasAdmin = getAdminMemberSet().contains(theId);
-      if (wasAdmin) {
-        // Pretend we received an AdminConsoleDisconnectMessage from the console that
-        // is no longer in the JavaGroup view.
-        // He must have died without sending a ShutdownMessage.
-        // This fixes bug 28454.
-        AdminConsoleDisconnectMessage message = new AdminConsoleDisconnectMessage();
-        message.setSender(theId);
-        message.setCrashed(crashed);
-        message.setAlertListenerExpected(true);
-        message.setIgnoreAlertListenerRemovalFailure(true); // we don't know if it was a listener so
-                                                            // don't issue a warning
-        message.setRecipient(localAddress);
-        message.setReason(reason); // added for #37950
-        handleIncomingDMsg(message);
-      }
-      dm.handleManagerDeparture(theId, crashed, reason);
-    }
-
-    @Override
-    public void memberSuspect(InternalDistributedMember suspect,
-        InternalDistributedMember whoSuspected, String reason) {
-      dm.handleManagerSuspect(suspect, whoSuspected, reason);
-    }
-
-    @Override
-    public void viewInstalled(NetView view) {
-      processElderSelection();
-      dm.handleViewInstalled(view);
-    }
-
-    @Override
-    public void quorumLost(Set<InternalDistributedMember> failures,
-        List<InternalDistributedMember> remaining) {
-      dm.handleQuorumLost(failures, remaining);
-    }
-
-    @Override
-    public DistributionManager getDM() {
-      return dm;
-    }
-
-    private void processElderSelection() {
-      // If we currently had no elder, this member might be the elder;
-      // go through the selection process and decide now.
-      try {
-        dm.selectElder();
-      } catch (DistributedSystemDisconnectedException e) {
-        // ignore
-      }
-    }
-  }
-
-
-  private abstract static class MemberEvent {
-
-    private InternalDistributedMember id;
-
-    MemberEvent(InternalDistributedMember id) {
-      this.id = id;
-    }
-
-    public InternalDistributedMember getId() {
-      return this.id;
-    }
-
-    void handleEvent(DistributionManager manager) {
-      handleEvent(manager, manager.membershipListeners.keySet());
-      handleEvent(manager, manager.allMembershipListeners);
-    }
-
-    protected abstract void handleEvent(MembershipListener listener);
-
-    private void handleEvent(DistributionManager manager,
-        Set<MembershipListener> membershipListeners) {
-      for (MembershipListener listener : membershipListeners) {
-        try {
-          handleEvent(listener);
-        } catch (CancelException e) {
-          if (manager.isCloseInProgress()) {
-            if (logger.isTraceEnabled()) {
-              logger.trace("MemberEventInvoker: cancelled");
-            }
-          } else {
-            logger.warn(LocalizedMessage
-                .create(LocalizedStrings.DistributionManager_UNEXPECTED_CANCELLATION), e);
-          }
-          break;
-        } catch (VirtualMachineError err) {
-          SystemFailure.initiateFailure(err);
-          // If this ever returns, rethrow the error. We're poisoned
-          // now, so don't let this thread continue.
-          throw err;
-        } catch (Throwable t) {
-          // Whenever you catch Error or Throwable, you must also
-          // catch VirtualMachineError (see above). However, there is
-          // _still_ a possibility that you are dealing with a cascading
-          // error condition, so you also need to check to see if the JVM
-          // is still usable:
-          SystemFailure.checkFailure();
-          logger.warn(LocalizedMessage.create(
-              LocalizedStrings.DistributionManager_EXCEPTION_WHILE_CALLING_MEMBERSHIP_LISTENER_FOR_EVENT__0,
-              this), t);
-        }
-      }
-    }
-  }
-
-  /**
-   * This is an event reflecting that a InternalDistributedMember has joined the system.
-   *
-   *
-   */
-  private static class MemberJoinedEvent extends MemberEvent {
-    MemberJoinedEvent(InternalDistributedMember id) {
-      super(id);
-    }
-
-    @Override
-    public String toString() {
-      return "member " + getId() + " joined";
-    }
-
-    @Override
-    protected void handleEvent(MembershipListener listener) {
-      listener.memberJoined(getId());
-    }
-  }
-
-  /**
-   * This is an event reflecting that a InternalDistributedMember has left the system.
-   *
-   */
-  private static class MemberDepartedEvent extends MemberEvent {
-    String reason;
-
-    MemberDepartedEvent(InternalDistributedMember id, String r) {
-      super(id);
-      reason = r;
-    }
-
-    @Override
-    public String toString() {
-      return "member " + getId() + " departed (" + reason + ")";
-    }
-
-    @Override
-    protected void handleEvent(MembershipListener listener) {
-      listener.memberDeparted(getId(), false);
-    }
-  }
-
-  /**
-   * This is an event reflecting that a InternalDistributedMember has left the system in an
-   * unexpected way.
-   *
-   *
-   */
-  private static class MemberCrashedEvent extends MemberEvent {
-    String reason;
-
-    MemberCrashedEvent(InternalDistributedMember id, String r) {
-      super(id);
-      reason = r;
-    }
-
-    @Override
-    public String toString() {
-      return "member " + getId() + " crashed: " + reason;
-    }
-
-    @Override
-    protected void handleEvent(MembershipListener listener) {
-      listener.memberDeparted(getId(), true/* crashed */);
-    }
-  }
-
-  /**
-   * This is an event reflecting that a InternalDistributedMember may be missing but has not yet
-   * left the system.
-   */
-  private static class MemberSuspectEvent extends MemberEvent {
-    InternalDistributedMember whoSuspected;
-    String reason;
-
-    MemberSuspectEvent(InternalDistributedMember suspect, InternalDistributedMember whoSuspected,
-        String reason) {
-      super(suspect);
-      this.whoSuspected = whoSuspected;
-      this.reason = reason;
-    }
-
-    public InternalDistributedMember whoSuspected() {
-      return this.whoSuspected;
-    }
-
-    public String getReason() {
-      return this.reason;
-    }
-
-    @Override
-    public String toString() {
-      return "member " + getId() + " suspected by: " + this.whoSuspected + " reason: " + reason;
-    }
-
-    @Override
-    protected void handleEvent(MembershipListener listener) {
-      listener.memberSuspect(getId(), whoSuspected(), reason);
-    }
-  }
-
-  private static class ViewInstalledEvent extends MemberEvent {
-    NetView view;
-
-    ViewInstalledEvent(NetView view) {
-      super(null);
-      this.view = view;
-    }
-
-    public long getViewId() {
-      return view.getViewId();
-    }
-
-    @Override
-    public String toString() {
-      return "view installed: " + this.view;
-    }
-
-    @Override
-    public void handleEvent(DistributionManager manager) {
-      manager.handleViewInstalledEvent(this);
-    }
-
-    @Override
-    protected void handleEvent(MembershipListener listener) {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  private static class QuorumLostEvent extends MemberEvent {
-    Set<InternalDistributedMember> failures;
-    List<InternalDistributedMember> remaining;
-
-    QuorumLostEvent(Set<InternalDistributedMember> failures,
-        List<InternalDistributedMember> remaining) {
-      super(null);
-      this.failures = failures;
-      this.remaining = remaining;
-    }
-
-    public Set<InternalDistributedMember> getFailures() {
-      return this.failures;
-    }
-
-    public List<InternalDistributedMember> getRemaining() {
-      return this.remaining;
-    }
-
-    @Override
-    public String toString() {
-      return "quorum lost.  failures=" + failures + "; remaining=" + remaining;
-    }
-
-    @Override
-    protected void handleEvent(MembershipListener listener) {
-      listener.quorumLost(getFailures(), getRemaining());
-    }
-  }
-
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see org.apache.geode.distributed.internal.DM#getRootCause()
-   */
-  @Override
-  public Throwable getRootCause() {
-    return this.rootCause;
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see org.apache.geode.distributed.internal.DM#setRootCause(java.lang.Throwable)
-   */
-  @Override
-  public void setRootCause(Throwable t) {
-    this.rootCause = t;
-  }
-
-  /*
-   * (non-Javadoc)
-   *
-   * @see org.apache.geode.distributed.internal.DM#getMembersOnThisHost()
-   *
-   * @since GemFire 5.9
-   */
-  @Override
-  public Set<InternalDistributedMember> getMembersInThisZone() {
-    return getMembersInSameZone(getDistributionManagerId());
-  }
-
-  @Override
-  public Set<InternalDistributedMember> getMembersInSameZone(
-      InternalDistributedMember targetMember) {
-    Set<InternalDistributedMember> buddyMembers = new HashSet<InternalDistributedMember>();
-    if (!redundancyZones.isEmpty()) {
-      synchronized (redundancyZones) {
-        String targetZone = redundancyZones.get(targetMember);
-        for (Map.Entry<InternalDistributedMember, String> entry : redundancyZones.entrySet()) {
-          if (entry.getValue().equals(targetZone)) {
-            buddyMembers.add(entry.getKey());
-          }
-        }
-      }
-    } else {
-      buddyMembers.add(targetMember);
-      Set targetAddrs = getEquivalents(targetMember.getInetAddress());
-      for (Iterator i = getDistributionManagerIds().iterator(); i.hasNext();) {
-        InternalDistributedMember o = (InternalDistributedMember) i.next();
-        if (SetUtils.intersectsWith(targetAddrs, getEquivalents(o.getInetAddress()))) {
-          buddyMembers.add(o);
-        }
-      }
-    }
-    return buddyMembers;
-  }
-
-  @Override
-  public boolean areInSameZone(InternalDistributedMember member1,
-      InternalDistributedMember member2) {
-
-    if (!redundancyZones.isEmpty()) {
-      String zone1 = redundancyZones.get(member1);
-      String zone2 = redundancyZones.get(member2);
-      return zone1 != null && zone1.equals(zone2);
-    } else {
-      return areOnEquivalentHost(member1, member2);
-    }
-  }
-
-  @Override
-  public void acquireGIIPermitUninterruptibly() {
-    this.parallelGIIs.acquireUninterruptibly();
-    this.stats.incInitialImageRequestsInProgress(1);
-  }
-
-  @Override
-  public void releaseGIIPermit() {
-    this.stats.incInitialImageRequestsInProgress(-1);
-    this.parallelGIIs.release();
-  }
-
-  public void setDistributedSystemId(int distributedSystemId) {
-    if (distributedSystemId != -1) {
-      this.distributedSystemId = distributedSystemId;
-    }
-  }
-
-  @Override
-  public int getDistributedSystemId() {
-    return this.distributedSystemId;
-  }
-
-  /**
-   * this causes the given InternalDistributedMembers to log thread dumps. If useNative is true we
-   * attempt to use OSProcess native code for the dumps. This goes to stdout instead of the
-   * system.log files.
-   */
-  public void printStacks(Collection ids, boolean useNative) {
-    Set requiresMessage = new HashSet();
-    if (ids.contains(localAddress)) {
-      OSProcess.printStacks(0, useNative);
-    }
-    if (useNative) {
-      requiresMessage.addAll(ids);
-      ids.remove(localAddress);
-    } else {
-      for (Iterator it = ids.iterator(); it.hasNext();) {
-        InternalDistributedMember mbr = (InternalDistributedMember) it.next();
-        if (mbr.getProcessId() > 0
-            && mbr.getInetAddress().equals(this.localAddress.getInetAddress())) {
-          if (!mbr.equals(localAddress)) {
-            if (!OSProcess.printStacks(mbr.getProcessId(), false)) {
-              requiresMessage.add(mbr);
-            }
-          }
-        } else {
-          requiresMessage.add(mbr);
-        }
-      }
-    }
-    if (requiresMessage.size() > 0) {
-      HighPriorityAckedMessage msg = new HighPriorityAckedMessage();
-      msg.dumpStacks(requiresMessage, useNative, false);
-    }
-  }
-
-  @Override
-  public Set<DistributedMember> getGroupMembers(String group) {
-    HashSet<DistributedMember> result = null;
-    for (DistributedMember m : (Set<DistributedMember>) getDistributionManagerIdsIncludingAdmin()) {
-      if (m.getGroups().contains(group)) {
-        if (result == null) {
-          result = new HashSet<DistributedMember>();
-        }
-        result.add(m);
-      }
-    }
-    if (result == null) {
-      return Collections.emptySet();
-    } else {
-      return result;
-    }
-  }
-
-  @Override
-  public Set getNormalDistributionManagerIds() {
-    // access to members synchronized under membersLock in order to
-    // ensure serialization
-    synchronized (this.membersLock) {
-      HashSet<InternalDistributedMember> result = new HashSet<InternalDistributedMember>();
-      for (InternalDistributedMember m : this.members.keySet()) {
-        if (m.getVmKind() != DistributionManager.LOCATOR_DM_TYPE) {
-          result.add(m);
-        }
-      }
-      return result;
-    }
-  }
-
-  /** test method to get the member IDs of all locators in the distributed system */
-  public Set<InternalDistributedMember> getLocatorDistributionManagerIds() {
-    // access to members synchronized under membersLock in order to
-    // ensure serialization
-    synchronized (this.membersLock) {
-      HashSet<InternalDistributedMember> result = new HashSet<InternalDistributedMember>();
-      for (InternalDistributedMember m : this.members.keySet()) {
-        if (m.getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
-          result.add(m);
-        }
-      }
-      return result;
-    }
-  }
-
-  @Override
-  public void setCache(InternalCache instance) {
-    this.cache = instance;
-  }
-
-  @Override
-  public InternalCache getCache() {
-    return this.cache;
-  }
-
-  @Override
-  public InternalCache getExistingCache() {
-    InternalCache result = this.cache;
-    if (result == null) {
-      throw new CacheClosedException(
-          LocalizedStrings.CacheFactory_A_CACHE_HAS_NOT_YET_BEEN_CREATED.toLocalizedString());
-    }
-    result.getCancelCriterion().checkCancelInProgress(null);
-    if (result.isClosed()) {
-      throw result.getCacheClosedException(
-          LocalizedStrings.CacheFactory_THE_CACHE_HAS_BEEN_CLOSED.toLocalizedString(), null);
-    }
-    return result;
-  }
-
-
-  private static class Stopper extends CancelCriterion {
-    private DistributionManager dm;
-
-    Stopper(DistributionManager dm) {
-      this.dm = dm;
-    }
-
-    @Override
-    public String cancelInProgress() {
-      checkFailure();
-
-      // remove call to validateDM() to fix bug 38356
-
-      if (dm.shutdownMsgSent) {
-        return LocalizedStrings.DistributionManager__0_MESSAGE_DISTRIBUTION_HAS_TERMINATED
-            .toLocalizedString(dm.toString());
-      }
-      if (dm.rootCause != null) {
-        return dm.toString() + ": " + dm.rootCause.getMessage();
-      }
-
-      // Nope.
-      return null;
-    }
-
-    @Override
-    public RuntimeException generateCancelledException(Throwable e) {
-      String reason = cancelInProgress();
-      if (reason == null) {
-        return null;
-      }
-      Throwable rc = dm.rootCause; // volatile read
-      if (rc == null) {
-        // No root cause, specify the one given and be done with it.
-        return new DistributedSystemDisconnectedException(reason, e);
-      }
-
-      if (e == null) {
-        // Caller did not specify any root cause, so just use our own.
-        return new DistributedSystemDisconnectedException(reason, rc);
-      }
-
-      // Attempt to stick rootCause at tail end of the exception chain.
-      Throwable nt = e;
-      while (nt.getCause() != null) {
-        nt = nt.getCause();
-      }
-      if (nt == rc) {
-        // Root cause already in place; we're done
-        return new DistributedSystemDisconnectedException(reason, e);
-      }
+  InternalCache getExistingCache();
 
-      try {
-        nt.initCause(rc);
-        return new DistributedSystemDisconnectedException(reason, e);
-      } catch (IllegalStateException e2) {
-        // Bug 39496 (Jrockit related) Give up. The following
-        // error is not entirely sane but gives the correct general picture.
-        return new DistributedSystemDisconnectedException(reason, rc);
-      }
-    }
-  }
+  void setCache(InternalCache instance);
 
-  private final Stopper stopper = new Stopper(this);
+  HealthMonitor getHealthMonitor(InternalDistributedMember owner);
 
-  @Override
-  public CancelCriterion getCancelCriterion() {
-    return stopper;
-  }
+  void removeHealthMonitor(InternalDistributedMember owner, int theId);
 
+  void createHealthMonitor(InternalDistributedMember owner, GemFireHealthConfig cfg);
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessage.java
index 134488b..0c47b88 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessage.java
@@ -168,7 +168,7 @@ public abstract class DistributionMessage implements DataSerializableFixedID, Cl
     this.acker = acker;
   }
 
-  public ReplySender getReplySender(DM dm) {
+  public ReplySender getReplySender(DistributionManager dm) {
     if (acker != null) {
       return acker;
     } else {
@@ -189,12 +189,12 @@ public abstract class DistributionMessage implements DataSerializableFixedID, Cl
   public boolean orderedDelivery() {
     final int processorType = getProcessorType();
     switch (processorType) {
-      case DistributionManager.SERIAL_EXECUTOR:
+      case ClusterDistributionManager.SERIAL_EXECUTOR:
         // no need to use orderedDelivery for PR ops particularly when thread
         // does not own resources
         // case DistributionManager.PARTITIONED_REGION_EXECUTOR:
         return true;
-      case DistributionManager.REGION_FUNCTION_EXECUTION_EXECUTOR:
+      case ClusterDistributionManager.REGION_FUNCTION_EXECUTION_EXECUTOR:
         // allow nested distributed functions to be executed from within the
         // execution of a function
         return false;
@@ -331,7 +331,7 @@ public abstract class DistributionMessage implements DataSerializableFixedID, Cl
   /**
    * Return the Executor in which to process this message.
    */
-  protected Executor getExecutor(DistributionManager dm) {
+  protected Executor getExecutor(ClusterDistributionManager dm) {
     return dm.getExecutor(getProcessorType(), sender);
   }
 
@@ -346,12 +346,12 @@ public abstract class DistributionMessage implements DataSerializableFixedID, Cl
    *
    * @param dm the distribution manager that is processing the message.
    */
-  protected abstract void process(DistributionManager dm);
+  protected abstract void process(ClusterDistributionManager dm);
 
   /**
    * Scheduled action to take when on this message when we are ready to process it.
    */
-  protected void scheduleAction(final DistributionManager dm) {
+  protected void scheduleAction(final ClusterDistributionManager dm) {
     if (logger.isTraceEnabled(LogMarker.DM)) {
       logger.trace(LogMarker.DM, "Processing '{}'", this);
     }
@@ -417,9 +417,9 @@ public abstract class DistributionMessage implements DataSerializableFixedID, Cl
   /**
    * Schedule this message's process() method in a thread determined by getExecutor()
    */
-  protected void schedule(final DistributionManager dm) {
+  protected void schedule(final ClusterDistributionManager dm) {
     boolean inlineProcess = INLINE_PROCESS
-        && getProcessorType() == DistributionManager.SERIAL_EXECUTOR && !isPreciousThread();
+        && getProcessorType() == ClusterDistributionManager.SERIAL_EXECUTOR && !isPreciousThread();
 
     boolean forceInline = this.acker != null || getInlineProcess() || Connection.isDominoThread();
 
@@ -481,7 +481,7 @@ public abstract class DistributionMessage implements DataSerializableFixedID, Cl
     } // not inline
   }
 
-  protected boolean mayAddToMultipleSerialGateways(DistributionManager dm) {
+  protected boolean mayAddToMultipleSerialGateways(ClusterDistributionManager dm) {
     // subclasses should override this method if processing
     // them may add to multiple serial gateways.
     return false;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessageObserver.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessageObserver.java
index 72e4811..0261887 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessageObserver.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessageObserver.java
@@ -42,7 +42,7 @@ public abstract class DistributionMessageObserver {
    * @param dm the distribution manager that received the message
    * @param message The message itself
    */
-  public void beforeProcessMessage(DistributionManager dm, DistributionMessage message) {
+  public void beforeProcessMessage(ClusterDistributionManager dm, DistributionMessage message) {
     // override as needed
   }
 
@@ -52,7 +52,7 @@ public abstract class DistributionMessageObserver {
    * @param dm the distribution manager that received the message
    * @param message The message itself
    */
-  public void afterProcessMessage(DistributionManager dm, DistributionMessage message) {
+  public void afterProcessMessage(ClusterDistributionManager dm, DistributionMessage message) {
     // override as needed
   }
 
@@ -62,7 +62,7 @@ public abstract class DistributionMessageObserver {
    * @param dm the distribution manager that's sending the message
    * @param message the message itself
    */
-  public void beforeSendMessage(DistributionManager dm, DistributionMessage message) {
+  public void beforeSendMessage(ClusterDistributionManager dm, DistributionMessage message) {
     // override as needed
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/HealthMonitorImpl.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/HealthMonitorImpl.java
index 2d1793b..3ce6834 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/HealthMonitorImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/HealthMonitorImpl.java
@@ -37,7 +37,7 @@ public class HealthMonitorImpl implements HealthMonitor, Runnable {
 
   private final InternalDistributedMember owner;
   private final int id;
-  private final DistributionManager dm;
+  private final ClusterDistributionManager dm;
   private final GemFireHealthEvaluator eval;
 
   /**
@@ -56,7 +56,7 @@ public class HealthMonitorImpl implements HealthMonitor, Runnable {
    * Creates a health monitor given its owner, configuration, and its dm
    */
   public HealthMonitorImpl(InternalDistributedMember owner, GemFireHealthConfig config,
-      DistributionManager dm) {
+      ClusterDistributionManager dm) {
     this.owner = owner;
     this.id = getNewId();
     this.dm = dm;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/HighPriorityAckedMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/HighPriorityAckedMessage.java
index 39879a8..19494b7 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/HighPriorityAckedMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/HighPriorityAckedMessage.java
@@ -57,7 +57,7 @@ public class HighPriorityAckedMessage extends HighPriorityDistributionMessage
     DRAIN_POOL, DUMP_STACK
   };
 
-  transient DistributionManager originDm;
+  transient ClusterDistributionManager originDm;
   private transient ReplyProcessor21 rp;
   private boolean useNative;
 
@@ -65,7 +65,7 @@ public class HighPriorityAckedMessage extends HighPriorityDistributionMessage
     super();
     InternalDistributedSystem ds = InternalDistributedSystem.getAnyInstance();
     if (ds != null) {
-      this.originDm = (DistributionManager) ds.getDistributionManager();
+      this.originDm = (ClusterDistributionManager) ds.getDistributionManager();
     }
     if (this.originDm != null) {
       this.id = this.originDm.getDistributionManagerId();
@@ -156,7 +156,7 @@ public class HighPriorityAckedMessage extends HighPriorityDistributionMessage
    * This method is invoked on the receiver side
    */
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     switch (this.op) {
       case DRAIN_POOL:
         Assert.assertTrue(this.id != null);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/HighPriorityDistributionMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/HighPriorityDistributionMessage.java
index d83eaf1..8aebe90 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/HighPriorityDistributionMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/HighPriorityDistributionMessage.java
@@ -22,7 +22,7 @@ public abstract class HighPriorityDistributionMessage extends DistributionMessag
 
   @Override
   public int getProcessorType() {
-    return DistributionManager.HIGH_PRIORITY_EXECUTOR;
+    return ClusterDistributionManager.HIGH_PRIORITY_EXECUTOR;
   }
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index a8b3eb6..d65f9ff 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -156,7 +156,7 @@ public class InternalDistributedSystem extends DistributedSystem
   /**
    * The distribution manager that is used to communicate with the distributed system.
    */
-  protected DM dm;
+  protected DistributionManager dm;
 
   private final GrantorRequestProcessor.GrantorRequestContext grc;
 
@@ -349,7 +349,8 @@ public class InternalDistributedSystem extends DistributedSystem
    *
    * @param nonDefault - non-default distributed system properties
    */
-  public static InternalDistributedSystem newInstanceForTesting(DM dm, Properties nonDefault) {
+  public static InternalDistributedSystem newInstanceForTesting(DistributionManager dm,
+      Properties nonDefault) {
     InternalDistributedSystem sys = new InternalDistributedSystem(nonDefault);
     sys.config = new RuntimeDistributionConfigImpl(sys);
     sys.dm = dm;
@@ -737,7 +738,7 @@ public class InternalDistributedSystem extends DistributedSystem
           if (this.quorumChecker != null) {
             this.quorumChecker.suspend();
           }
-          this.dm = DistributionManager.create(this);
+          this.dm = ClusterDistributionManager.create(this);
           // fix bug #46324
           if (InternalLocator.hasLocator()) {
             InternalLocator locator = InternalLocator.getLocator();
@@ -885,7 +886,7 @@ public class InternalDistributedSystem extends DistributedSystem
   /**
    * Used by DistributionManager to fix bug 33362
    */
-  void setDM(DM dm) {
+  void setDM(DistributionManager dm) {
     this.dm = dm;
   }
 
@@ -1487,7 +1488,7 @@ public class InternalDistributedSystem extends DistributedSystem
   /**
    * Returns the distribution manager for accessing this distributed system.
    */
-  public DM getDistributionManager() {
+  public DistributionManager getDistributionManager() {
     checkConnected();
     return this.dm;
   }
@@ -1495,7 +1496,7 @@ public class InternalDistributedSystem extends DistributedSystem
   /**
    * Returns the distribution manager without checking for connected or not so can also return null.
    */
-  public DM getDM() {
+  public DistributionManager getDM() {
     return this.dm;
   }
 
@@ -2762,11 +2763,11 @@ public class InternalDistributedSystem extends DistributedSystem
         }
 
 
-        DM newDM = this.reconnectDS.getDistributionManager();
-        if (newDM instanceof DistributionManager) {
+        DistributionManager newDM = this.reconnectDS.getDistributionManager();
+        if (newDM instanceof ClusterDistributionManager) {
           // Admin systems don't carry a cache, but for others we can now create
           // a cache
-          if (newDM.getDMType() != DistributionManager.ADMIN_ONLY_DM_TYPE) {
+          if (newDM.getDMType() != ClusterDistributionManager.ADMIN_ONLY_DM_TYPE) {
             try {
               CacheConfig config = new CacheConfig();
               if (cacheXML != null) {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
index 70fe1c9..8608b5c 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
@@ -361,12 +361,12 @@ public class InternalLocator extends Locator implements ConnectListener {
     if (ids == null) {
       return false;
     }
-    DM dm = ids.getDistributionManager();
+    DistributionManager dm = ids.getDistributionManager();
     if (dm.isLoner()) {
       return false;
     }
-    DistributionManager distMgr = (DistributionManager) ids.getDistributionManager();
-    return distMgr.getDMType() == DistributionManager.LOCATOR_DM_TYPE;
+    ClusterDistributionManager distMgr = (ClusterDistributionManager) ids.getDistributionManager();
+    return distMgr.getDMType() == ClusterDistributionManager.LOCATOR_DM_TYPE;
   }
 
   /**
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 9dc1ceb..b0d04ec 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
@@ -45,7 +45,7 @@ import org.apache.geode.internal.net.SocketCreator;
  *
  * @since GemFire 3.5
  */
-public class LonerDistributionManager implements DM {
+public class LonerDistributionManager implements DistributionManager {
   private final InternalDistributedSystem system;
   private final InternalLogWriter logger;
   private ElderState elderState;
@@ -1200,7 +1200,7 @@ public class LonerDistributionManager implements DM {
             config.getDurableClientTimeout());
       }
       result = new InternalDistributedMember(host, lonerPort, name, uniqueString,
-          DistributionManager.LONER_DM_TYPE,
+          ClusterDistributionManager.LONER_DM_TYPE,
           MemberAttributes.parseGroups(config.getRoles(), config.getGroups()), dac);
 
     } catch (UnknownHostException ex) {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/MembershipListener.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/MembershipListener.java
index 71e7f5c..6e9df18 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/MembershipListener.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/MembershipListener.java
@@ -26,7 +26,7 @@ import org.apache.geode.distributed.internal.membership.*;
  * but the callback methods are always invoked in the GemFire manager VM. Thus, the callback methods
  * should not perform time-consuming operations.
  *
- * @see DistributionManager#addMembershipListener
+ * @see ClusterDistributionManager#addMembershipListener
  */
 public interface MembershipListener {
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/PooledDistributionMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/PooledDistributionMessage.java
index 61939ae..beafa32 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/PooledDistributionMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/PooledDistributionMessage.java
@@ -24,7 +24,7 @@ public abstract class PooledDistributionMessage extends DistributionMessage {
 
   @Override
   public int getProcessorType() {
-    return DistributionManager.STANDARD_EXECUTOR;
+    return ClusterDistributionManager.STANDARD_EXECUTOR;
   }
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ProductUseLog.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ProductUseLog.java
index 8ef6fc5..1e99b86 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ProductUseLog.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ProductUseLog.java
@@ -63,7 +63,7 @@ public class ProductUseLog implements MembershipListener {
    */
   public void monitorUse(InternalDistributedSystem system) {
     this.system = system;
-    DM dmgr = system.getDistributionManager();
+    DistributionManager dmgr = system.getDistributionManager();
     dmgr.addMembershipListener(this);
     MembershipManager mmgr = dmgr.getMembershipManager();
     if (mmgr != null) {
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ReliableReplyProcessor21.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ReliableReplyProcessor21.java
index 545d4ac..431e635 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ReliableReplyProcessor21.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ReliableReplyProcessor21.java
@@ -36,11 +36,11 @@ public class ReliableReplyProcessor21 extends ReplyProcessor21 {
     super(system, member);
   }
 
-  public ReliableReplyProcessor21(DM dm, InternalDistributedMember member) {
+  public ReliableReplyProcessor21(DistributionManager dm, InternalDistributedMember member) {
     super(dm, member);
   }
 
-  public ReliableReplyProcessor21(DM dm, Collection initMembers) {
+  public ReliableReplyProcessor21(DistributionManager dm, Collection initMembers) {
     super(dm, initMembers);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyMessage.java
index d53939b..e2c22a3 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyMessage.java
@@ -182,11 +182,11 @@ public class ReplyMessage extends HighPriorityDistributionMessage {
    * @param dm the distribution manager that is processing the message.
    */
   @Override
-  protected void process(final DistributionManager dm) {
+  protected void process(final ClusterDistributionManager dm) {
     dmProcess(dm);
   }
 
-  public void dmProcess(final DM dm) {
+  public void dmProcess(final DistributionManager dm) {
     final long startTime = getTimestamp();
     ReplyProcessor21 processor = ReplyProcessor21.getProcessor(processorId);
     try {
@@ -207,7 +207,7 @@ public class ReplyMessage extends HighPriorityDistributionMessage {
    * @param dm
    * @param processor
    */
-  public void process(final DM dm, ReplyProcessor21 processor) {
+  public void process(final DistributionManager dm, ReplyProcessor21 processor) {
     if (processor == null)
       return;
     processor.process(ReplyMessage.this);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyProcessor21.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyProcessor21.java
index 84a0b08..4111c0b 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyProcessor21.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplyProcessor21.java
@@ -137,7 +137,7 @@ public class ReplyProcessor21 implements MembershipListener {
   protected final InternalDistributedSystem system;
 
   /** the distribution manager - if null, get the manager from the system */
-  protected final DM dmgr;
+  protected final DistributionManager dmgr;
 
   /** Start time for replyWait stat, in nanos */
   long statStart;
@@ -232,26 +232,28 @@ public class ReplyProcessor21 implements MembershipListener {
    * @param dm the DistributionManager to use for messaging and membership
    * @param member the member this processor wants a reply from
    */
-  public ReplyProcessor21(DM dm, InternalDistributedMember member) {
+  public ReplyProcessor21(DistributionManager dm, InternalDistributedMember member) {
     this(dm, Collections.singleton(member));
   }
 
   /**
    * Creates a new <code>ReplyProcessor</code> that wants replies from some number of members of a
-   * distributed system. Call this method with {@link DistributionManager#getDistributionManagerIds}
-   * if you want replies from all DMs including the one hosted in this VM.
+   * distributed system. Call this method with
+   * {@link ClusterDistributionManager#getDistributionManagerIds} if you want replies from all DMs
+   * including the one hosted in this VM.
    *
    * @param dm the DistributionManager to use for messaging and membership
    * @param initMembers the Set of members this processor wants replies from
    */
-  public ReplyProcessor21(DM dm, Collection initMembers) {
+  public ReplyProcessor21(DistributionManager dm, Collection initMembers) {
     this(dm, dm.getSystem(), initMembers, null);
   }
 
   /**
    * Creates a new <code>ReplyProcessor</code> that wants replies from some number of members of a
-   * distributed system. Call this method with {@link DistributionManager#getDistributionManagerIds}
-   * if you want replies from all DMs including the one hosted in this VM.
+   * distributed system. Call this method with
+   * {@link ClusterDistributionManager#getDistributionManagerIds} if you want replies from all DMs
+   * including the one hosted in this VM.
    *
    * @param system the DistributedSystem connection
    * @param initMembers the Set of members this processor wants replies from
@@ -262,8 +264,9 @@ public class ReplyProcessor21 implements MembershipListener {
 
   /**
    * Creates a new <code>ReplyProcessor</code> that wants replies from some number of members of a
-   * distributed system. Call this method with {@link DistributionManager#getDistributionManagerIds}
-   * if you want replies from all DMs including the one hosted in this VM.
+   * distributed system. Call this method with
+   * {@link ClusterDistributionManager#getDistributionManagerIds} if you want replies from all DMs
+   * including the one hosted in this VM.
    *
    * @param system the DistributedSystem connection
    * @param initMembers the Set of members this processor wants replies from
@@ -283,8 +286,8 @@ public class ReplyProcessor21 implements MembershipListener {
    * @param initMembers the collection of members this processor wants replies from
    * @param cancelCriterion optional CancelCriterion to use; will use the dm if null
    */
-  private ReplyProcessor21(DM dm, InternalDistributedSystem system, Collection initMembers,
-      CancelCriterion cancelCriterion) {
+  private ReplyProcessor21(DistributionManager dm, InternalDistributedSystem system,
+      Collection initMembers, CancelCriterion cancelCriterion) {
 
     this(dm, system, initMembers, cancelCriterion, true);
   }
@@ -297,8 +300,8 @@ public class ReplyProcessor21 implements MembershipListener {
    * @param initMembers the collection of members this processor wants replies from
    * @param cancelCriterion optional CancelCriterion to use; will use the dm if null
    */
-  protected ReplyProcessor21(DM dm, InternalDistributedSystem system, Collection initMembers,
-      CancelCriterion cancelCriterion, boolean register) {
+  protected ReplyProcessor21(DistributionManager dm, InternalDistributedSystem system,
+      Collection initMembers, CancelCriterion cancelCriterion, boolean register) {
     if (!allowReplyFromSender()) {
       Assert.assertTrue(initMembers != null, "null initMembers");
       Assert.assertTrue(system != null, "null system");
@@ -343,9 +346,9 @@ public class ReplyProcessor21 implements MembershipListener {
    * manager, it is used. Otherwise, we expect a distribution manager has been set with
    * setDistributionManager and we'll use that
    */
-  protected DM getDistributionManager() {
+  protected DistributionManager getDistributionManager() {
     try {
-      DM result = this.system.getDistributionManager();
+      DistributionManager result = this.system.getDistributionManager();
       if (result == null) {
         result = this.dmgr;
         Assert.assertTrue(result != null, "null DistributionManager");
@@ -409,7 +412,7 @@ public class ReplyProcessor21 implements MembershipListener {
     final InternalDistributedMember sender = msg.getSender();
     if (!removeMember(sender, false) && warn) {
       // if the member hasn't left the system, something is wrong
-      final DM dm = getDistributionManager(); // fix for bug 33253
+      final DistributionManager dm = getDistributionManager(); // fix for bug 33253
       Set ids = getDistributionManagerIds();
       if (ids == null || ids.contains(sender)) {
         List viewMembers = dm.getViewMembers();
@@ -550,7 +553,7 @@ public class ReplyProcessor21 implements MembershipListener {
 
   protected void preWait() {
     waiting = true;
-    DM mgr = getDistributionManager();
+    DistributionManager mgr = getDistributionManager();
     statStart = mgr.getStats().startReplyWait();
     synchronized (this.members) {
       Set activeMembers = addListenerAndGetMembers();
@@ -576,7 +579,7 @@ public class ReplyProcessor21 implements MembershipListener {
   private void postWait() {
     waiting = false;
     removeListener();
-    final DM mgr = getDistributionManager();
+    final DistributionManager mgr = getDistributionManager();
     mgr.getStats().endReplyWait(this.statStart, this.initTime);
     mgr.getCancelCriterion().checkCancelInProgress(null);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplySender.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplySender.java
index 513cab7..74070fc 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplySender.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ReplySender.java
@@ -16,13 +16,13 @@ package org.apache.geode.distributed.internal;
 
 import java.util.Set;
 
-import org.apache.geode.i18n.LogWriterI18n;
 import org.apache.geode.internal.cache.DirectReplyMessage;
 
 /**
  * This interface is used by direct ack messages to send a reply to the original sender of the
  * message. Any message which implements {@link DirectReplyMessage} must reply by calling
- * putOutgoing on the ReplySender returned by {@link DistributionMessage#getReplySender(DM)}
+ * putOutgoing on the ReplySender returned by
+ * {@link DistributionMessage#getReplySender(DistributionManager)}
  *
  * The reply sender may be the distribution manager itself, or it may send the reply directly back
  * on the same socket the message as received on.
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/SerialAckedMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/SerialAckedMessage.java
index c11d434..7755dab 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/SerialAckedMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/SerialAckedMessage.java
@@ -41,14 +41,14 @@ public class SerialAckedMessage extends SerialDistributionMessage implements Mes
   private InternalDistributedMember id;
   private int processorId;
 
-  transient DistributionManager originDm;
+  transient ClusterDistributionManager originDm;
   private transient ReplyProcessor21 rp;
 
   public SerialAckedMessage() {
     super();
     InternalDistributedSystem ds = InternalDistributedSystem.getAnyInstance();
     if (ds != null) { // this constructor is used in serialization as well as when sending to others
-      this.originDm = (DistributionManager) ds.getDistributionManager();
+      this.originDm = (ClusterDistributionManager) ds.getDistributionManager();
       this.id = this.originDm.getDistributionManagerId();
     }
   }
@@ -119,7 +119,7 @@ public class SerialAckedMessage extends SerialDistributionMessage implements Mes
    * This method is invoked on the receiver side
    */
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     Assert.assertTrue(this.id != null);
     ReplyMessage reply = new ReplyMessage();
     reply.setProcessorId(processorId);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/SerialDistributionMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/SerialDistributionMessage.java
index b9bf3f9..f745ea5 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/SerialDistributionMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/SerialDistributionMessage.java
@@ -24,7 +24,7 @@ public abstract class SerialDistributionMessage extends DistributionMessage {
   // not "final" because it's overridden in ViewMessage
   @Override
   public int getProcessorType() {
-    return DistributionManager.SERIAL_EXECUTOR;
+    return ClusterDistributionManager.SERIAL_EXECUTOR;
   }
 
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ServerLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ServerLocator.java
index 2c92610..a21bb3b 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ServerLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ServerLocator.java
@@ -292,7 +292,7 @@ public class ServerLocator implements TcpHandler, DistributionAdvisee {
   }
 
   // DistributionAdvisee methods
-  public DM getDistributionManager() {
+  public DistributionManager getDistributionManager() {
     return getSystem().getDistributionManager();
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/ShutdownMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/ShutdownMessage.java
index ed3eb1e..4c6b546 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/ShutdownMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/ShutdownMessage.java
@@ -66,7 +66,7 @@ public class ShutdownMessage extends HighPriorityDistributionMessage
    * This method is invoked on the receiver side
    */
   @Override
-  protected void process(final DistributionManager dm) {
+  protected void process(final ClusterDistributionManager dm) {
     Assert.assertTrue(this.id != null);
     // The peer goes deaf after sending us this message, so do not
     // attempt a reply.
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupMessage.java
index df7b5ba..7651ee0 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupMessage.java
@@ -63,7 +63,7 @@ public class StartupMessage extends HighPriorityDistributionMessage implements A
    * @return list of addresses for this host
    * @since GemFire 5.7
    */
-  public static Set getMyAddresses(DistributionManager dm) {
+  public static Set getMyAddresses(ClusterDistributionManager dm) {
     try {
       Set addresses = SocketCreator.getMyAddresses();
       return addresses;
@@ -176,10 +176,11 @@ public class StartupMessage extends HighPriorityDistributionMessage implements A
    * This method is invoked on the receiver side
    */
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     String rejectionMessage = null;
-    final boolean isAdminDM = dm.getId().getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE
-        || dm.getId().getVmKind() == DistributionManager.LOCATOR_DM_TYPE;
+    final boolean isAdminDM =
+        dm.getId().getVmKind() == ClusterDistributionManager.ADMIN_ONLY_DM_TYPE
+            || dm.getId().getVmKind() == ClusterDistributionManager.LOCATOR_DM_TYPE;
 
     String myVersion = GemFireVersion.getGemFireVersion();
     String theirVersion = this.version;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupMessageReplyProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupMessageReplyProcessor.java
index c0f4806..6f10b49 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupMessageReplyProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupMessageReplyProcessor.java
@@ -28,9 +28,9 @@ public class StartupMessageReplyProcessor extends ReplyProcessor21 {
    * receive replies from admin dm but they do not have the authority to accept us into the group.
    */
   private boolean receivedAcceptance;
-  private DM dm;
+  private DistributionManager dm;
 
-  public StartupMessageReplyProcessor(DM dm, Set recipients) {
+  public StartupMessageReplyProcessor(DistributionManager dm, Set recipients) {
     super(dm, recipients);
     this.dm = dm;
   }
@@ -87,7 +87,7 @@ public class StartupMessageReplyProcessor extends ReplyProcessor21 {
   @Override
   protected void preWait() {
     this.waiting = true;
-    DM mgr = getDistributionManager();
+    DistributionManager mgr = getDistributionManager();
     this.statStart = mgr.getStats().startReplyWait();
     // Note we do not use addMembershipListenerAndGetDistributionManagerIds
     // because this is the startup message and we do not yet have any
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupOperation.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupOperation.java
index 6d8ebc9..71c3272 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupOperation.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupOperation.java
@@ -30,11 +30,11 @@ import org.apache.geode.internal.logging.log4j.LocalizedMessage;
 public class StartupOperation {
   private static final Logger logger = LogService.getLogger();
 
-  DistributionManager dm;
+  ClusterDistributionManager dm;
   RemoteTransportConfig transport;
   Set newlyDeparted;
 
-  StartupOperation(DistributionManager dm, RemoteTransportConfig transport) {
+  StartupOperation(ClusterDistributionManager dm, RemoteTransportConfig transport) {
     this.dm = dm;
     this.transport = transport;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupResponseMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupResponseMessage.java
index 45ed83c..06b835b 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupResponseMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupResponseMessage.java
@@ -61,7 +61,7 @@ public class StartupResponseMessage extends HighPriorityDistributionMessage
 
   }
 
-  StartupResponseMessage(DistributionManager dm, int processorId,
+  StartupResponseMessage(ClusterDistributionManager dm, int processorId,
       InternalDistributedMember recipient, String rejectionMessage, boolean responderIsAdmin) {
     // StartupResponseMessage m = new StartupResponseMessage();
 
@@ -131,7 +131,7 @@ public class StartupResponseMessage extends HighPriorityDistributionMessage
    * This method is invoked on the receiver side
    */
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
 
     if (this.interfaces == null || this.interfaces.size() == 0) {
       // this.rejectionMessage = "Peer " + getSender() + " has no network interfaces";
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupResponseWithVersionMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupResponseWithVersionMessage.java
index 5bcbb47..c9ff742 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupResponseWithVersionMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/StartupResponseWithVersionMessage.java
@@ -24,7 +24,6 @@ import org.apache.logging.log4j.Logger;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.GemFireVersion;
-import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.logging.LogService;
 
@@ -44,7 +43,7 @@ public class StartupResponseWithVersionMessage extends StartupResponseMessage {
 
   }
 
-  StartupResponseWithVersionMessage(DistributionManager dm, int processorId,
+  StartupResponseWithVersionMessage(ClusterDistributionManager dm, int processorId,
       InternalDistributedMember recipient, String rejectionMessage, boolean responderIsAdmin) {
     super(dm, processorId, recipient, rejectionMessage, responderIsAdmin);
     version = GemFireVersion.getGemFireVersion();
@@ -56,7 +55,7 @@ public class StartupResponseWithVersionMessage extends StartupResponseMessage {
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     if (this.hostedLocators != null) {
       dm.addHostedLocators(getSender(), this.hostedLocators, this.isSharedConfigurationEnabled);
     }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/WaitForViewInstallation.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/WaitForViewInstallation.java
index c010ce7..d1e2a5f 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/WaitForViewInstallation.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/WaitForViewInstallation.java
@@ -34,7 +34,7 @@ public class WaitForViewInstallation extends HighPriorityDistributionMessage
 
   private static final Logger logger = LogService.getLogger();
 
-  public static void send(DistributionManager dm) throws InterruptedException {
+  public static void send(ClusterDistributionManager dm) throws InterruptedException {
     long viewId = dm.getMembershipManager().getView().getViewId();
     ReplyProcessor21 rp = new ReplyProcessor21(dm, dm.getOtherDistributionManagerIds());
     rp.enableSevereAlertProcessing();
@@ -57,7 +57,7 @@ public class WaitForViewInstallation extends HighPriorityDistributionMessage
 
   @Override
   public int getProcessorType() {
-    return DistributionManager.WAITING_POOL_EXECUTOR;
+    return ClusterDistributionManager.WAITING_POOL_EXECUTOR;
   }
 
   @Override
@@ -112,7 +112,7 @@ public class WaitForViewInstallation extends HighPriorityDistributionMessage
    * internal.DistributionManager)
    */
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     boolean interrupted = false;
     try {
       dm.waitForViewInstallation(this.viewId);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannel.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannel.java
index 527a611..7dd099b 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannel.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannel.java
@@ -210,7 +210,7 @@ public class DirectChannel {
    * Returns true if calling thread owns its own communication resources.
    */
   boolean threadOwnsResources() {
-    DM d = getDM();
+    DistributionManager d = getDM();
     if (d != null) {
       return d.getSystem().threadOwnsResources() && !AlertAppender.isThreadAlerting();
     }
@@ -613,7 +613,7 @@ public class DirectChannel {
    * Returns null if no stats available.
    */
   public DMStats getDMStats() {
-    DM dm = getDM();
+    DistributionManager dm = getDM();
     if (dm != null) {
       return dm.getStats(); // fix for bug#34004
     } else {
@@ -627,7 +627,7 @@ public class DirectChannel {
    * @since GemFire 4.2.2
    */
   public DistributionConfig getDMConfig() {
-    DM dm = getDM();
+    DistributionManager dm = getDM();
     if (dm != null) {
       return dm.getConfig();
     } else {
@@ -638,7 +638,7 @@ public class DirectChannel {
   /**
    * Returns null if no dm available.
    */
-  public DM getDM() {
+  public DistributionManager getDM() {
     return this.receiver.getDM();
   }
 
@@ -652,7 +652,7 @@ public class DirectChannel {
    */
   private void handleAckTimeout(long ackTimeout, long ackSATimeout, Connection c,
       DirectReplyProcessor processor) throws ConnectionException {
-    DM dm = getDM();
+    DistributionManager dm = getDM();
     Set activeMembers = dm.getDistributionManagerIds();
 
     // Increment the stat
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannelListener.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannelListener.java
index 24506fc..37fa2bf 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannelListener.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/direct/DirectChannelListener.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.distributed.internal.direct;
 
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 
 public interface DirectChannelListener {
@@ -32,6 +32,6 @@ public interface DirectChannelListener {
    *
    * @return the distribution manager
    */
-  public DistributionManager getDM();
+  public ClusterDistributionManager getDM();
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockGrantor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockGrantor.java
index 2907544..96f6e75 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockGrantor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockGrantor.java
@@ -26,8 +26,8 @@ import org.apache.geode.CancelException;
 import org.apache.geode.cache.CommitConflictException;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.LockServiceDestroyedException;
-import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.MembershipListener;
 import org.apache.geode.distributed.internal.locks.DLockQueryProcessor.DLockQueryMessage;
 import org.apache.geode.distributed.internal.locks.DLockRequestProcessor.DLockRequestMessage;
@@ -161,7 +161,7 @@ public class DLockGrantor {
   /**
    * Used to verify that requestor member is still in view when granting.
    */
-  protected final DM dm;
+  protected final DistributionManager dm;
 
   // -------------------------------------------------------------------------
   // SuspendLocking state (BEGIN)
@@ -3373,7 +3373,7 @@ public class DLockGrantor {
     }
 
     private long now() {
-      DM dm = this.grantor.dlock.getDistributionManager();
+      DistributionManager dm = this.grantor.dlock.getDistributionManager();
       return DLockService.getLockTimeStamp(dm);
     }
 
@@ -3646,7 +3646,7 @@ public class DLockGrantor {
 
     public void memberDeparted(final InternalDistributedMember id, final boolean crashed) {
       final DLockGrantor me = DLockGrantor.this;
-      final DM distMgr = me.dlock.getDistributionManager();
+      final DistributionManager distMgr = me.dlock.getDistributionManager();
       // if the VM is being forcibly disconnected, we shouldn't release locks as it
       // will take longer than the time allowed by the InternalDistributedSystem
       // shutdown mechanism.
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockQueryProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockQueryProcessor.java
index ced1ea5..9fe4ef3 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockQueryProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockQueryProcessor.java
@@ -25,7 +25,7 @@ import org.apache.geode.DataSerializer;
 import org.apache.geode.SystemFailure;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.LockServiceDestroyedException;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.MessageWithReply;
@@ -59,7 +59,8 @@ public class DLockQueryProcessor extends ReplyProcessor21 {
    * @return the query reply or null if there was no reply due to membership change
    */
   static DLockQueryReplyMessage query(final InternalDistributedMember grantor,
-      final String serviceName, final Object objectName, final boolean lockBatch, final DM dm) {
+      final String serviceName, final Object objectName, final boolean lockBatch,
+      final DistributionManager dm) {
     DLockQueryProcessor processor = new DLockQueryProcessor(dm, grantor, serviceName);
 
     DLockQueryMessage msg = new DLockQueryMessage();
@@ -98,7 +99,8 @@ public class DLockQueryProcessor extends ReplyProcessor21 {
    * @param grantor the member to query for lock leasing info
    * @param serviceName the name of the lock service
    */
-  private DLockQueryProcessor(DM dm, InternalDistributedMember grantor, String serviceName) {
+  private DLockQueryProcessor(DistributionManager dm, InternalDistributedMember grantor,
+      String serviceName) {
     super(dm, grantor);
   }
 
@@ -152,7 +154,7 @@ public class DLockQueryProcessor extends ReplyProcessor21 {
      * Processes this message - invoked on the node that is the lock grantor.
      */
     @Override
-    protected void process(final DistributionManager dm) {
+    protected void process(final ClusterDistributionManager dm) {
       boolean failed = true;
       ReplyException replyException = null;
       try {
@@ -197,7 +199,7 @@ public class DLockQueryProcessor extends ReplyProcessor21 {
     }
 
     /** Process locally without using messaging or executor */
-    protected void processLocally(final DM dm) {
+    protected void processLocally(final DistributionManager dm) {
       this.svc = DLockService.getInternalServiceNamed(this.serviceName);
       basicProcess(dm, true); // don't use executor
     }
@@ -208,7 +210,7 @@ public class DLockQueryProcessor extends ReplyProcessor21 {
      * <p>
      * this.svc and this.grantor must be set before calling this method.
      */
-    private void executeBasicProcess(final DM dm) {
+    private void executeBasicProcess(final DistributionManager dm) {
       final DLockQueryMessage msg = this;
       dm.getWaitingThreadPool().execute(new Runnable() {
         public void run() {
@@ -225,7 +227,7 @@ public class DLockQueryProcessor extends ReplyProcessor21 {
      * <p>
      * this.svc and this.grantor must be set before calling this method.
      */
-    protected void basicProcess(final DM dm, final boolean waitForGrantor) {
+    protected void basicProcess(final DistributionManager dm, final boolean waitForGrantor) {
       final boolean isDebugEnabled_DLS = logger.isTraceEnabled(LogMarker.DLS);
       if (isDebugEnabled_DLS) {
         logger.trace(LogMarker.DLS, "[basicProcess] {}", this);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRecoverGrantorProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRecoverGrantorProcessor.java
index 1423bde..3b7e415 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRecoverGrantorProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRecoverGrantorProcessor.java
@@ -25,7 +25,7 @@ import java.util.Set;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.MessageWithReply;
@@ -52,7 +52,7 @@ public class DLockRecoverGrantorProcessor extends ReplyProcessor21 {
   protected static final DefaultMessageProcessor nullServiceProcessor =
       new DefaultMessageProcessor();
 
-  private DM dm;
+  private DistributionManager dm;
 
   private DLockGrantor newGrantor;
 
@@ -68,7 +68,7 @@ public class DLockRecoverGrantorProcessor extends ReplyProcessor21 {
    * This method should block until transfer of lock grantor has completed.
    */
   static boolean recoverLockGrantor(Set members, DLockService service, DLockGrantor newGrantor,
-      DM dm, InternalDistributedMember elder) {
+      DistributionManager dm, InternalDistributedMember elder) {
     // proc will wait for replies from everyone including THIS member...
     DLockRecoverGrantorProcessor processor =
         new DLockRecoverGrantorProcessor(dm, members, newGrantor);
@@ -112,7 +112,8 @@ public class DLockRecoverGrantorProcessor extends ReplyProcessor21 {
   // -------------------------------------------------------------------------
 
   /** Creates a new instance of DLockRecoverGrantorProcessor */
-  private DLockRecoverGrantorProcessor(DM dm, Set members, DLockGrantor newGrantor) {
+  private DLockRecoverGrantorProcessor(DistributionManager dm, Set members,
+      DLockGrantor newGrantor) {
     super(dm, members);
     this.dm = dm;
     this.newGrantor = newGrantor;
@@ -235,7 +236,7 @@ public class DLockRecoverGrantorProcessor extends ReplyProcessor21 {
     }
 
     @Override
-    protected void process(DistributionManager dm) {
+    protected void process(ClusterDistributionManager dm) {
       processMessage(dm);
     }
 
@@ -245,15 +246,15 @@ public class DLockRecoverGrantorProcessor extends ReplyProcessor21 {
      *
      * @param dm the distribution manager
      */
-    protected void scheduleMessage(DM dm) {
-      if (dm instanceof DistributionManager) {
-        super.scheduleAction((DistributionManager) dm);
+    protected void scheduleMessage(DistributionManager dm) {
+      if (dm instanceof ClusterDistributionManager) {
+        super.scheduleAction((ClusterDistributionManager) dm);
       } else {
         processMessage(dm);
       }
     }
 
-    protected void processMessage(DM dm) {
+    protected void processMessage(DistributionManager dm) {
       MessageProcessor processor = nullServiceProcessor;
 
       DLockService svc = DLockService.getInternalServiceNamed(this.serviceName);
@@ -383,11 +384,11 @@ public class DLockRecoverGrantorProcessor extends ReplyProcessor21 {
   }
 
   public static interface MessageProcessor {
-    public void process(DM dm, DLockRecoverGrantorMessage msg);
+    public void process(DistributionManager dm, DLockRecoverGrantorMessage msg);
   }
 
   static class DefaultMessageProcessor implements MessageProcessor {
-    public void process(DM dm, DLockRecoverGrantorMessage msg) {
+    public void process(DistributionManager dm, DLockRecoverGrantorMessage msg) {
       ReplyException replyException = null;
       int replyCode = DLockRecoverGrantorReplyMessage.OK;
       DLockRemoteToken[] heldLocks = new DLockRemoteToken[0];
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockReleaseProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockReleaseProcessor.java
index b1045f1..14eb070 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockReleaseProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockReleaseProcessor.java
@@ -24,7 +24,7 @@ import org.apache.logging.log4j.Logger;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.SystemFailure;
 import org.apache.geode.distributed.LockServiceDestroyedException;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
@@ -48,8 +48,8 @@ public class DLockReleaseProcessor extends ReplyProcessor21 {
 
   protected Object objectName;
 
-  public DLockReleaseProcessor(DM dm, InternalDistributedMember member, String serviceName,
-      Object objectName) {
+  public DLockReleaseProcessor(DistributionManager dm, InternalDistributedMember member,
+      String serviceName, Object objectName) {
     super(dm, member);
     this.objectName = objectName;
   }
@@ -61,7 +61,7 @@ public class DLockReleaseProcessor extends ReplyProcessor21 {
   protected boolean release(InternalDistributedMember grantor, String serviceName,
       boolean lockBatch, int lockId) {
 
-    DM dm = getDistributionManager();
+    DistributionManager dm = getDistributionManager();
     DLockReleaseMessage msg = new DLockReleaseMessage();
     msg.processorId = getProcessorId();
     msg.serviceName = serviceName;
@@ -164,7 +164,7 @@ public class DLockReleaseProcessor extends ReplyProcessor21 {
      * Processes this message - invoked on the node that is the lock grantor.
      */
     @Override
-    protected void process(final DistributionManager dm) {
+    protected void process(final ClusterDistributionManager dm) {
       boolean failed = true;
       ReplyException replyException = null;
       try {
@@ -212,7 +212,7 @@ public class DLockReleaseProcessor extends ReplyProcessor21 {
     }
 
     /** Process locally without using messaging or executor */
-    protected void processLocally(final DM dm) {
+    protected void processLocally(final DistributionManager dm) {
       this.svc = DLockService.getInternalServiceNamed(this.serviceName);
       basicProcess(dm, true); // don't use executor
     }
@@ -223,7 +223,7 @@ public class DLockReleaseProcessor extends ReplyProcessor21 {
      * <p>
      * this.svc and this.grantor must be set before calling this method.
      */
-    private void executeBasicProcess(final DM dm) {
+    private void executeBasicProcess(final DistributionManager dm) {
       final DLockReleaseMessage msg = this;
       dm.getWaitingThreadPool().execute(new Runnable() {
         public void run() {
@@ -240,7 +240,7 @@ public class DLockReleaseProcessor extends ReplyProcessor21 {
      * <p>
      * this.svc and this.grantor must be set before calling this method.
      */
-    protected void basicProcess(final DM dm, final boolean waitForGrantor) {
+    protected void basicProcess(final DistributionManager dm, final boolean waitForGrantor) {
       final boolean isDebugEnabled_DLS = logger.isTraceEnabled(LogMarker.DLS);
       if (isDebugEnabled_DLS) {
         logger.trace(LogMarker.DLS, "[basicProcess] {}", this);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRequestProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRequestProcessor.java
index 3f42adb..9fd6c4b 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRequestProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockRequestProcessor.java
@@ -27,7 +27,7 @@ import org.apache.geode.DataSerializer;
 import org.apache.geode.InternalGemFireError;
 import org.apache.geode.SystemFailure;
 import org.apache.geode.distributed.LockServiceDestroyedException;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
@@ -56,7 +56,7 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
 
   protected final DLockRequestMessage request;
 
-  private final DM dm;
+  private final DistributionManager dm;
 
   protected final DLockService svc;
 
@@ -89,28 +89,28 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
 
   protected DLockRequestProcessor(LockGrantorId lockGrantorId, DLockService svc, Object objectName,
       int threadId, long startTime, long leaseMillis, long waitMillis, boolean reentrant,
-      boolean tryLock, DM dm) {
+      boolean tryLock, DistributionManager dm) {
     this(lockGrantorId, svc, objectName, threadId, startTime, leaseMillis, waitMillis, reentrant,
         tryLock, false, dm, false);
   }
 
   protected DLockRequestProcessor(LockGrantorId lockGrantorId, DLockService svc, Object objectName,
       int threadId, long startTime, long leaseMillis, long waitMillis, boolean reentrant,
-      boolean tryLock, boolean disableAlerts, DM dm) {
+      boolean tryLock, boolean disableAlerts, DistributionManager dm) {
     this(lockGrantorId, svc, objectName, threadId, startTime, leaseMillis, waitMillis, reentrant,
         tryLock, disableAlerts, dm, false);
   }
 
   protected DLockRequestProcessor(LockGrantorId lockGrantorId, DLockService svc, Object objectName,
       int threadId, long startTime, long leaseMillis, long waitMillis, boolean reentrant,
-      boolean tryLock, DM dm, boolean async) {
+      boolean tryLock, DistributionManager dm, boolean async) {
     this(lockGrantorId, svc, objectName, threadId, startTime, leaseMillis, waitMillis, reentrant,
         tryLock, false, dm, false);
   }
 
   protected DLockRequestProcessor(LockGrantorId lockGrantorId, DLockService svc, Object objectName,
       int threadId, long startTime, long leaseMillis, long waitMillis, boolean reentrant,
-      boolean tryLock, boolean disableAlerts, DM dm, boolean async) {
+      boolean tryLock, boolean disableAlerts, DistributionManager dm, boolean async) {
     super(dm, lockGrantorId.getLockGrantorMember());
 
     this.svc = svc;
@@ -142,7 +142,7 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
     return new DLockRequestMessage();
   }
 
-  protected CancelCriterion getCancelCriterion(DM ignoreDM) {
+  protected CancelCriterion getCancelCriterion(DistributionManager ignoreDM) {
     return this.svc.getCancelCriterion();
   }
 
@@ -440,7 +440,7 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
     protected transient DLockService svc;
     protected transient DLockGrantor grantor;
     private transient long statStart = -1;
-    private transient volatile DM receivingDM;
+    private transient volatile DistributionManager receivingDM;
     private transient DLockResponseMessage response;
     private transient RemoteThread rThread;
 
@@ -533,7 +533,7 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
      * Processes this message - invoked on the node that is the lock grantor.
      */
     @Override
-    protected void process(final DistributionManager dm) {
+    protected void process(final ClusterDistributionManager dm) {
       boolean failed = false;
       Throwable replyException = null;
       try {
@@ -575,7 +575,7 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
     }
 
     /** Process locally without using messaging or executor */
-    protected void processLocally(final DM dm) {
+    protected void processLocally(final DistributionManager dm) {
       this.statStart = startGrantWait();
       this.svc = DLockService.getInternalServiceNamed(this.serviceName);
       basicProcess(dm, true); // don't use executor
@@ -587,7 +587,7 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
      * <p>
      * this.svc and this.grantor must be set before calling this method.
      */
-    private void executeBasicProcess(final DM dm) {
+    private void executeBasicProcess(final DistributionManager dm) {
       final DLockRequestMessage msg = this;
       dm.getWaitingThreadPool().execute(new Runnable() {
         public void run() {
@@ -599,7 +599,7 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
       });
     }
 
-    protected void basicProcess(final DM dm, final boolean waitForGrantor) {
+    protected void basicProcess(final DistributionManager dm, final boolean waitForGrantor) {
       final boolean isDebugEnabled_DLS = logger.isTraceEnabled(LogMarker.DLS);
       try {
         this.receivingDM = dm;
@@ -1068,7 +1068,7 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
      * receiving a GRANT response which may already be in transit to this node.
      */
     @Override
-    public void process(final DM dm, final ReplyProcessor21 processor) {
+    public void process(final DistributionManager dm, final ReplyProcessor21 processor) {
       if (processor == null) {
         // The processor was probably cleaned up because of memberDeparted and we need to abandon
         return;
@@ -1099,7 +1099,8 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
       }
     }
 
-    protected boolean callReleaseProcessor(DM dm, InternalDistributedMember grantor) {
+    protected boolean callReleaseProcessor(DistributionManager dm,
+        InternalDistributedMember grantor) {
       return DLockService.callReleaseProcessor(dm, this.serviceName, grantor, this.objectName,
           false, this.lockId);
     }
@@ -1108,7 +1109,7 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
      * Releases a granted lock that was orphaned by interruption of the lock request. This also
      * releases any lock grant for which we cannot find an active reply processor.
      */
-    public void releaseOrphanedGrant(DM dm) {
+    public void releaseOrphanedGrant(DistributionManager dm) {
       InternalDistributedMember grantor = getSender();
       // method is rewritten to fix bug 35252
       boolean released = false;
@@ -1261,7 +1262,7 @@ public class DLockRequestProcessor extends ReplyProcessor21 {
     }
   }
 
-  public static void waitToProcessDLockResponse(DM dm) {
+  public static void waitToProcessDLockResponse(DistributionManager dm) {
     synchronized (waitToProcessDLockResponseLock) {
       while (waitToProcessDLockResponse) {
         dm.getCancelCriterion().checkCancelInProgress(null);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java
index 4008d2e..4434044 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockService.java
@@ -44,8 +44,8 @@ import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.LeaseExpiredException;
 import org.apache.geode.distributed.LockNotHeldException;
 import org.apache.geode.distributed.LockServiceDestroyedException;
-import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionConfig;
+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.deadlock.UnsafeThreadLocal;
@@ -91,7 +91,7 @@ public class DLockService extends DistributedLockService {
   protected final String serviceName;
 
   /** DistributionManager for this member */
-  private final DM dm;
+  private final DistributionManager dm;
 
   /**
    * DistributedSystem connection for this member (used for DisconnectListener, logging, etc)
@@ -1319,7 +1319,7 @@ public class DLockService extends DistributedLockService {
         lockId);
   }
 
-  protected static boolean callReleaseProcessor(DM dm, String serviceName,
+  protected static boolean callReleaseProcessor(DistributionManager dm, String serviceName,
       InternalDistributedMember grantor, Object name, boolean lockBatch, int lockId) {
     DLockReleaseProcessor processor = new DLockReleaseProcessor(dm, grantor, serviceName, name);
     return processor.release(grantor, serviceName, lockBatch, lockId);
@@ -2168,7 +2168,7 @@ public class DLockService extends DistributedLockService {
     return this.serviceName;
   }
 
-  public DM getDistributionManager() {
+  public DistributionManager getDistributionManager() {
     return this.dm;
   }
 
@@ -2852,7 +2852,7 @@ public class DLockService extends DistributedLockService {
    *
    * @param dm our local DM
    */
-  public static void recoverLocalElder(DM dm, Map grantors, Set needsRecovery) {
+  public static void recoverLocalElder(DistributionManager dm, Map grantors, Set needsRecovery) {
     synchronized (services) {
       Iterator entries = services.entrySet().iterator();
       while (entries.hasNext()) {
@@ -2952,7 +2952,7 @@ public class DLockService extends DistributedLockService {
    *
    * @since GemFire 3.5
    */
-  static long getLockTimeStamp(DM dm) {
+  static long getLockTimeStamp(DistributionManager dm) {
     return dm.cacheTimeMillis();
   }
 
@@ -3097,7 +3097,7 @@ public class DLockService extends DistributedLockService {
      * @param dm the DM to check for shutdown
      * @param dls the DLockService to check for DLS destroy
      */
-    DLockStopper(DM dm, DLockService dls) {
+    DLockStopper(DistributionManager dm, DLockService dls) {
       Assert.assertTrue(dls != null);
       this.dls = dls;
       Assert.assertTrue(dls.getDistributionManager() != null);
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockToken.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockToken.java
index 9d36d74..1a1a68d 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockToken.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DLockToken.java
@@ -20,7 +20,7 @@ import java.util.WeakHashMap;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.distributed.LeaseExpiredException;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
@@ -46,9 +46,9 @@ public class DLockToken {
 
   /**
    * DistributionManager using this lock token. Reference is used to identify local member identity
-   * and to {@link DLockService#getLockTimeStamp(DM)}.
+   * and to {@link DLockService#getLockTimeStamp(DistributionManager)}.
    */
-  private final DM dm;
+  private final DistributionManager dm;
 
   /**
    * The reply processor id is used to identify the distinct lease which a thread has used to lease
@@ -112,7 +112,7 @@ public class DLockToken {
    * @param dm the DistributionManager for this member
    * @param name the identifying name of this lock
    */
-  public DLockToken(DM dm, Object name) {
+  public DLockToken(DistributionManager dm, Object name) {
     this.dm = dm;
     this.name = name;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DeposeGrantorProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DeposeGrantorProcessor.java
index 92c36ea..17afdd9 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DeposeGrantorProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DeposeGrantorProcessor.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.MessageWithReply;
 import org.apache.geode.distributed.internal.PooledDistributionMessage;
@@ -52,7 +52,7 @@ public class DeposeGrantorProcessor extends ReplyProcessor21 {
    */
   static void send(String serviceName, InternalDistributedMember oldGrantor,
       InternalDistributedMember newGrantor, long newGrantorVersion, int newGrantorSerialNumber,
-      DM dm) {
+      DistributionManager dm) {
     final InternalDistributedMember elder = dm.getId();
     if (elder.equals(oldGrantor)) {
       doOldGrantorWork(serviceName, elder, newGrantor, newGrantorVersion, newGrantorSerialNumber,
@@ -71,7 +71,7 @@ public class DeposeGrantorProcessor extends ReplyProcessor21 {
 
   protected static void doOldGrantorWork(final String serviceName,
       final InternalDistributedMember elder, final InternalDistributedMember youngTurk,
-      final long newGrantorVersion, final int newGrantorSerialNumber, final DM dm,
+      final long newGrantorVersion, final int newGrantorSerialNumber, final DistributionManager dm,
       final DeposeGrantorMessage msg) {
     try {
       DLockService svc = DLockService.getInternalServiceNamed(serviceName);
@@ -92,7 +92,7 @@ public class DeposeGrantorProcessor extends ReplyProcessor21 {
   /**
    * Creates a new instance of DeposeGrantorProcessor
    */
-  private DeposeGrantorProcessor(DM dm, InternalDistributedMember oldGrantor) {
+  private DeposeGrantorProcessor(DistributionManager dm, InternalDistributedMember oldGrantor) {
     super(dm, oldGrantor);
   }
 
@@ -109,7 +109,7 @@ public class DeposeGrantorProcessor extends ReplyProcessor21 {
 
     protected static void send(String serviceName, InternalDistributedMember oldGrantor,
         InternalDistributedMember newGrantor, long newGrantorVersion, int newGrantorSerialNumber,
-        DM dm, ReplyProcessor21 proc) {
+        DistributionManager dm, ReplyProcessor21 proc) {
       DeposeGrantorMessage msg = new DeposeGrantorMessage();
       msg.serviceName = serviceName;
       msg.newGrantor = newGrantor;
@@ -128,12 +128,12 @@ public class DeposeGrantorProcessor extends ReplyProcessor21 {
       return this.processorId;
     }
 
-    void reply(DM dm) {
+    void reply(DistributionManager dm) {
       ReplyMessage.send(this.getSender(), this.getProcessorId(), null, dm);
     }
 
     @Override
-    protected void process(final DistributionManager dm) {
+    protected void process(final ClusterDistributionManager dm) {
       // if we are currently the grantor then
       // mark it as being destroyed until we hear from this.newGrantor
       // or he goes away or the grantor that sent us this message goes away.
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DistributedMemberLock.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DistributedMemberLock.java
index 54fe848..1996ae6 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DistributedMemberLock.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/DistributedMemberLock.java
@@ -20,7 +20,7 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 
 import org.apache.geode.distributed.DistributedLockService;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.locks.DLockService.ThreadRequestState;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.i18n.LocalizedStrings;
@@ -244,7 +244,7 @@ public class DistributedMemberLock implements Lock {
     }
   }
 
-  private DM getDM() {
+  private DistributionManager getDM() {
     return this.dls.getDistributionManager();
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/ElderInitProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/ElderInitProcessor.java
index 9f4914a..71f2b47 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/ElderInitProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/ElderInitProcessor.java
@@ -27,7 +27,7 @@ import java.util.Set;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.MessageWithReply;
@@ -61,7 +61,7 @@ public class ElderInitProcessor extends ReplyProcessor21 {
    * Initializes ElderState map by recovering all existing grantors and crashed grantors in the
    * current ds.
    */
-  static void init(DM dm, HashMap map) {
+  static void init(DistributionManager dm, HashMap map) {
     HashSet crashedGrantors = new HashSet();
     if (!dm.isAdam()) {
       Set others = dm.getOtherDistributionManagerIds();
@@ -90,7 +90,8 @@ public class ElderInitProcessor extends ReplyProcessor21 {
   /**
    * Creates a new instance of ElderInitProcessor
    */
-  private ElderInitProcessor(DM dm, Set others, HashMap grantors, HashSet crashedGrantors) {
+  private ElderInitProcessor(DistributionManager dm, Set others, HashMap grantors,
+      HashSet crashedGrantors) {
     super(dm/* fix bug 33297 */, others);
     this.grantors = grantors;
     this.crashedGrantors = crashedGrantors;
@@ -147,7 +148,7 @@ public class ElderInitProcessor extends ReplyProcessor21 {
       implements MessageWithReply {
     private int processorId;
 
-    protected static void send(Set others, DM dm, ReplyProcessor21 proc) {
+    protected static void send(Set others, DistributionManager dm, ReplyProcessor21 proc) {
       ElderInitMessage msg = new ElderInitMessage();
       msg.processorId = proc.getProcessorId();
       msg.setRecipients(others);
@@ -162,14 +163,14 @@ public class ElderInitProcessor extends ReplyProcessor21 {
       return this.processorId;
     }
 
-    private void reply(DM dm, ArrayList grantors, ArrayList grantorVersions,
+    private void reply(DistributionManager dm, ArrayList grantors, ArrayList grantorVersions,
         ArrayList grantorSerialNumbers, ArrayList nonGrantors) {
       ElderInitReplyMessage.send(this, dm, grantors, grantorVersions, grantorSerialNumbers,
           nonGrantors);
     }
 
     @Override
-    protected void process(DistributionManager dm) {
+    protected void process(ClusterDistributionManager dm) {
       ArrayList grantors = new ArrayList(); // svc names grantor for
       ArrayList grantorVersions = new ArrayList(); // grantor versions
       ArrayList grantorSerialNumbers = new ArrayList(); // serial numbers of grantor svcs
@@ -228,7 +229,7 @@ public class ElderInitProcessor extends ReplyProcessor21 {
     private ArrayList grantorSerialNumbers; // grantor dls serial number ints
     private ArrayList nonGrantors; // svc names
 
-    public static void send(MessageWithReply reqMsg, DM dm, ArrayList grantors,
+    public static void send(MessageWithReply reqMsg, DistributionManager dm, ArrayList grantors,
         ArrayList grantorVersions, ArrayList grantorSerialNumbers, ArrayList nonGrantors) {
       ElderInitReplyMessage m = new ElderInitReplyMessage();
       m.grantors = grantors;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/ElderState.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/ElderState.java
index 58003d8..994ddc1 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/ElderState.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/ElderState.java
@@ -21,7 +21,7 @@ import java.util.Iterator;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.InternalGemFireError;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.i18n.LocalizedStrings;
@@ -41,13 +41,13 @@ public class ElderState {
    * Maps service name keys to GrantorInfo values.
    */
   private final HashMap nameToInfo;
-  private final DM dm;
+  private final DistributionManager dm;
 
   /**
    * Constructs the EdlerState for the given dm. Note that this constructor does not complete until
    * elder recovery is complete.
    */
-  public ElderState(DM dm) {
+  public ElderState(DistributionManager dm) {
     Assert.assertTrue(dm != null);
     this.dm = dm;
     this.nameToInfo = new HashMap();
@@ -81,7 +81,7 @@ public class ElderState {
     }
   }
 
-  private void checkForProblem(DM checkDM) {
+  private void checkForProblem(DistributionManager checkDM) {
     if (checkDM.getSystem() == null) {
       logger.warn(LogMarker.DLS, LocalizedMessage
           .create(LocalizedStrings.ElderState_ELDERSTATE_PROBLEM_SYSTEM_0, checkDM.getSystem()));
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/GrantorRequestProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/GrantorRequestProcessor.java
index 221f568..6d73f4a 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/GrantorRequestProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/GrantorRequestProcessor.java
@@ -19,14 +19,12 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.Set;
-import java.util.concurrent.*;
-import java.util.concurrent.locks.*;
 
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
@@ -222,7 +220,7 @@ public class GrantorRequestProcessor extends ReplyProcessor21 {
     InternalDistributedMember elder;
     ElderState es = null;
 
-    final DM dm = sys.getDistributionManager();
+    final DistributionManager dm = sys.getDistributionManager();
     boolean elderCallStarted = false;
     while (!elderCallStarted) {
       dm.throwIfDistributionStopped();
@@ -334,7 +332,7 @@ public class GrantorRequestProcessor extends ReplyProcessor21 {
       int dlsSerialNumber, InternalDistributedSystem system, InternalDistributedMember oldTurk,
       byte opCode) {
     GrantorInfo result = null;
-    DM dm = system.getDistributionManager();
+    DistributionManager dm = system.getDistributionManager();
     GrantorRequestContext grc = system.getGrantorRequestContext();
     boolean tryNewElder;
     boolean interrupted = false;
@@ -478,7 +476,7 @@ public class GrantorRequestProcessor extends ReplyProcessor21 {
      * @return true if the message was sent
      */
     protected static boolean send(long grantorVersion, int dlsSerialNumber, String serviceName,
-        InternalDistributedMember elder, DM dm, ReplyProcessor21 proc,
+        InternalDistributedMember elder, DistributionManager dm, ReplyProcessor21 proc,
         InternalDistributedMember oldTurk, byte opCode) {
       // bug36361: the following assertion doesn't work, since the client that sent us
       // the request might have a different notion of the elder (no view synchrony on the
@@ -512,16 +510,16 @@ public class GrantorRequestProcessor extends ReplyProcessor21 {
       return this.processorId;
     }
 
-    private void replyGrantorInfo(DM dm, GrantorInfo gi) {
+    private void replyGrantorInfo(DistributionManager dm, GrantorInfo gi) {
       GrantorInfoReplyMessage.send(this, dm, gi);
     }
 
-    private void replyClear(DM dm) {
+    private void replyClear(DistributionManager dm) {
       ReplyMessage.send(this.getSender(), this.getProcessorId(), null, dm);
     }
 
     @Override
-    protected void process(DistributionManager dm) {
+    protected void process(ClusterDistributionManager dm) {
       // executeBasicProcess(dm); // TODO change to this after things are stable
       basicProcess(dm);
     }
@@ -540,7 +538,7 @@ public class GrantorRequestProcessor extends ReplyProcessor21 {
     // }
     // }
 
-    protected void basicProcess(final DM dm) {
+    protected void basicProcess(final DistributionManager dm) {
       // we should be in the elder
       ElderState es = dm.getElderState(true, false);
       switch (this.opCode) {
@@ -643,7 +641,7 @@ public class GrantorRequestProcessor extends ReplyProcessor21 {
     private int grantorSerialNumber;
     private boolean needsRecovery;
 
-    public static void send(MessageWithReply reqMsg, DM dm, GrantorInfo gi) {
+    public static void send(MessageWithReply reqMsg, DistributionManager dm, GrantorInfo gi) {
       GrantorInfoReplyMessage m = new GrantorInfoReplyMessage();
       m.grantor = gi.getId();
       m.needsRecovery = gi.needsRecovery();
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/LockGrantorId.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/LockGrantorId.java
index a887d84..e86d828 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/LockGrantorId.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/LockGrantorId.java
@@ -14,8 +14,8 @@
  */
 package org.apache.geode.distributed.internal.locks;
 
-import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 
@@ -31,7 +31,7 @@ public class LockGrantorId {
           10000)
       .intValue();
 
-  private final DM dm;
+  private final DistributionManager dm;
   private final InternalDistributedMember lockGrantorMember;
   private final long lockGrantorVersion;
   private final int lockGrantorSerialNumber;
@@ -43,8 +43,8 @@ public class LockGrantorId {
    * @param lockGrantorMember the non-null member hosting the grantor
    * @param lockGrantorVersion the long grantor version number
    */
-  public LockGrantorId(DM dm, InternalDistributedMember lockGrantorMember, long lockGrantorVersion,
-      int lockGrantorSerialNumber) {
+  public LockGrantorId(DistributionManager dm, InternalDistributedMember lockGrantorMember,
+      long lockGrantorVersion, int lockGrantorSerialNumber) {
     if (lockGrantorMember == null) {
       throw new NullPointerException(
           LocalizedStrings.LockGrantorId_LOCKGRANTORMEMBER_IS_NULL.toLocalizedString());
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/NonGrantorDestroyedProcessor.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/NonGrantorDestroyedProcessor.java
index cfc1888..615498d 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/NonGrantorDestroyedProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/locks/NonGrantorDestroyedProcessor.java
@@ -23,7 +23,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
 import org.apache.geode.distributed.LockServiceDestroyedException;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.MessageWithReply;
@@ -57,7 +57,7 @@ public class NonGrantorDestroyedProcessor extends ReplyProcessor21 {
    * Caller should loop, getting the grantor, calling <code>send</code>, and checking
    * <code>informedGrantor()</code> until the grantor has acknowledged being informed.
    */
-  static boolean send(String serviceName, LockGrantorId theLockGrantorId, DM dm) {
+  static boolean send(String serviceName, LockGrantorId theLockGrantorId, DistributionManager dm) {
     InternalDistributedMember recipient = theLockGrantorId.getLockGrantorMember();
     NonGrantorDestroyedProcessor processor = new NonGrantorDestroyedProcessor(dm, recipient);
     NonGrantorDestroyedMessage.send(serviceName, recipient, dm, processor);
@@ -72,7 +72,7 @@ public class NonGrantorDestroyedProcessor extends ReplyProcessor21 {
   //////////// Instance methods //////////////
 
   /** Creates a new instance of NonGrantorDestroyedProcessor */
-  private NonGrantorDestroyedProcessor(DM dm, InternalDistributedMember grantor) {
+  private NonGrantorDestroyedProcessor(DistributionManager dm, InternalDistributedMember grantor) {
     super(dm, grantor);
   }
 
@@ -108,8 +108,8 @@ public class NonGrantorDestroyedProcessor extends ReplyProcessor21 {
     /** The name of the DistributedLockService */
     private String serviceName;
 
-    protected static void send(String serviceName, InternalDistributedMember grantor, DM dm,
-        ReplyProcessor21 proc) {
+    protected static void send(String serviceName, InternalDistributedMember grantor,
+        DistributionManager dm, ReplyProcessor21 proc) {
       Assert.assertTrue(grantor != null, "Cannot send NonGrantorDestroyedMessage to null grantor");
 
       NonGrantorDestroyedMessage msg = new NonGrantorDestroyedMessage();
@@ -134,22 +134,22 @@ public class NonGrantorDestroyedProcessor extends ReplyProcessor21 {
       return this.processorId;
     }
 
-    private void reply(byte replyCode, DM dm) {
+    private void reply(byte replyCode, DistributionManager dm) {
       NonGrantorDestroyedReplyMessage.send(this, replyCode, dm);
     }
 
     @Override
-    protected void process(DistributionManager dm) {
+    protected void process(ClusterDistributionManager dm) {
       basicProcess(dm);
     }
 
     /** Process locally without using messaging */
-    protected void processLocally(final DM dm) {
+    protected void processLocally(final DistributionManager dm) {
       basicProcess(dm);
     }
 
     /** Perform basic processing of this message */
-    private void basicProcess(final DM dm) {
+    private void basicProcess(final DistributionManager dm) {
       boolean replied = false;
       try {
         DLockService svc = DLockService.getInternalServiceNamed(this.serviceName);
@@ -222,7 +222,7 @@ public class NonGrantorDestroyedProcessor extends ReplyProcessor21 {
 
     private byte replyCode;
 
-    public static void send(MessageWithReply destroyedMsg, byte replyCode, DM dm) {
+    public static void send(MessageWithReply destroyedMsg, byte replyCode, DistributionManager dm) {
       NonGrantorDestroyedReplyMessage m = new NonGrantorDestroyedReplyMessage();
       m.processorId = destroyedMsg.getProcessorId();
       m.setRecipient(destroyedMsg.getSender());
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/DistributedMembershipListener.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/DistributedMembershipListener.java
index 73d896d..181db45 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/DistributedMembershipListener.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/DistributedMembershipListener.java
@@ -17,7 +17,6 @@ package org.apache.geode.distributed.internal.membership;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.direct.DirectChannelListener;
 
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 b5db42c..f17a766 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
@@ -15,7 +15,6 @@
 package org.apache.geode.distributed.internal.membership;
 
 import java.io.*;
-import java.net.Inet4Address;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
@@ -27,9 +26,9 @@ import org.apache.geode.cache.UnsupportedVersionException;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DurableClientAttributes;
 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.DistributionConfig;
-import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.internal.*;
 import org.apache.geode.internal.cache.versions.VersionSource;
@@ -219,7 +218,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
       throw new GemFireConfigException("Unable to resolve server location " + location, e);
     }
     netMbr = MemberFactory.newNetMember(addr, location.getPort());
-    netMbr.setVmKind(DistributionManager.NORMAL_DM_TYPE);
+    netMbr.setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
     versionObj = Version.CURRENT;
     netMbr.setVersion(versionObj);
   }
@@ -253,7 +252,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
   public InternalDistributedMember(String i, int p, Version version, NetMember netMember) {
     netMbr = netMember;
     defaultToCurrentHost();
-    netMember.setVmKind(DistributionManager.NORMAL_DM_TYPE);
+    netMember.setVmKind(ClusterDistributionManager.NORMAL_DM_TYPE);
     this.versionObj = version;
     netMember.setVersion(version);
   }
@@ -364,8 +363,8 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
   /**
    * [GemStone] Returns the kind of VM that hosts the distribution manager with this address.
    *
-   * @see org.apache.geode.distributed.internal.DistributionManager#getDMType()
-   * @see org.apache.geode.distributed.internal.DistributionManager#NORMAL_DM_TYPE
+   * @see ClusterDistributionManager#getDMType()
+   * @see ClusterDistributionManager#NORMAL_DM_TYPE
    */
   public int getVmKind() {
     return netMbr.getVmKind();
@@ -700,7 +699,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     String myName = getName();
     int vmPid = netMbr.getProcessId();
     int vmKind = netMbr.getVmKind();
-    if (vmPid > 0 || vmKind != DistributionManager.NORMAL_DM_TYPE || !"".equals(myName)) {
+    if (vmPid > 0 || vmKind != ClusterDistributionManager.NORMAL_DM_TYPE || !"".equals(myName)) {
       sb.append("(");
 
       if (!"".equals(myName)) {
@@ -715,16 +714,16 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
 
       String vmStr = "";
       switch (vmKind) {
-        case DistributionManager.NORMAL_DM_TYPE:
+        case ClusterDistributionManager.NORMAL_DM_TYPE:
           // vmStr = ":local"; // let this be silent
           break;
-        case DistributionManager.LOCATOR_DM_TYPE:
+        case ClusterDistributionManager.LOCATOR_DM_TYPE:
           vmStr = ":locator";
           break;
-        case DistributionManager.ADMIN_ONLY_DM_TYPE:
+        case ClusterDistributionManager.ADMIN_ONLY_DM_TYPE:
           vmStr = ":admin";
           break;
-        case DistributionManager.LONER_DM_TYPE:
+        case ClusterDistributionManager.LONER_DM_TYPE:
           vmStr = ":loner";
           break;
         default:
@@ -734,7 +733,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
       sb.append(vmStr);
       sb.append(")");
     }
-    if (vmKind != DistributionManager.LONER_DM_TYPE && netMbr.preferredForCoordinator()) {
+    if (vmKind != ClusterDistributionManager.LONER_DM_TYPE && netMbr.preferredForCoordinator()) {
       sb.append("<ec>");
     }
     int vmViewId = getVmViewId();
@@ -749,7 +748,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     // sb.append(Integer.toString(dcPort));
     // }
 
-    if (vmKind == DistributionManager.LONER_DM_TYPE) {
+    if (vmKind == ClusterDistributionManager.LONER_DM_TYPE) {
       // add some more info that was added in 4.2.1 for loner bridge clients
       // impact on non-bridge loners is ok
       if (this.uniqueTag != null && this.uniqueTag.length() != 0) {
@@ -913,7 +912,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     DataSerializer.writeStringArray(netMbr.getGroups(), out);
 
     DataSerializer.writeString(netMbr.getName(), out);
-    if (vmKind == DistributionManager.LONER_DM_TYPE) {
+    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);
@@ -961,7 +960,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
 
     DataSerializer.writeString(netMbr.getName(), out);
     int vmKind = netMbr.getVmKind();
-    if (vmKind == DistributionManager.LONER_DM_TYPE) {
+    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);
@@ -1008,7 +1007,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     int vmViewId = -1;
 
     String name = DataSerializer.readString(in);
-    if (vmKind == DistributionManager.LONER_DM_TYPE) {
+    if (vmKind == ClusterDistributionManager.LONER_DM_TYPE) {
       this.uniqueTag = DataSerializer.readString(in);
     } else {
       String str = DataSerializer.readString(in);
@@ -1053,7 +1052,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     int vmViewId = -1;
 
     String name = DataSerializer.readString(in);
-    if (vmKind == DistributionManager.LONER_DM_TYPE) {
+    if (vmKind == ClusterDistributionManager.LONER_DM_TYPE) {
       this.uniqueTag = DataSerializer.readString(in);
     } else {
       String str = DataSerializer.readString(in);
@@ -1099,7 +1098,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     int vmKind = in.readUnsignedByte();
     int vmViewId = -1;
 
-    if (vmKind == DistributionManager.LONER_DM_TYPE) {
+    if (vmKind == ClusterDistributionManager.LONER_DM_TYPE) {
       this.uniqueTag = DataSerializer.readString(in);
     } else {
       String str = DataSerializer.readString(in);
@@ -1137,7 +1136,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
     byte vmKind = netMbr.getVmKind();
     out.writeByte(vmKind);
 
-    if (vmKind == DistributionManager.LONER_DM_TYPE) {
+    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);
@@ -1164,7 +1163,7 @@ public class InternalDistributedMember implements DistributedMember, Externaliza
    * information to help form a unique ID
    */
   public void setPort(int p) {
-    assert netMbr.getVmKind() == DistributionManager.LONER_DM_TYPE;
+    assert netMbr.getVmKind() == ClusterDistributionManager.LONER_DM_TYPE;
     this.netMbr.setPort(p);
     cachedToString = null;
   }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalRole.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalRole.java
index cff6908..00e8de1 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalRole.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/InternalRole.java
@@ -19,7 +19,7 @@ import java.util.*;
 import org.apache.commons.lang.StringUtils;
 
 import org.apache.geode.distributed.Role;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 
@@ -143,7 +143,7 @@ public class InternalRole implements Role {
           LocalizedStrings.InternalRole_ISPRESENT_REQUIRES_A_CONNECTION_TO_THE_DISTRIBUTED_SYSTEM
               .toLocalizedString());
     }
-    DM dm = sys.getDistributionManager();
+    DistributionManager dm = sys.getDistributionManager();
     return dm.isRolePresent(this);
   }
 
@@ -154,7 +154,7 @@ public class InternalRole implements Role {
           LocalizedStrings.InternalRole_GETCOUNT_REQUIRES_A_CONNECTION_TO_THE_DISTRIBUTED_SYSTEM
               .toLocalizedString());
     }
-    DM dm = sys.getDistributionManager();
+    DistributionManager dm = sys.getDistributionManager();
     return dm.getRoleCount(this);
   }
 
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
index e6d5053..b44b229 100755
--- 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
@@ -19,7 +19,7 @@ import java.util.List;
 import java.util.StringTokenizer;
 
 import org.apache.geode.distributed.DurableClientAttributes;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 
 /**
  * The attributes of a distributed member. This is largely deprecated as GMSMember holds all of this
@@ -32,7 +32,7 @@ public class MemberAttributes {
   public static final MemberAttributes INVALID =
       new MemberAttributes(-1, -1, -1, -1, null, null, null);
   public static final MemberAttributes DEFAULT =
-      new MemberAttributes(-1, -1, DistributionManager.NORMAL_DM_TYPE, -1, null, null, null);
+      new MemberAttributes(-1, -1, ClusterDistributionManager.NORMAL_DM_TYPE, -1, null, null, null);
 
   private int dcPort;
   private int vmPid;
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetView.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetView.java
index 17451a1..c9ed332 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetView.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/NetView.java
@@ -34,7 +34,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Version;
@@ -297,7 +297,7 @@ public class NetView implements DataSerializableFixedID {
 
   public InternalDistributedMember getLeadMember() {
     for (InternalDistributedMember mbr : this.members) {
-      if (mbr.getVmKind() == DistributionManager.NORMAL_DM_TYPE) {
+      if (mbr.getVmKind() == ClusterDistributionManager.NORMAL_DM_TYPE) {
         return mbr;
       }
     }
@@ -417,16 +417,16 @@ public class NetView implements DataSerializableFixedID {
     for (InternalDistributedMember mbr : this.members) {
       result += mbr.getNetMember().getMemberWeight();
       switch (mbr.getVmKind()) {
-        case DistributionManager.NORMAL_DM_TYPE:
+        case ClusterDistributionManager.NORMAL_DM_TYPE:
           result += 10;
           if (lead != null && mbr.equals(lead)) {
             result += 5;
           }
           break;
-        case DistributionManager.LOCATOR_DM_TYPE:
+        case ClusterDistributionManager.LOCATOR_DM_TYPE:
           result += 3;
           break;
-        case DistributionManager.ADMIN_ONLY_DM_TYPE:
+        case ClusterDistributionManager.ADMIN_ONLY_DM_TYPE:
           break;
         default:
           throw new IllegalStateException("Unknown member type: " + mbr.getVmKind());
@@ -448,16 +448,16 @@ public class NetView implements DataSerializableFixedID {
       }
       result += mbr.getNetMember().getMemberWeight();
       switch (mbr.getVmKind()) {
-        case DistributionManager.NORMAL_DM_TYPE:
+        case ClusterDistributionManager.NORMAL_DM_TYPE:
           result += 10;
           if (lead != null && mbr.equals(lead)) {
             result += 5;
           }
           break;
-        case DistributionManager.LOCATOR_DM_TYPE:
+        case ClusterDistributionManager.LOCATOR_DM_TYPE:
           result += 3;
           break;
-        case DistributionManager.ADMIN_ONLY_DM_TYPE:
+        case ClusterDistributionManager.ADMIN_ONLY_DM_TYPE:
           break;
         default:
           throw new IllegalStateException("Unknown member type: " + mbr.getVmKind());
@@ -473,7 +473,7 @@ public class NetView implements DataSerializableFixedID {
   public Set<InternalDistributedMember> getActualCrashedMembers(NetView oldView) {
     Set<InternalDistributedMember> result = new HashSet<>(this.crashedMembers.size());
     result.addAll(this.crashedMembers.stream()
-        .filter(mbr -> (mbr.getVmKind() != DistributionManager.ADMIN_ONLY_DM_TYPE))
+        .filter(mbr -> (mbr.getVmKind() != ClusterDistributionManager.ADMIN_ONLY_DM_TYPE))
         .filter(mbr -> oldView == null || oldView.contains(mbr)).collect(Collectors.toList()));
     return result;
   }
@@ -489,17 +489,17 @@ public class NetView implements DataSerializableFixedID {
       }
       int mbrWeight = mbr.getNetMember().getMemberWeight();
       switch (mbr.getVmKind()) {
-        case DistributionManager.NORMAL_DM_TYPE:
+        case ClusterDistributionManager.NORMAL_DM_TYPE:
           if (lead != null && mbr.equals(lead)) {
             mbrWeight += 15;
           } else {
             mbrWeight += 10;
           }
           break;
-        case DistributionManager.LOCATOR_DM_TYPE:
+        case ClusterDistributionManager.LOCATOR_DM_TYPE:
           mbrWeight += 3;
           break;
-        case DistributionManager.ADMIN_ONLY_DM_TYPE:
+        case ClusterDistributionManager.ADMIN_ONLY_DM_TYPE:
           break;
         default:
           throw new IllegalStateException("Unknown member type: " + mbr.getVmKind());
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 781ca63..0ede03e 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
@@ -21,10 +21,9 @@ import java.util.ArrayList;
 import java.util.Collection;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.DataSerializableFixedID;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Version;
 
@@ -134,7 +133,7 @@ public class FindCoordinatorRequest extends HighPriorityDistributionMessage
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message should not be executed");
   }
 
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 550d555..ec4b101 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,7 +22,7 @@ import java.util.HashSet;
 import java.util.Set;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.NetView;
@@ -178,7 +178,7 @@ public class FindCoordinatorResponse extends HighPriorityDistributionMessage
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message should not be executed");
   }
 
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 58ee027..cecc4a9 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
@@ -41,7 +41,7 @@ import org.apache.geode.InternalGemFireException;
 import org.apache.geode.cache.GemFireCache;
 import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.ClusterConfigurationService;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.LocatorStats;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
@@ -450,7 +450,7 @@ public class GMSLocator implements Locator, NetLocator {
       // GEODE-3052 - 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 (InternalDistributedMember member : members) {
-        if (member.getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
+        if (member.getVmKind() == ClusterDistributionManager.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 f90329c..3378000 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
@@ -56,8 +56,8 @@ import org.apache.geode.SystemConnectException;
 import org.apache.geode.distributed.DistributedMember;
 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.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.NetMember;
@@ -1608,7 +1608,7 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
       boolean preferred = false;
       if (services.getLocator() != null || Locator.hasLocator()
           || !services.getConfig().getDistributionConfig().getStartLocator().isEmpty()
-          || localAddress.getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
+          || localAddress.getVmKind() == ClusterDistributionManager.LOCATOR_DM_TYPE) {
         logger
             .info("This member is hosting a locator will be preferred as a membership coordinator");
         preferred = true;
@@ -2484,8 +2484,9 @@ public class GMSJoinLeave implements JoinLeave, MessageHandler {
             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() != DistributionManager.LOCATOR_DM_TYPE && conflictingView
-                .getCreator().getVmKind() == DistributionManager.LOCATOR_DM_TYPE) {
+            if (localAddress.getVmKind() != ClusterDistributionManager.LOCATOR_DM_TYPE
+                && conflictingView.getCreator()
+                    .getVmKind() == ClusterDistributionManager.LOCATOR_DM_TYPE) {
               logger.info("View preparation interrupted - a locator is taking over as "
                   + "membership coordinator in this view: {}", conflictingView);
               abandonedViews++;
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 b64917d..c112c8f 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
@@ -19,7 +19,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.Version;
@@ -43,7 +43,7 @@ public class FinalCheckPassedMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HeartbeatMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HeartbeatMessage.java
index 2ca9eca..21bbd08 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HeartbeatMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/HeartbeatMessage.java
@@ -18,7 +18,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.internal.Version;
 
@@ -46,7 +46,7 @@ public class HeartbeatMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 
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 8a6234d..16218ec 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
@@ -19,7 +19,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.Version;
@@ -53,7 +53,7 @@ public class HeartbeatRequestMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/InstallViewMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/InstallViewMessage.java
index c785a02..7dca4be 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/InstallViewMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/messages/InstallViewMessage.java
@@ -17,15 +17,11 @@ package org.apache.geode.distributed.internal.membership.gms.messages;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.NetView;
-import org.apache.geode.internal.InternalDataSerializer;
 
 public class InstallViewMessage extends HighPriorityDistributionMessage {
   enum messageType {
@@ -81,7 +77,7 @@ public class InstallViewMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 
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 caf15dd..c8e88fc 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,7 +19,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.Version;
@@ -54,7 +54,7 @@ public class JoinRequestMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 
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 4e0bcc8..dd718ab 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
@@ -17,17 +17,13 @@ package org.apache.geode.distributed.internal.membership.gms.messages;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.NetView;
-import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.Version;
 
 // TODO this class has been made unintelligible with different combinations of response values.
@@ -98,7 +94,7 @@ public class JoinResponseMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("JoinResponse is not intended to be executed");
   }
 
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 8ff0d97..ae4f6ce 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
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.Collection;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.Version;
@@ -55,7 +55,7 @@ public class LeaveRequestMessage extends HighPriorityDistributionMessage impleme
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 
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 8dd04b9..64edf73 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
@@ -16,7 +16,7 @@ package org.apache.geode.distributed.internal.membership.gms.messages;
 
 import java.util.Collection;
 
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 
@@ -34,7 +34,7 @@ public class NetworkPartitionMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message is not intended to be executed");
   }
 
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 8d08d5c..fab8d8e 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
@@ -17,11 +17,10 @@ package org.apache.geode.distributed.internal.membership.gms.messages;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.Version;
@@ -57,7 +56,7 @@ public class RemoveMemberMessage extends HighPriorityDistributionMessage impleme
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 
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 ac1ac15..1782a3f 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
@@ -21,7 +21,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.Version;
@@ -46,7 +46,7 @@ public class SuspectMembersMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 
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 12dd334..63266ba 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
@@ -19,7 +19,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.NetView;
@@ -72,7 +72,7 @@ public class ViewAckMessage extends HighPriorityDistributionMessage {
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     throw new IllegalStateException("this message is not intended to execute in a thread pool");
   }
 
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 46142c4..f524f68 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
@@ -72,9 +72,9 @@ import org.apache.geode.SystemConnectException;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.DurableClientAttributes;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.DistributionStats;
 import org.apache.geode.distributed.internal.HighPriorityDistributionMessage;
@@ -505,9 +505,9 @@ public class JGroupsMessenger implements Messenger {
     myChannel.down(new Event(Event.SET_LOCAL_ADDRESS, this.jgAddress));
 
     DistributionConfig config = services.getConfig().getDistributionConfig();
-    boolean isLocator =
-        (services.getConfig().getTransport().getVmKind() == DistributionManager.LOCATOR_DM_TYPE)
-            || !services.getConfig().getDistributionConfig().getStartLocator().isEmpty();
+    boolean isLocator = (services.getConfig().getTransport()
+        .getVmKind() == ClusterDistributionManager.LOCATOR_DM_TYPE)
+        || !services.getConfig().getDistributionConfig().getStartLocator().isEmpty();
 
     // establish the DistributedSystem's address
     DurableClientAttributes dca = null;
@@ -950,7 +950,7 @@ public class JGroupsMessenger implements Messenger {
     // which is fairly rare
     msg.setFlag(Flag.DONT_BUNDLE);
 
-    if (gfmsg.getProcessorType() == DistributionManager.HIGH_PRIORITY_EXECUTOR
+    if (gfmsg.getProcessorType() == ClusterDistributionManager.HIGH_PRIORITY_EXECUTOR
         || gfmsg instanceof HighPriorityDistributionMessage || AlertAppender.isThreadAlerting()) {
       msg.setFlag(Flag.OOB);
       msg.setFlag(Flag.NO_FC);
@@ -1289,7 +1289,7 @@ public class JGroupsMessenger implements Messenger {
         // multicast to them, avoiding deserialization cost and classpath
         // problems
         if ((services.getConfig().getTransport()
-            .getVmKind() == DistributionManager.ADMIN_ONLY_DM_TYPE)
+            .getVmKind() == ClusterDistributionManager.ADMIN_ONLY_DM_TYPE)
             && (msg instanceof DistributedCacheOperation.CacheOperationMessage)) {
           return;
         }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
index a2fc407..ea747c3 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/GMSMembershipManager.java
@@ -52,10 +52,10 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.DistributedSystemDisconnectedException;
 import org.apache.geode.distributed.Locator;
 import org.apache.geode.distributed.internal.AdminMessageType;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.DistributionException;
-import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.InternalLocator;
@@ -408,7 +408,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
       handleOrDeferMessage(msg);
     }
 
-    public DistributionManager getDM() {
+    public ClusterDistributionManager getDM() {
       return upCall.getDM();
     }
 
@@ -1956,7 +1956,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
   public boolean shutdownInProgress() {
     // Impossible condition (bug36329): make sure that we check DM's
     // view of shutdown here
-    DistributionManager dm = listener.getDM();
+    ClusterDistributionManager dm = listener.getDM();
     return shutdownInProgress || (dm != null && dm.shutdownInProgress());
   }
 
@@ -2521,7 +2521,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
       }
     }
 
-    protected void process(DistributionManager dm) {
+    protected void process(ClusterDistributionManager dm) {
       // not used
     }
 
@@ -2530,7 +2530,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
     }
 
     public int getProcessorType() {
-      return DistributionManager.SERIAL_EXECUTOR;
+      return ClusterDistributionManager.SERIAL_EXECUTOR;
     }
   }
 
@@ -2633,7 +2633,7 @@ public class GMSMembershipManager implements MembershipManager, Manager {
 
   @Override
   public boolean isShutdownStarted() {
-    DistributionManager dm = listener.getDM();
+    ClusterDistributionManager dm = listener.getDM();
     return shutdownInProgress || (dm != null && dm.isShutdownStarted());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/LocalViewMessage.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/LocalViewMessage.java
index 8b4f0f5..8634411 100755
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/LocalViewMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/membership/gms/mgr/LocalViewMessage.java
@@ -18,7 +18,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.SerialDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.membership.NetView;
@@ -48,12 +48,12 @@ public class LocalViewMessage extends SerialDistributionMessage {
 
   @Override
   public int getProcessorType() {
-    return DistributionManager.VIEW_EXECUTOR;
+    return ClusterDistributionManager.VIEW_EXECUTOR;
   }
 
 
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     // dm.getLogger().info("view message processed", new Exception());
     manager.processView(viewId, view);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/streaming/StreamingOperation.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/streaming/StreamingOperation.java
index b33273a..163eaa7 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/streaming/StreamingOperation.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/streaming/StreamingOperation.java
@@ -33,14 +33,13 @@ import org.apache.geode.GemFireRethrowable;
 import org.apache.geode.InternalGemFireError;
 import org.apache.geode.InternalGemFireException;
 import org.apache.geode.SystemFailure;
-import org.apache.geode.cache.query.Struct;
 import org.apache.geode.cache.query.internal.DefaultQuery;
 import org.apache.geode.cache.query.internal.PRQueryTraceInfo;
 import org.apache.geode.cache.query.internal.QueryMonitor;
 import org.apache.geode.cache.query.internal.StructImpl;
 import org.apache.geode.cache.query.internal.types.StructTypeImpl;
 import org.apache.geode.cache.query.types.ObjectType;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
@@ -278,7 +277,7 @@ public abstract class StreamingOperation {
     }
 
     @Override
-    protected void process(final DistributionManager dm) {
+    protected void process(final ClusterDistributionManager dm) {
       Throwable thr = null;
       ReplyException rex = null;
       Object nextObject = null;
@@ -381,13 +380,13 @@ public abstract class StreamingOperation {
     // StreamingReplyMessage.send(getSender(), this.processorId, null, dm, null, 0, 0, true);
     // }
 
-    protected void replyWithData(DistributionManager dm, HeapDataOutputStream outStream,
+    protected void replyWithData(ClusterDistributionManager dm, HeapDataOutputStream outStream,
         int numObjects, int msgNum, boolean lastMsg) {
       StreamingReplyMessage.send(getSender(), this.processorId, null, dm, outStream, numObjects,
           msgNum, lastMsg);
     }
 
-    protected void replyWithException(DistributionManager dm, ReplyException rex) {
+    protected void replyWithException(ClusterDistributionManager dm, ReplyException rex) {
       StreamingReplyMessage.send(getSender(), this.processorId, rex, dm, null, 0, 0, true);
     }
 
@@ -444,14 +443,14 @@ public abstract class StreamingOperation {
      * @param lastMsg if this is the last message in this series
      */
     public static void send(InternalDistributedMember recipient, int processorId,
-        ReplyException exception, DM dm, HeapDataOutputStream chunkStream, int numObjects,
-        int msgNum, boolean lastMsg) {
+        ReplyException exception, DistributionManager dm, HeapDataOutputStream chunkStream,
+        int numObjects, int msgNum, boolean lastMsg) {
       send(recipient, processorId, exception, dm, chunkStream, numObjects, msgNum, lastMsg, false);
     }
 
     public static void send(InternalDistributedMember recipient, int processorId,
-        ReplyException exception, DM dm, HeapDataOutputStream chunkStream, int numObjects,
-        int msgNum, boolean lastMsg, boolean pdxReadSerialized) {
+        ReplyException exception, DistributionManager dm, HeapDataOutputStream chunkStream,
+        int numObjects, int msgNum, boolean lastMsg, boolean pdxReadSerialized) {
       StreamingReplyMessage m = new StreamingReplyMessage();
       m.processorId = processorId;
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java b/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
index 313cd27..c00430e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
@@ -82,10 +82,10 @@ import org.apache.geode.SystemFailure;
 import org.apache.geode.ToDataException;
 import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.execute.Function;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributedSystemService;
 import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.LonerDistributionManager;
 import org.apache.geode.distributed.internal.SerialDistributionMessage;
@@ -3374,7 +3374,7 @@ public abstract class InternalDataSerializer extends DataSerializer implements D
     }
 
     @Override
-    protected void process(DistributionManager dm) {
+    protected void process(ClusterDistributionManager dm) {
       if (CacheClientNotifier.getInstance() != null) {
         // This is a server so we need to send the dataserializer to clients
         // right away. For that we need to load the class as the constructor of
diff --git a/geode-core/src/main/java/org/apache/geode/internal/InternalInstantiator.java b/geode-core/src/main/java/org/apache/geode/internal/InternalInstantiator.java
index 6e48681..2731ed5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/InternalInstantiator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/InternalInstantiator.java
@@ -30,7 +30,7 @@ import org.apache.logging.log4j.Logger;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.Instantiator;
 import org.apache.geode.cache.Cache;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.SerialDistributionMessage;
 import org.apache.geode.i18n.StringId;
@@ -863,7 +863,7 @@ public class InternalInstantiator {
     }
 
     @Override
-    protected void process(DistributionManager dm) {
+    protected void process(ClusterDistributionManager dm) {
       if (this.fromDataProblems != null) {
         if (logger.isDebugEnabled()) {
           logger.debug(this.fromDataProblems);
@@ -979,7 +979,7 @@ public class InternalInstantiator {
     }
 
     @Override
-    protected void process(DistributionManager dm) {
+    protected void process(ClusterDistributionManager dm) {
       if (fromDataProblems != null) {
         if (logger.isDebugEnabled()) {
           logger.debug(fromDataProblems);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/ClientMembershipMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/ClientMembershipMessage.java
index 98ff92c..76eed16 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/ClientMembershipMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/ClientMembershipMessage.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 
 import org.apache.geode.DataSerializer;
 import org.apache.geode.admin.internal.AdminDistributedSystemImpl;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.PooledDistributionMessage;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 
@@ -61,10 +61,10 @@ public class ClientMembershipMessage extends PooledDistributionMessage {
 
   /**
    *
-   * @see org.apache.geode.distributed.internal.DistributionMessage#process(org.apache.geode.distributed.internal.DistributionManager)
+   * @see org.apache.geode.distributed.internal.DistributionMessage#process(ClusterDistributionManager)
    */
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     AdminDistributedSystemImpl adminDs = AdminDistributedSystemImpl.getConnectedInstance();
 
     /*
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/GfManagerAgent.java b/geode-core/src/main/java/org/apache/geode/internal/admin/GfManagerAgent.java
index c129583..91f4855 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/GfManagerAgent.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/GfManagerAgent.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.internal.admin;
 
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 
 /**
@@ -86,7 +86,7 @@ public interface GfManagerAgent {
    *
    * @since GemFire 3.5
    */
-  public DM getDM();
+  public DistributionManager getDM();
 
   /**
    * Sets the alert level for this manager agent.
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/StatAlertsManager.java b/geode-core/src/main/java/org/apache/geode/internal/admin/StatAlertsManager.java
index f1a1164..f2e804a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/StatAlertsManager.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/StatAlertsManager.java
@@ -30,7 +30,7 @@ import org.apache.geode.StatisticDescriptor;
 import org.apache.geode.Statistics;
 import org.apache.geode.StatisticsType;
 import org.apache.geode.admin.jmx.internal.StatAlertsAggregator;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.SystemTimer;
 import org.apache.geode.internal.SystemTimer.SystemTimerTask;
@@ -88,9 +88,9 @@ public class StatAlertsManager {
   /**
    * Provides life cycle support
    */
-  protected final DistributionManager dm;
+  protected final ClusterDistributionManager dm;
 
-  private StatAlertsManager(DistributionManager dm) {
+  private StatAlertsManager(ClusterDistributionManager dm) {
     this.dm = dm;
     logger.info(
         LocalizedMessage.create(LocalizedStrings.StatAlertsManager_STATALERTSMANAGER_CREATED));
@@ -99,7 +99,7 @@ public class StatAlertsManager {
   /**
    * @return singleton instance of StatAlertsManager
    */
-  public static synchronized StatAlertsManager getInstance(DistributionManager dm) {
+  public static synchronized StatAlertsManager getInstance(ClusterDistributionManager dm) {
     // As per current implementation set up request will be send only once ,
     // when member joined to Admin distributed system
     // we don't need to care about race condition
@@ -258,7 +258,7 @@ public class StatAlertsManager {
    * Convert {@link StatAlertDefinition }(Created by client like GFMon2.0) with
    * {@link DummyStatisticInfoImpl} to StatAlertDefinition with {@link StatisticInfoImpl}
    */
-  private StatAlertDefinition[] createMemberStatAlertDefinition(DistributionManager dm,
+  private StatAlertDefinition[] createMemberStatAlertDefinition(ClusterDistributionManager dm,
       StatAlertDefinition[] defns) {
     dm.getCancelCriterion().checkCancelInProgress(null);
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddHealthListenerRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddHealthListenerRequest.java
index 4ffe064..795b00c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddHealthListenerRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddHealthListenerRequest.java
@@ -57,7 +57,7 @@ public class AddHealthListenerRequest extends AdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DM dm) {
+  protected AdminResponse createResponse(DistributionManager dm) {
     return AddHealthListenerResponse.create(dm, this.getSender(), this.cfg);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddHealthListenerResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddHealthListenerResponse.java
index 50aed79..3fda3f7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddHealthListenerResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddHealthListenerResponse.java
@@ -35,8 +35,8 @@ public class AddHealthListenerResponse extends AdminResponse {
    * Returns a <code>AddHealthListenerResponse</code> that will be returned to the specified
    * recipient.
    */
-  public static AddHealthListenerResponse create(DM dm, InternalDistributedMember recipient,
-      GemFireHealthConfig cfg) {
+  public static AddHealthListenerResponse create(DistributionManager dm,
+      InternalDistributedMember recipient, GemFireHealthConfig cfg) {
     AddHealthListenerResponse m = new AddHealthListenerResponse();
     m.setRecipient(recipient);
     dm.createHealthMonitor(recipient, cfg);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddStatListenerRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddStatListenerRequest.java
index 37111dc..36e9deb 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddStatListenerRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddStatListenerRequest.java
@@ -51,7 +51,7 @@ public class AddStatListenerRequest extends AdminRequest {
    * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DM dm) {
+  protected AdminResponse createResponse(DistributionManager dm) {
     return AddStatListenerResponse.create(dm, this.getSender(), this.resourceId, this.statName);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddStatListenerResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddStatListenerResponse.java
index 19b430e..b5f52a9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddStatListenerResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AddStatListenerResponse.java
@@ -34,8 +34,8 @@ public class AddStatListenerResponse extends AdminResponse {
    * Returns a <code>AddStatListenerResponse</code> that will be returned to the specified
    * recipient. The message will contains a copy of the local manager's system config.
    */
-  public static AddStatListenerResponse create(DM dm, InternalDistributedMember recipient,
-      long resourceId, String statName) {
+  public static AddStatListenerResponse create(DistributionManager dm,
+      InternalDistributedMember recipient, long resourceId, String statName) {
     AddStatListenerResponse m = new AddStatListenerResponse();
     m.setRecipient(recipient);
     GemFireStatSampler sampler = null;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleDisconnectMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleDisconnectMessage.java
index 0fbcb3c..1baf38c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleDisconnectMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleDisconnectMessage.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.PooledDistributionMessage;
 import org.apache.geode.internal.i18n.LocalizedStrings;
@@ -77,7 +77,7 @@ public class AdminConsoleDisconnectMessage extends PooledDistributionMessage {
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     InternalDistributedSystem sys = dm.getSystem();
     // DistributionConfig config = sys.getConfig();
     if (alertListenerExpected) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleMessage.java
index d8c9665..d4e9532 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminConsoleMessage.java
@@ -20,15 +20,10 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.PooledDistributionMessage;
 import org.apache.geode.internal.admin.Alert;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.ManagerLogWriter;
 import org.apache.geode.internal.logging.log4j.AlertAppender;
-import org.apache.geode.internal.logging.log4j.LogWriterLogger;
 
 /**
  * A message that is sent to a particular distribution manager to let it know that the sender is an
@@ -49,7 +44,7 @@ public class AdminConsoleMessage extends PooledDistributionMessage {
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     if (this.level != Alert.OFF) {
       AlertAppender.getInstance().addAlertListener(this.getSender(), this.level);
     }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminMultipleReplyProcessor.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminMultipleReplyProcessor.java
index 3a5cff9..a0fa2c4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminMultipleReplyProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminMultipleReplyProcessor.java
@@ -16,7 +16,7 @@ package org.apache.geode.internal.admin.remote;
 
 import java.util.Collection;
 
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.ReplyException;
 import org.apache.geode.distributed.internal.ReplyProcessor21;
@@ -28,7 +28,7 @@ import org.apache.geode.distributed.internal.ReplyProcessor21;
  */
 public class AdminMultipleReplyProcessor extends ReplyProcessor21 {
 
-  public AdminMultipleReplyProcessor(DM dm, Collection initMembers) {
+  public AdminMultipleReplyProcessor(DistributionManager dm, Collection initMembers) {
     super(dm, initMembers);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminRequest.java
index 58c078b..d83d6b7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminRequest.java
@@ -24,7 +24,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
 import org.apache.geode.admin.RuntimeAdminException;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.PooledDistributionMessage;
 import org.apache.geode.distributed.internal.ReplyException;
@@ -71,7 +71,7 @@ public abstract class AdminRequest extends PooledDistributionMessage {
   /**
    * Sends this request, waits for the AdminReponse, and returns it
    */
-  public AdminResponse sendAndWait(DistributionManager dm) {
+  public AdminResponse sendAndWait(ClusterDistributionManager dm) {
     InternalDistributedMember recipient = this.getRecipient();
     if (dm.getId().equals(recipient)) {
       // We're sending this message to ourselves, we won't need a
@@ -128,7 +128,7 @@ public abstract class AdminRequest extends PooledDistributionMessage {
    * outgoing queue.
    */
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     AdminResponse response = null;
     InspectionClasspathManager cpMgr = InspectionClasspathManager.getInstance();
     try {
@@ -151,7 +151,7 @@ public abstract class AdminRequest extends PooledDistributionMessage {
   /**
    * Must return a proper response to this request.
    */
-  protected abstract AdminResponse createResponse(DM dm);
+  protected abstract AdminResponse createResponse(DistributionManager dm);
 
   @Override
   public void toData(DataOutput out) throws IOException {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminResponse.java
index adb72b1..750935b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminResponse.java
@@ -51,7 +51,7 @@ public abstract class AdminResponse extends HighPriorityDistributionMessage
    * This method is invoked on the side that sent the original AdminRequest.
    */
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     AdminWaiters.sendResponse(this);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminWaiters.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminWaiters.java
index b964a4e..d19885a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminWaiters.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminWaiters.java
@@ -22,7 +22,7 @@ import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.admin.OperationCancelledException;
 import org.apache.geode.admin.RuntimeAdminException;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.ReplyProcessor21;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.i18n.LocalizedStrings;
@@ -53,7 +53,7 @@ public class AdminWaiters {
    * @throws RuntimeAdminException if this method is interrupted, times out, cancelled
    *         ({@link #cancelWaiters}), or failed with an exception on the server side.
    */
-  public static AdminResponse sendAndWait(AdminRequest msg, DistributionManager dm) {
+  public static AdminResponse sendAndWait(AdminRequest msg, ClusterDistributionManager dm) {
 
     // Prior to GemFire 4.0 admin messages were only sent to other
     // VMs; it was impossible for an admin message to be destined for
@@ -166,7 +166,7 @@ public class AdminWaiters {
     // that depart.
   }
 
-  public static void cancelRequest(int msgId, DistributionManager dm) {
+  public static void cancelRequest(int msgId, ClusterDistributionManager dm) {
     AdminReplyProcessor processor = (AdminReplyProcessor) ReplyProcessor21.getProcessor(msgId);
     if (processor != null) {
       InternalDistributedMember recipient = processor.getResponder();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertLevelChangeMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertLevelChangeMessage.java
index 442b244..3117654 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertLevelChangeMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertLevelChangeMessage.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.admin.AlertLevel;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.SerialDistributionMessage;
 import org.apache.geode.internal.admin.Alert;
 import org.apache.geode.internal.i18n.LocalizedStrings;
@@ -58,7 +58,7 @@ public class AlertLevelChangeMessage extends SerialDistributionMessage {
   ////////////////////// Instance Methods //////////////////////
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     AlertAppender.getInstance().removeAlertListener(this.getSender());
 
     if (this.newLevel != Alert.OFF) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertListenerMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertListenerMessage.java
index 48b3a37..ee5c68b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertListenerMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertListenerMessage.java
@@ -66,7 +66,7 @@ public class AlertListenerMessage extends PooledDistributionMessage implements A
   }
 
   @Override
-  public void process(DistributionManager dm) {
+  public void process(ClusterDistributionManager dm) {
     RemoteGfManagerAgent agent = dm.getAgent();
     if (agent != null) {
       RemoteGemFireVM mgr = agent.getMemberById(this.getSender());
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertsNotificationMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertsNotificationMessage.java
index 4a944dc..4d78e0b 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertsNotificationMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AlertsNotificationMessage.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.admin.internal.AdminDistributedSystemImpl;
 import org.apache.geode.admin.jmx.internal.StatAlertsAggregator;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
 import org.apache.geode.distributed.internal.PooledDistributionMessage;
 import org.apache.geode.internal.admin.StatAlert;
 
@@ -57,7 +57,7 @@ public class AlertsNotificationMessage extends PooledDistributionMessage {
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     // TODO add code to invoke process notification of agrregator
     // TODO: need to check whether it's a valid implimentation
     AdminDistributedSystemImpl ds = AdminDistributedSystemImpl.getConnectedInstance();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AppCacheSnapshotMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AppCacheSnapshotMessage.java
index c8770ce..37fafc4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AppCacheSnapshotMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AppCacheSnapshotMessage.java
@@ -36,7 +36,7 @@ public class AppCacheSnapshotMessage extends RegionAdminMessage {
   }
 
   @Override
-  protected void process(DistributionManager dm) {
+  protected void process(ClusterDistributionManager dm) {
     Region r = getRegion(dm.getSystem());
     if (r != null) {
       try {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/BridgeServerRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/BridgeServerRequest.java
index 853d5bf..04c4233 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/BridgeServerRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/BridgeServerRequest.java
@@ -17,7 +17,7 @@ package org.apache.geode.internal.admin.remote;
 import java.io.*;
 
 import org.apache.geode.DataSerializer;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.internal.admin.CacheInfo;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 
@@ -132,7 +132,7 @@ public class BridgeServerRequest extends AdminRequest {
    * Creates a <Code>BridgeServerResponse</code> to this request
    */
   @Override
-  protected AdminResponse createResponse(DM dm) {
+  protected AdminResponse createResponse(DistributionManager dm) {
     return BridgeServerResponse.create(dm, this);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/BridgeServerResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/BridgeServerResponse.java
index 6e9b32d..ddc102d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/BridgeServerResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/BridgeServerResponse.java
@@ -22,7 +22,7 @@ import org.apache.geode.CancelException;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.server.CacheServer;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.cache.CacheServerImpl;
 import org.apache.geode.internal.cache.InternalCache;
@@ -44,7 +44,7 @@ public class BridgeServerResponse extends AdminResponse {
   /**
    * Creates a {@code BridgeServerResponse} in response to the given request.
    */
-  static BridgeServerResponse create(DM dm, BridgeServerRequest request) {
+  static BridgeServerResponse create(DistributionManager dm, BridgeServerRequest request) {
     BridgeServerResponse m = new BridgeServerResponse();
     m.setRecipient(request.getSender());
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheConfigRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheConfigRequest.java
index d833507..dba5810 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheConfigRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheConfigRequest.java
@@ -53,7 +53,7 @@ public class CacheConfigRequest extends AdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DM dm) {
+  protected AdminResponse createResponse(DistributionManager dm) {
     return CacheConfigResponse.create(dm, this.getSender(), this.cacheId, this.attributeCode,
         this.newValue);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheConfigResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheConfigResponse.java
index 098d3f1..0aae5ff 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheConfigResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheConfigResponse.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import org.apache.geode.CancelException;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.cache.InternalCache;
@@ -45,8 +45,8 @@ public class CacheConfigResponse extends AdminResponse {
   /**
    * Returns a {@code CacheConfigResponse} that will be returned to the specified recipient.
    */
-  public static CacheConfigResponse create(DM dm, InternalDistributedMember recipient, int cacheId,
-      byte attributeCode, int newValue) {
+  public static CacheConfigResponse create(DistributionManager dm,
+      InternalDistributedMember recipient, int cacheId, byte attributeCode, int newValue) {
     CacheConfigResponse m = new CacheConfigResponse();
     m.setRecipient(recipient);
     try {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheInfoRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheInfoRequest.java
index 2692d28..200e18e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheInfoRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheInfoRequest.java
@@ -46,7 +46,7 @@ public class CacheInfoRequest extends AdminRequest {
    * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DM dm) {
+  protected AdminResponse createResponse(DistributionManager dm) {
     return CacheInfoResponse.create(dm, this.getSender());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheInfoResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheInfoResponse.java
index bc79498..f6191c5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheInfoResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CacheInfoResponse.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import org.apache.geode.CancelException;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalCache;
 
@@ -37,7 +37,8 @@ public class CacheInfoResponse extends AdminResponse {
   /**
    * Returns a {@code CacheInfoResponse} that will be returned to the specified recipient.
    */
-  public static CacheInfoResponse create(DM dm, InternalDistributedMember recipient) {
+  public static CacheInfoResponse create(DistributionManager dm,
+      InternalDistributedMember recipient) {
     CacheInfoResponse m = new CacheInfoResponse();
... 13567 lines suppressed ...

-- 
To stop receiving notification emails like this one, please contact
"commits@geode.apache.org" <co...@geode.apache.org>.