You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2017/11/14 00:11:42 UTC

[geode] branch feature/GEODE-3940 updated (96e327a -> 7377e1b)

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

dschneider pushed a change to branch feature/GEODE-3940
in repository https://gitbox.apache.org/repos/asf/geode.git.


 discard 96e327a  Added FlushToDiskOperation
 discard cbebe49  refactored operation code for Prepare and Finish into a common BackupOperation abstract class. Implemented new classes for the FinishBackup operation.
 discard 566e44f  createRequest no longer takes a PrepareBackupFactory parameter
 discard d89c7ed  renamed BackupResultCollector to PrepareBackupResultCollector
 discard 7ee6ce4  PrepareBackupFactory unit test coverage is now 100%
 discard c788c2b  PrepareBackupOperation now has 100% coverage by unit test
 discard ede5368  BackupDistributedTest now passes
 discard c84330a  spotless
 discard f380c1d  WIP
    omit d55e6d6  GEODE-3940: prevent warning when processing message by sender
    omit 81ff81d  GEODE-3940: fix deadlock in backup messages
     new 7377e1b  GEODE-3940: fix deadlock in backup messages

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (96e327a)
            \
             N -- N -- N   refs/heads/feature/GEODE-3940 (7377e1b)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 1 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:

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

[geode] 01/01: GEODE-3940: fix deadlock in backup messages

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

dschneider pushed a commit to branch feature/GEODE-3940
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 7377e1bf98f8b317bdda3572877acc95a07e63cb
Author: Lynn Gallinat <lg...@pivotal.io>
AuthorDate: Fri Nov 3 17:09:42 2017 -0700

    GEODE-3940: fix deadlock in backup messages
    
    When backup sends FlushToDisk, PrepareBackup, and FinishBackup,
    it will no longer wait for all other members to respond before
    doing the same work on its own member.
    In addition to preventing a distributed deadlock, this will
    also improve how long it takes for a backup to complete.
    As part of this work the backup classes and tests have been
    moved into their own backup package.
---
 .../admin/internal/AdminDistributedSystemImpl.java |   2 +-
 .../geode/admin/internal/FinishBackupRequest.java  | 174 ----
 .../geode/admin/internal/FinishBackupResponse.java |  76 --
 .../geode/admin/internal/FlushToDiskRequest.java   |  92 --
 .../geode/admin/internal/PrepareBackupRequest.java | 134 ---
 .../org/apache/geode/distributed/internal/DM.java  | 145 ++--
 .../internal/DistributionMessageObserver.java      |  16 +-
 .../internal/LonerDistributionManager.java         |  19 +
 .../org/apache/geode/internal/DSFIDFactory.java    |  19 +-
 .../geode/internal/DataSerializableFixedID.java    |   2 +-
 .../org/apache/geode/internal/SystemAdmin.java     |   2 +-
 .../admin/remote/AddHealthListenerRequest.java     |   2 +-
 .../admin/remote/AddHealthListenerResponse.java    |   4 +-
 .../admin/remote/AddStatListenerRequest.java       |   4 +-
 .../admin/remote/AddStatListenerResponse.java      |   4 +-
 .../admin/remote/AdminFailureResponse.java         |  36 +-
 .../admin/remote/AdminMultipleReplyProcessor.java  |  51 +-
 .../geode/internal/admin/remote/AdminRequest.java  |   5 +-
 .../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    |   4 +-
 .../internal/admin/remote/CacheInfoResponse.java   |   5 +-
 .../admin/remote/CancelStatListenerRequest.java    |   2 +-
 .../admin/remote/CancelStatListenerResponse.java   |   4 +-
 .../internal/admin/remote/CliLegacyMessage.java    |   3 +-
 .../internal/admin/remote/CompactRequest.java      |  13 +-
 .../admin/remote/DurableClientInfoRequest.java     |   4 +-
 .../admin/remote/DurableClientInfoResponse.java    |   6 +-
 .../admin/remote/FetchDistLockInfoRequest.java     |   4 +-
 .../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    |  29 +-
 .../internal/admin/remote/FetchStatsRequest.java   |   2 +-
 .../internal/admin/remote/FetchStatsResponse.java  |  20 +-
 .../internal/admin/remote/FetchSysCfgRequest.java  |   2 +-
 .../internal/admin/remote/FetchSysCfgResponse.java |   3 +-
 .../internal/admin/remote/LicenseInfoRequest.java  |   4 +-
 .../internal/admin/remote/LicenseInfoResponse.java |   9 +-
 .../admin/remote/MissingPersistentIDsRequest.java  |  13 +-
 .../admin/remote/ObjectDetailsRequest.java         |   4 +-
 .../admin/remote/ObjectDetailsResponse.java        |   3 +-
 .../internal/admin/remote/ObjectNamesRequest.java  |   4 +-
 .../internal/admin/remote/ObjectNamesResponse.java |   3 +-
 .../remote/PrepareRevokePersistentIDRequest.java   |   3 +-
 .../admin/remote/RefreshMemberSnapshotRequest.java |   2 +-
 .../remote/RefreshMemberSnapshotResponse.java      |   5 +-
 .../admin/remote/RegionAttributesRequest.java      |   4 +-
 .../admin/remote/RegionAttributesResponse.java     |   4 +-
 .../geode/internal/admin/remote/RegionRequest.java |   4 +-
 .../internal/admin/remote/RegionResponse.java      |   2 +-
 .../internal/admin/remote/RegionSizeRequest.java   |   4 +-
 .../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/RemoveHealthListenerRequest.java  |   2 +-
 .../admin/remote/RemoveHealthListenerResponse.java |   4 +-
 .../admin/remote/ResetHealthStatusRequest.java     |   4 +-
 .../admin/remote/ResetHealthStatusResponse.java    |   4 +-
 .../admin/remote/RevokePersistentIDRequest.java    |   3 +-
 .../internal/admin/remote/RootRegionRequest.java   |   2 +-
 .../internal/admin/remote/RootRegionResponse.java  |   5 +-
 .../internal/admin/remote/ShutdownAllRequest.java  |   4 +-
 .../internal/admin/remote/StoreSysCfgRequest.java  |   2 +-
 .../internal/admin/remote/StoreSysCfgResponse.java |   3 +-
 .../internal/admin/remote/SubRegionRequest.java    |   4 +-
 .../internal/admin/remote/SubRegionResponse.java   |   3 +-
 .../internal/admin/remote/TailLogRequest.java      |   2 +-
 .../internal/admin/remote/TailLogResponse.java     |   5 +-
 .../internal/admin/remote/VersionInfoRequest.java  |   4 +-
 .../internal/admin/remote/VersionInfoResponse.java |   3 +-
 .../geode/internal/cache/DirectoryHolder.java      |   2 +-
 .../geode/internal/cache/DiskStoreBackup.java      |   2 -
 .../geode/internal/cache/DiskStoreFactoryImpl.java |   1 +
 .../apache/geode/internal/cache/DiskStoreImpl.java |  10 +-
 .../geode/internal/cache/GemFireCacheImpl.java     |   3 +-
 .../apache/geode/internal/cache/InternalCache.java |   1 +
 .../org/apache/geode/internal/cache/Oplog.java     |  22 +-
 .../internal/cache/PartitionedRegionDataStore.java |   1 +
 .../cache/backup}/BackupDataStoreHelper.java       |  21 +-
 .../cache/backup}/BackupDataStoreResult.java       |   2 +-
 .../{persistence => backup}/BackupInspector.java   |   2 +-
 .../internal/cache/{ => backup}/BackupLock.java    |  45 +-
 .../internal/cache/{ => backup}/BackupManager.java |  27 +-
 .../internal/cache/backup/BackupOperation.java     |  85 ++
 .../cache/backup/BackupReplyProcessor.java         |  59 ++
 .../cache/backup/BackupResponse.java}              |  29 +-
 .../cache/backup/BackupResultCollector.java}       |  16 +-
 .../internal/cache/{ => backup}/BackupUtil.java    |   4 +-
 .../geode/internal/cache/backup/FinishBackup.java  |  47 +
 .../internal/cache/backup/FinishBackupFactory.java |  49 ++
 .../cache/backup/FinishBackupOperation.java        |  75 ++
 .../internal/cache/backup/FinishBackupRequest.java | 100 +++
 .../geode/internal/cache/backup/FlushToDisk.java}  |  23 +-
 .../cache/backup/FlushToDiskFactory.java}          |  35 +-
 .../cache/backup/FlushToDiskOperation.java         |  87 ++
 .../cache/backup/FlushToDiskProcessor.java}        |  31 +-
 .../internal/cache/backup/FlushToDiskRequest.java  |  57 ++
 .../cache/backup}/FlushToDiskResponse.java         |   2 +-
 .../cache/backup/PrepareBackup.java}               |  39 +-
 .../cache/backup/PrepareBackupFactory.java         |  45 +
 .../cache/backup/PrepareBackupOperation.java       |  64 ++
 .../cache/backup/PrepareBackupRequest.java         |  73 ++
 .../{persistence => backup}/RestoreScript.java     |   2 +-
 .../{persistence => backup}/ScriptGenerator.java   |   2 +-
 .../UnixBackupInspector.java                       |   2 +-
 .../UnixScriptGenerator.java                       |   5 +-
 .../WindowsBackupInspector.java                    |   2 +-
 .../WindowsScriptGenerator.java                    |   3 +-
 .../cache/persistence/PersistentMemberManager.java |   8 +-
 .../internal/cache/xmlcache/CacheCreation.java     |   2 +-
 .../org/apache/geode/management/BackupStatus.java  |   7 +-
 .../internal/beans/DistributedSystemBridge.java    |   4 +-
 .../internal/beans/MemberMBeanBridge.java          |   2 +-
 .../cli/commands/BackupDiskStoreCommand.java       |   2 +-
 .../internal/messages/CompactRequest.java          |  12 +-
 .../BackupDistributedTest.java}                    | 590 ++++++-------
 .../BackupInspectorIntegrationTest.java            |   2 +-
 .../BackupIntegrationTest.java}                    |  29 +-
 .../internal/cache/backup/BackupLockTest.java      | 105 +++
 .../BackupPrepareAndFinishMsgDUnitTest.java        |  20 +-
 .../cache/backup/BackupReplyProcessorTest.java     |  90 ++
 .../cache/backup/FinishBackupFactoryTest.java      |  87 ++
 .../cache/backup/FinishBackupOperationTest.java    | 236 +++++
 .../cache/backup/FinishBackupRequestTest.java      | 117 +++
 .../cache/backup/FlushToDiskFactoryTest.java       |  83 ++
 .../cache/backup/FlushToDiskOperationTest.java     | 121 +++
 .../cache/backup/FlushToDiskRequestTest.java       |  82 ++
 .../IncrementalBackupDistributedTest.java}         |  17 +-
 ...titionedBackupPrepareAndFinishMsgDUnitTest.java |   5 +-
 .../cache/backup/PrepareBackupFactoryTest.java     |  85 ++
 .../cache/backup/PrepareBackupOperationTest.java   | 232 +++++
 .../cache/backup/PrepareBackupRequestTest.java     |  98 +++
 ...eplicateBackupPrepareAndFinishMsgDUnitTest.java |   5 +-
 .../UnixScriptGeneratorTest.java                   |   2 +-
 .../WindowsScriptGeneratorTest.java                |   2 +-
 .../cache/partitioned/PersistPRKRFDUnitTest.java   |  36 +-
 ...sistentColocatedPartitionedRegionDUnitTest.java | 226 ++---
 .../PersistentPartitionedRegionDUnitTest.java      | 118 ++-
 .../PersistentPartitionedRegionTestBase.java       | 961 ++++++++-------------
 ...tPartitionedRegionWithTransactionDUnitTest.java |  28 +-
 .../PersistentRecoveryOrderDUnitTest.java          |   2 +-
 .../beans/DistributedSystemBridgeJUnitTest.java    |   8 +-
 .../pdx/ClientsWithVersioningRetryDUnitTest.java   |   4 +-
 .../BackupIntegrationTest.cache.xml}               |   0
 152 files changed, 3504 insertions(+), 2063 deletions(-)

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 303fda8..8e23a31 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
@@ -38,7 +38,7 @@ import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.Banner;
 import org.apache.geode.internal.admin.*;
 import org.apache.geode.internal.admin.remote.*;
-import org.apache.geode.internal.cache.BackupUtil;
+import org.apache.geode.internal.cache.backup.BackupUtil;
 import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.InternalLogWriter;
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java b/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java
deleted file mode 100644
index 12f5bb9..0000000
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupRequest.java
+++ /dev/null
@@ -1,174 +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.admin.internal;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.File;
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.CancelException;
-import org.apache.geode.DataSerializer;
-import org.apache.geode.cache.persistence.PersistentID;
-import org.apache.geode.distributed.DistributedMember;
-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.internal.admin.remote.AdminFailureResponse;
-import org.apache.geode.internal.admin.remote.AdminMultipleReplyProcessor;
-import org.apache.geode.internal.admin.remote.AdminResponse;
-import org.apache.geode.internal.admin.remote.CliLegacyMessage;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-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.log4j.LocalizedMessage;
-
-/**
- * A request send from an admin VM to all of the peers to indicate that that should complete the
- * backup operation.
- *
- *
- */
-public class FinishBackupRequest extends CliLegacyMessage {
-  private static final Logger logger = LogService.getLogger();
-
-  private File targetDir;
-  private File baselineDir;
-  private boolean abort;
-
-  public FinishBackupRequest() {
-    super();
-  }
-
-  public FinishBackupRequest(File targetDir, File baselineDir, boolean abort) {
-    this.targetDir = targetDir;
-    this.baselineDir = baselineDir;
-    this.abort = abort;
-  }
-
-  public static Map<DistributedMember, Set<PersistentID>> send(DM dm, Set recipients,
-      File targetDir, File baselineDir, boolean abort) {
-    FinishBackupRequest request = new FinishBackupRequest(targetDir, baselineDir, abort);
-    request.setRecipients(recipients);
-
-    FinishBackupReplyProcessor replyProcessor = new FinishBackupReplyProcessor(dm, recipients);
-    request.msgId = replyProcessor.getProcessorId();
-    dm.putOutgoing(request);
-    try {
-      replyProcessor.waitForReplies();
-    } catch (ReplyException e) {
-      if (!(e.getCause() instanceof CancelException)) {
-        throw e;
-      }
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    }
-    AdminResponse response = request.createResponse((DistributionManager) dm);
-    response.setSender(dm.getDistributionManagerId());
-    replyProcessor.process(response);
-    return replyProcessor.results;
-  }
-
-  @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
-    InternalCache cache = dm.getCache();
-    HashSet<PersistentID> persistentIds;
-    if (cache == null || cache.getBackupManager() == null) {
-      persistentIds = new HashSet<PersistentID>();
-    } else {
-      try {
-        persistentIds = cache.getBackupManager().doBackup(targetDir, baselineDir, abort);
-      } catch (IOException e) {
-        logger.error(
-            LocalizedMessage.create(LocalizedStrings.CliLegacyMessage_ERROR, this.getClass()), e);
-        return AdminFailureResponse.create(dm, getSender(), e);
-      }
-    }
-
-    return new FinishBackupResponse(this.getSender(), persistentIds);
-  }
-
-  public int getDSFID() {
-    return FINISH_BACKUP_REQUEST;
-  }
-
-  @Override
-  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-    super.fromData(in);
-    targetDir = DataSerializer.readFile(in);
-    baselineDir = DataSerializer.readFile(in);
-    abort = DataSerializer.readBoolean(in);
-  }
-
-  @Override
-  public void toData(DataOutput out) throws IOException {
-    super.toData(out);
-    DataSerializer.writeFile(targetDir, out);
-    DataSerializer.writeFile(baselineDir, out);
-    DataSerializer.writeBoolean(abort, out);
-  }
-
-  private static class FinishBackupReplyProcessor extends AdminMultipleReplyProcessor {
-    Map<DistributedMember, Set<PersistentID>> results =
-        Collections.synchronizedMap(new HashMap<DistributedMember, Set<PersistentID>>());
-
-    public FinishBackupReplyProcessor(DM dm, Collection initMembers) {
-      super(dm, initMembers);
-    }
-
-    @Override
-    protected boolean stopBecauseOfExceptions() {
-      return false;
-    }
-
-
-
-    @Override
-    protected int getAckWaitThreshold() {
-      // Disable the 15 second warning if the backup is taking a long time
-      return 0;
-    }
-
-    @Override
-    public long getAckSevereAlertThresholdMS() {
-      // Don't log severe alerts for backups either
-      return Long.MAX_VALUE;
-    }
-
-    @Override
-    protected void process(DistributionMessage msg, boolean warn) {
-      if (msg instanceof FinishBackupResponse) {
-        final HashSet<PersistentID> persistentIds = ((FinishBackupResponse) msg).getPersistentIds();
-        if (persistentIds != null && !persistentIds.isEmpty()) {
-          results.put(msg.getSender(), persistentIds);
-        }
-      }
-      super.process(msg, warn);
-    }
-
-
-
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupResponse.java b/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupResponse.java
deleted file mode 100644
index 97fdb30..0000000
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/FinishBackupResponse.java
+++ /dev/null
@@ -1,76 +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.admin.internal;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.HashSet;
-
-import org.apache.geode.DataSerializer;
-import org.apache.geode.cache.persistence.PersistentID;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.admin.remote.AdminResponse;
-
-/**
- * The reply for a {@link FinishBackupRequest}. The reply contains the persistent ids of the disk
- * stores that were backed up on this member.
- *
- *
- */
-public class FinishBackupResponse extends AdminResponse {
-
-  private HashSet<PersistentID> persistentIds;
-
-  public FinishBackupResponse() {
-    super();
-  }
-
-  public FinishBackupResponse(InternalDistributedMember sender,
-      HashSet<PersistentID> persistentIds) {
-    this.setRecipient(sender);
-    this.persistentIds = persistentIds;
-  }
-
-  public HashSet<PersistentID> getPersistentIds() {
-    return persistentIds;
-  }
-
-  @Override
-  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
-    super.fromData(in);
-    persistentIds = DataSerializer.readHashSet(in);
-  }
-
-  @Override
-  public void toData(DataOutput out) throws IOException {
-    super.toData(out);
-    DataSerializer.writeHashSet(persistentIds, out);
-  }
-
-  @Override
-  protected Object clone() throws CloneNotSupportedException {
-    return super.clone();
-  }
-
-  public int getDSFID() {
-    return FINISH_BACKUP_RESPONSE;
-  }
-
-  @Override
-  public String toString() {
-    return getClass().getName() + ": " + persistentIds;
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskRequest.java b/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskRequest.java
deleted file mode 100644
index 3a0048a..0000000
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskRequest.java
+++ /dev/null
@@ -1,92 +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.admin.internal;
-
-import java.util.Collection;
-import java.util.Set;
-
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.CancelException;
-import org.apache.geode.cache.DiskStore;
-import org.apache.geode.distributed.internal.DM;
-import org.apache.geode.distributed.internal.DistributionManager;
-import org.apache.geode.distributed.internal.ReplyException;
-import org.apache.geode.internal.admin.remote.AdminMultipleReplyProcessor;
-import org.apache.geode.internal.admin.remote.AdminResponse;
-import org.apache.geode.internal.admin.remote.CliLegacyMessage;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.logging.LogService;
-
-/**
- * A request to from an admin VM to all non admin members to start a backup. In the prepare phase of
- * the backup, the members will suspend bucket destroys to make sure buckets aren't missed during
- * the backup.
- */
-public class FlushToDiskRequest extends CliLegacyMessage {
-  private static final Logger logger = LogService.getLogger();
-
-  public FlushToDiskRequest() {
-    // nothing
-  }
-
-  public static void send(DM dm, Set recipients) {
-    FlushToDiskRequest request = new FlushToDiskRequest();
-    request.setRecipients(recipients);
-
-    FlushToDiskProcessor replyProcessor = new FlushToDiskProcessor(dm, recipients);
-    request.msgId = replyProcessor.getProcessorId();
-    dm.putOutgoing(request);
-    try {
-      replyProcessor.waitForReplies();
-    } catch (ReplyException e) {
-      if (!(e.getCause() instanceof CancelException)) {
-        throw e;
-      }
-    } catch (InterruptedException e) {
-      logger.debug(e);
-    }
-    AdminResponse response = request.createResponse((DistributionManager) dm);
-    response.setSender(dm.getDistributionManagerId());
-    replyProcessor.process(response);
-  }
-
-  @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
-    InternalCache cache = dm.getCache();
-    if (cache != null) {
-      cache.listDiskStoresIncludingRegionOwned().forEach(DiskStore::flush);
-    }
-
-    return new FlushToDiskResponse(this.getSender());
-  }
-
-  @Override
-  public int getDSFID() {
-    return FLUSH_TO_DISK_REQUEST;
-  }
-
-  private static class FlushToDiskProcessor extends AdminMultipleReplyProcessor {
-    public FlushToDiskProcessor(DM dm, Collection initMembers) {
-      super(dm, initMembers);
-    }
-
-    @Override
-    protected boolean stopBecauseOfExceptions() {
-      return false;
-    }
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java b/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java
deleted file mode 100644
index 869f774..0000000
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupRequest.java
+++ /dev/null
@@ -1,134 +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.admin.internal;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.logging.log4j.Logger;
-
-import org.apache.geode.CancelException;
-import org.apache.geode.cache.persistence.PersistentID;
-import org.apache.geode.distributed.DistributedMember;
-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.internal.admin.remote.AdminFailureResponse;
-import org.apache.geode.internal.admin.remote.AdminMultipleReplyProcessor;
-import org.apache.geode.internal.admin.remote.AdminResponse;
-import org.apache.geode.internal.admin.remote.CliLegacyMessage;
-import org.apache.geode.internal.cache.BackupManager;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-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.log4j.LocalizedMessage;
-
-/**
- * A request to from an admin VM to all non admin members to start a backup. In the prepare phase of
- * the backup, the members will suspend bucket destroys to make sure buckets aren't missed during
- * the backup.
- *
- *
- */
-public class PrepareBackupRequest extends CliLegacyMessage {
-  private static final Logger logger = LogService.getLogger();
-
-  public PrepareBackupRequest() {
-
-  }
-
-  public static Map<DistributedMember, Set<PersistentID>> send(DM dm, Set recipients) {
-    PrepareBackupRequest request = new PrepareBackupRequest();
-    request.setRecipients(recipients);
-
-    PrepareBackupReplyProcessor replyProcessor = new PrepareBackupReplyProcessor(dm, recipients);
-    request.msgId = replyProcessor.getProcessorId();
-    dm.putOutgoing(request);
-    try {
-      replyProcessor.waitForReplies();
-    } catch (ReplyException e) {
-      if (!(e.getCause() instanceof CancelException)) {
-        throw e;
-      }
-    } catch (InterruptedException e) {
-      e.printStackTrace();
-    }
-    AdminResponse response = request.createResponse((DistributionManager) dm);
-    response.setSender(dm.getDistributionManagerId());
-    replyProcessor.process(response);
-    return replyProcessor.results;
-  }
-
-  @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
-    InternalCache cache = dm.getCache();
-    HashSet<PersistentID> persistentIds;
-    if (cache == null) {
-      persistentIds = new HashSet<>();
-    } else {
-      try {
-        BackupManager manager = cache.startBackup(getSender());
-        persistentIds = manager.prepareForBackup();
-      } catch (IOException e) {
-        logger.error(
-            LocalizedMessage.create(LocalizedStrings.CliLegacyMessage_ERROR, this.getClass()), e);
-        return AdminFailureResponse.create(dm, getSender(), e);
-      }
-    }
-
-
-    return new PrepareBackupResponse(this.getSender(), persistentIds);
-  }
-
-  public int getDSFID() {
-    return PREPARE_BACKUP_REQUEST;
-  }
-
-  private static class PrepareBackupReplyProcessor extends AdminMultipleReplyProcessor {
-    Map<DistributedMember, Set<PersistentID>> results =
-        Collections.synchronizedMap(new HashMap<DistributedMember, Set<PersistentID>>());
-
-    public PrepareBackupReplyProcessor(DM dm, Collection initMembers) {
-      super(dm, initMembers);
-    }
-
-    @Override
-    protected boolean stopBecauseOfExceptions() {
-      return false;
-    }
-
-    @Override
-    protected void process(DistributionMessage msg, boolean warn) {
-      if (msg instanceof PrepareBackupResponse) {
-        final HashSet<PersistentID> persistentIds =
-            ((PrepareBackupResponse) msg).getPersistentIds();
-        if (persistentIds != null && !persistentIds.isEmpty()) {
-          results.put(msg.getSender(), persistentIds);
-        }
-      }
-      super.process(msg, warn);
-    }
-
-
-
-  }
-}
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
index 1ce7421..bdc18f4 100644
--- 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
@@ -24,6 +24,7 @@ import java.util.Set;
 import java.util.concurrent.ExecutorService;
 
 import org.apache.geode.CancelCriterion;
+import org.apache.geode.admin.GemFireHealthConfig;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.Role;
 import org.apache.geode.distributed.internal.locks.ElderState;
@@ -37,31 +38,31 @@ import org.apache.geode.internal.cache.InternalCache;
  */
 public interface DM extends ReplySender {
 
-  public boolean shutdownInProgress();
+  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.
    */
-  public long cacheTimeMillis();
+  long cacheTimeMillis();
 
   /**
    * Returns the id of this distribution manager.
    */
-  public InternalDistributedMember getDistributionManagerId();
+  InternalDistributedMember getDistributionManagerId();
 
   /**
    * Get a set of all other members (both admin ones and normal).
    *
    * @since GemFire 5.7
    */
-  public Set getAllOtherMembers();
+  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.
    */
-  public InternalDistributedMember getCanonicalId(DistributedMember id);
+  InternalDistributedMember getCanonicalId(DistributedMember id);
 
   /**
    * removes members that have older versions from the given collection, typically a Set from a
@@ -69,7 +70,7 @@ public interface DM extends ReplySender {
    *
    * @since GemFire 8.0
    */
-  public void retainMembersWithSameOrNewerVersion(Collection<InternalDistributedMember> members,
+  void retainMembersWithSameOrNewerVersion(Collection<InternalDistributedMember> members,
       Version version);
 
   /**
@@ -78,20 +79,20 @@ public interface DM extends ReplySender {
    *
    * @since GemFire 8.0
    */
-  public void removeMembersWithSameOrNewerVersion(Collection<InternalDistributedMember> members,
+  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.
    */
-  public Set getDistributionManagerIds();
+  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.
    */
-  public Set getNormalDistributionManagerIds();
+  Set getNormalDistributionManagerIds();
 
   /**
    * Returns an unmodifiable set containing the identities of all of the known distribution managers
@@ -99,22 +100,22 @@ public interface DM extends ReplySender {
    *
    * @since GemFire 5.7
    */
-  public Set getDistributionManagerIdsIncludingAdmin();
+  Set getDistributionManagerIdsIncludingAdmin();
 
   /**
    * Returns a private-memory list containing getDistributionManagerIds() minus our id.
    */
-  public Set getOtherDistributionManagerIds();
+  Set getOtherDistributionManagerIds();
 
   /**
    * Returns a private-memory list containing getNormalDistributionManagerIds() minus our id.
    */
-  public Set getOtherNormalDistributionManagerIds();
+  Set getOtherNormalDistributionManagerIds();
 
   /**
    * Add a membership listener and return other DistribtionManagerIds as an atomic operation
    */
-  public Set addMembershipListenerAndGetDistributionManagerIds(MembershipListener l);
+  Set addMembershipListenerAndGetDistributionManagerIds(MembershipListener l);
 
   /**
    * Add a membership listener for all members and return other DistribtionManagerIds as an atomic
@@ -122,19 +123,19 @@ public interface DM extends ReplySender {
    *
    * @since GemFire 5.7
    */
-  public Set addAllMembershipListenerAndGetAllIds(MembershipListener l);
+  Set addAllMembershipListenerAndGetAllIds(MembershipListener l);
 
   /**
    * Returns the identity of this <code>DistributionManager</code>
    */
-  public InternalDistributedMember getId();
+  InternalDistributedMember getId();
 
   /**
    * Return true if no other distribution manager was in this group when he joined.
    *
    * @since GemFire 4.0
    */
-  public boolean isAdam();
+  boolean isAdam();
 
   /**
    * Returns the identity of the oldest DM in this group.
@@ -144,19 +145,19 @@ public interface DM extends ReplySender {
    * @return the elder member, possibly null
    * @since GemFire 4.0
    */
-  public InternalDistributedMember getElderId();
+  InternalDistributedMember getElderId();
 
   /**
    * Return true if this is the oldest DM in this group.
    *
    * @since GemFire 5.0
    */
-  public boolean isElder();
+  boolean isElder();
 
   /**
    * Return true if this DM is a loner that is not part of a real distributed system.
    */
-  public boolean isLoner();
+  boolean isLoner();
 
   /**
    * Returns the elder state or null if this DM is not the elder.
@@ -169,14 +170,14 @@ public interface DM extends ReplySender {
    * @throws IllegalStateException if elder try lock fails
    * @since GemFire 4.0
    */
-  public ElderState getElderState(boolean force, boolean useTryLock);
+  ElderState getElderState(boolean force, boolean useTryLock);
 
   /**
    * Returns the id of the underlying distribution channel used for communication.
    *
    * @since GemFire 3.0
    */
-  public long getChannelId();
+  long getChannelId();
 
   /**
    * Adds a message to the outgoing queue. Note that <code>message</code> should not be modified
@@ -187,7 +188,7 @@ public interface DM extends ReplySender {
    * @throws NotSerializableException If <code>message</code> cannot be serialized
    * @see #putOutgoing(DistributionMessage)
    */
-  public Set putOutgoingUserData(DistributionMessage message) throws NotSerializableException;
+  Set putOutgoingUserData(DistributionMessage message) throws NotSerializableException;
 
   /**
    * Sends a message, guaranteed to be serialized
@@ -196,24 +197,24 @@ public interface DM extends ReplySender {
    * @param msg
    * @return recipients who did not receive the message
    */
-  public Set putOutgoing(DistributionMessage msg);
+  Set putOutgoing(DistributionMessage msg);
 
   /**
    * Returns the distributed system to which this distribution manager is connected.
    */
-  public InternalDistributedSystem getSystem();
+  InternalDistributedSystem getSystem();
 
   /**
    * Adds a <code>MembershipListener</code> to this distribution manager.
    */
-  public void addMembershipListener(MembershipListener l);
+  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
    */
-  public void removeMembershipListener(MembershipListener l);
+  void removeMembershipListener(MembershipListener l);
 
   /**
    * Removes a <code>MembershipListener</code> listening for all members from this distribution
@@ -222,11 +223,11 @@ public interface DM extends ReplySender {
    * @throws IllegalArgumentException <code>l</code> was not registered on this distribution manager
    * @since GemFire 5.7
    */
-  public void removeAllMembershipListener(MembershipListener l);
+  void removeAllMembershipListener(MembershipListener l);
 
-  public void addAdminConsole(InternalDistributedMember id);
+  void addAdminConsole(InternalDistributedMember id);
 
-  public DMStats getStats();
+  DMStats getStats();
 
   /**
    * Used to get the DistributionConfig so that Connection can figure out if it is configured for
@@ -234,7 +235,7 @@ public interface DM extends ReplySender {
    *
    * @since GemFire 4.2.1
    */
-  public DistributionConfig getConfig();
+  DistributionConfig getConfig();
 
   /**
    * Makes note of a distribution manager that has shut down. Invokes the appropriate listeners.
@@ -243,28 +244,27 @@ public interface DM extends ReplySender {
    *
    * @see ShutdownMessage#process
    */
-  public void handleManagerDeparture(InternalDistributedMember theId, boolean crashed,
-      String reason);
+  void handleManagerDeparture(InternalDistributedMember theId, boolean crashed, String reason);
 
   /**
    * getThreadPool gets this distribution manager's message-processing thread pool
    */
-  public ExecutorService getThreadPool();
+  ExecutorService getThreadPool();
 
   /**
    * Return the high-priority message-processing executor
    */
-  public ExecutorService getHighPriorityThreadPool();
+  ExecutorService getHighPriorityThreadPool();
 
   /**
    * Return the waiting message-processing executor
    */
-  public ExecutorService getWaitingThreadPool();
+  ExecutorService getWaitingThreadPool();
 
   /**
    * Return the special waiting message-processing executor
    */
-  public ExecutorService getPrMetaDataCleanupThreadPool();
+  ExecutorService getPrMetaDataCleanupThreadPool();
 
   /**
    * gets this distribution manager's message-processing executor for ordered (i.e. serialized)
@@ -272,13 +272,13 @@ public interface DM extends ReplySender {
    */
   // public Executor getSerialExecutor();
 
-  public void close();
+  void close();
 
   /**
    * Returns the ordered list of current DistributionManagers in oldest-to-youngest order. Added for
    * DLockGrantor
    */
-  public List<InternalDistributedMember> getViewMembers();
+  List<InternalDistributedMember> getViewMembers();
 
   /**
    * Returns the oldest member in the given set of distribution managers. The current implementation
@@ -288,27 +288,27 @@ public interface DM extends ReplySender {
    * @throws NoSuchElementException when none of the given members is actually a member of the
    *         distributed system.
    */
-  public DistributedMember getOldestMember(Collection members) throws NoSuchElementException;
+  DistributedMember getOldestMember(Collection members) throws NoSuchElementException;
 
   /**
    * @return Set of Admin VM nodes
    */
-  public Set getAdminMemberSet();
+  Set getAdminMemberSet();
 
   /** Throws ShutdownException if closeInProgress returns true. */
-  public void throwIfDistributionStopped();
+  void throwIfDistributionStopped();
 
   /** Returns count of members filling the specified role */
-  public int getRoleCount(Role role);
+  int getRoleCount(Role role);
 
   /** Returns true if at least one member is filling the specified role */
-  public boolean isRolePresent(Role role);
+  boolean isRolePresent(Role role);
 
   /** Returns a set of all roles currently in the distributed system. */
-  public Set getAllRoles();
+  Set getAllRoles();
 
   /** Returns true if id is a current member of the distributed system */
-  public boolean isCurrentMember(InternalDistributedMember id);
+  boolean isCurrentMember(InternalDistributedMember id);
 
   /**
    * Remove given member from list of members who are pending a startup reply
@@ -316,37 +316,37 @@ public interface DM extends ReplySender {
    * @param m the member
    * @param departed true if we're removing them due to membership
    */
-  public void removeUnfinishedStartup(InternalDistributedMember m, boolean departed);
+  void removeUnfinishedStartup(InternalDistributedMember m, boolean departed);
 
-  public void setUnfinishedStartups(Collection s);
+  void setUnfinishedStartups(Collection s);
 
   /**
    * Return the CancelCriterion for this DM.
    *
    * @return CancelCriterion for this DM
    */
-  public CancelCriterion getCancelCriterion();
+  CancelCriterion getCancelCriterion();
 
   /**
    * Return the membership manager for this DM
    *
    * @return the membership manager
    */
-  public MembershipManager getMembershipManager();
+  MembershipManager getMembershipManager();
 
   /**
    * Set the root cause for DM failure
    *
    * @param t the underlying failure
    */
-  public void setRootCause(Throwable t);
+  void setRootCause(Throwable t);
 
   /**
    * Return the underlying root cause for DM failure, possibly null
    *
    * @return the underlying root cause
    */
-  public Throwable getRootCause();
+  Throwable getRootCause();
 
   /**
    * Return all members that are on the the this host
@@ -354,27 +354,25 @@ public interface DM extends ReplySender {
    * @return set of {@link InternalDistributedMember} including this VM
    * @since GemFire 5.9
    */
-  public Set<InternalDistributedMember> getMembersInThisZone();
+  Set<InternalDistributedMember> getMembersInThisZone();
 
   /**
    * Acquire a permit to request a GII from another member
    */
-  public void acquireGIIPermitUninterruptibly();
+  void acquireGIIPermitUninterruptibly();
 
   /**
    * Release a permit to request a GII from another member.
    */
-  public void releaseGIIPermit();
+  void releaseGIIPermit();
 
-  public int getDistributedSystemId();
+  int getDistributedSystemId();
 
-  public boolean enforceUniqueZone();
+  boolean enforceUniqueZone();
 
-  public Set<InternalDistributedMember> getMembersInSameZone(
-      InternalDistributedMember acceptedMember);
+  Set<InternalDistributedMember> getMembersInSameZone(InternalDistributedMember acceptedMember);
 
-  public boolean areInSameZone(InternalDistributedMember member1,
-      InternalDistributedMember member2);
+  boolean areInSameZone(InternalDistributedMember member1, InternalDistributedMember member2);
 
   /**
    * Returns true is the two members are on the same equivalent host machine based on overlapping IP
@@ -383,12 +381,11 @@ public interface DM extends ReplySender {
    * @param member1 First member
    * @param member2 Second member
    */
-  public boolean areOnEquivalentHost(InternalDistributedMember member1,
-      InternalDistributedMember member2);
+  boolean areOnEquivalentHost(InternalDistributedMember member1, InternalDistributedMember member2);
 
-  public Set<InetAddress> getEquivalents(InetAddress in);
+  Set<InetAddress> getEquivalents(InetAddress in);
 
-  public Set<DistributedMember> getGroupMembers(String group);
+  Set<DistributedMember> getGroupMembers(String group);
 
   /**
    * Adds the entry in hostedLocators for a member with one or more hosted locators. The value is a
@@ -402,7 +399,7 @@ public interface DM extends ReplySender {
    *
    * @since GemFire 6.6.3
    */
-  public void addHostedLocators(InternalDistributedMember member, Collection<String> locators,
+  void addHostedLocators(InternalDistributedMember member, Collection<String> locators,
       boolean isSharedConfigurationEnabled);
 
 
@@ -415,7 +412,7 @@ public interface DM extends ReplySender {
    *
    * @since GemFire 6.6.3
    */
-  public Collection<String> getHostedLocators(InternalDistributedMember member);
+  Collection<String> getHostedLocators(InternalDistributedMember member);
 
   /**
    * Gets the map of all members hosting locators. The key is the member, and the value is a
@@ -428,7 +425,7 @@ public interface DM extends ReplySender {
    *
    * @since GemFire 6.6.3
    */
-  public Map<InternalDistributedMember, Collection<String>> getAllHostedLocators();
+  Map<InternalDistributedMember, Collection<String>> getAllHostedLocators();
 
   /**
    * Gets the map of all members hosting locators with shared configuration. The key is the member,
@@ -439,7 +436,7 @@ public interface DM extends ReplySender {
    *
    * @since GemFire 8.0
    */
-  public Map<InternalDistributedMember, Collection<String>> getAllHostedLocatorsWithSharedConfiguration();
+  Map<InternalDistributedMember, Collection<String>> getAllHostedLocatorsWithSharedConfiguration();
 
   /****
    * Determines if the distributed system has the shared configuration service enabled or not.
@@ -447,24 +444,30 @@ public interface DM extends ReplySender {
    * @return true if the distributed system was started or had a locator with
    *         enable-cluster-configuration = true
    */
-  public boolean isSharedConfigurationServiceEnabledForDS();
+  boolean isSharedConfigurationServiceEnabledForDS();
 
   /**
    * 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.
    */
-  public void forceUDPMessagingForCurrentThread();
+  void forceUDPMessagingForCurrentThread();
 
   /**
    * Releases use of UDP for all communications in the current thread, as established by
    * forceUDPMessagingForCurrentThread.
    */
-  public void releaseUDPMessagingForCurrentThread();
+  void releaseUDPMessagingForCurrentThread();
 
   int getDMType();
 
   InternalCache getCache();
 
   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/DistributionMessageObserver.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionMessageObserver.java
index e918e12..72e4811 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
@@ -14,11 +14,8 @@
  */
 package org.apache.geode.distributed.internal;
 
-
 /**
  * This class is a test hook to intercept DistributionMessages in the VM receiving the message.
- *
- *
  */
 public abstract class DistributionMessageObserver {
 
@@ -27,7 +24,6 @@ public abstract class DistributionMessageObserver {
   /**
    * Set the instance of the observer. Setting to null will clear the observer.
    *
-   * @param instance
    * @return the old observer, or null if there was no old observer.
    */
   public static DistributionMessageObserver setInstance(DistributionMessageObserver instance) {
@@ -47,7 +43,7 @@ public abstract class DistributionMessageObserver {
    * @param message The message itself
    */
   public void beforeProcessMessage(DistributionManager dm, DistributionMessage message) {
-
+    // override as needed
   }
 
   /**
@@ -57,16 +53,16 @@ public abstract class DistributionMessageObserver {
    * @param message The message itself
    */
   public void afterProcessMessage(DistributionManager dm, DistributionMessage message) {
-
+    // override as needed
   }
 
   /**
    * Called just before a message is distributed.
    *
-   * @param dm the distribution manager that's sending the messsage
-   * @param msg the message itself
+   * @param dm the distribution manager that's sending the message
+   * @param message the message itself
    */
-  public void beforeSendMessage(DistributionManager dm, DistributionMessage msg) {
-
+  public void beforeSendMessage(DistributionManager dm, DistributionMessage message) {
+    // override as needed
   }
 }
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 8bbe019..b1213ca 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
@@ -22,6 +22,7 @@ import java.util.concurrent.*;
 
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.InternalGemFireError;
+import org.apache.geode.admin.GemFireHealthConfig;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.DurableClientAttributes;
 import org.apache.geode.distributed.Role;
@@ -1387,4 +1388,22 @@ public class LonerDistributionManager implements DM {
   public void setCache(InternalCache instance) {
     this.cache = instance;
   }
+
+  @Override
+  public HealthMonitor getHealthMonitor(InternalDistributedMember owner) {
+    throw new UnsupportedOperationException(
+        "getHealthMonitor is not supported by " + getClass().getSimpleName());
+  }
+
+  @Override
+  public void removeHealthMonitor(InternalDistributedMember owner, int theId) {
+    throw new UnsupportedOperationException(
+        "removeHealthMonitor is not supported by " + getClass().getSimpleName());
+  }
+
+  @Override
+  public void createHealthMonitor(InternalDistributedMember owner, GemFireHealthConfig cfg) {
+    throw new UnsupportedOperationException(
+        "createHealthMonitor is not supported by " + getClass().getSimpleName());
+  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/DSFIDFactory.java b/geode-core/src/main/java/org/apache/geode/internal/DSFIDFactory.java
index f8c8d80..c02be89 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DSFIDFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DSFIDFactory.java
@@ -24,12 +24,6 @@ import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
 
 import org.apache.geode.DataSerializer;
 import org.apache.geode.InternalGemFireError;
-import org.apache.geode.admin.internal.FinishBackupRequest;
-import org.apache.geode.admin.internal.FinishBackupResponse;
-import org.apache.geode.admin.internal.FlushToDiskRequest;
-import org.apache.geode.admin.internal.FlushToDiskResponse;
-import org.apache.geode.admin.internal.PrepareBackupRequest;
-import org.apache.geode.admin.internal.PrepareBackupResponse;
 import org.apache.geode.admin.internal.SystemMemberCacheEventProcessor;
 import org.apache.geode.admin.jmx.internal.StatAlertNotification;
 import org.apache.geode.cache.InterestResultPolicy;
@@ -281,6 +275,11 @@ import org.apache.geode.internal.cache.UpdateAttributesProcessor;
 import org.apache.geode.internal.cache.UpdateEntryVersionOperation.UpdateEntryVersionMessage;
 import org.apache.geode.internal.cache.UpdateOperation;
 import org.apache.geode.internal.cache.VMCachedDeserializable;
+import org.apache.geode.internal.cache.backup.BackupResponse;
+import org.apache.geode.internal.cache.backup.FinishBackupRequest;
+import org.apache.geode.internal.cache.backup.FlushToDiskRequest;
+import org.apache.geode.internal.cache.backup.FlushToDiskResponse;
+import org.apache.geode.internal.cache.backup.PrepareBackupRequest;
 import org.apache.geode.internal.cache.compression.SnappyCompressedCachedDeserializable;
 import org.apache.geode.internal.cache.control.ResourceAdvisor.ResourceManagerProfile;
 import org.apache.geode.internal.cache.control.ResourceAdvisor.ResourceProfileMessage;
@@ -874,9 +873,13 @@ public class DSFIDFactory implements DataSerializableFixedID {
     registerDSFID(CLIENT_MEMBERSHIP_MESSAGE, ClientMembershipMessage.class);
     registerDSFID(END_BUCKET_CREATION_MESSAGE, EndBucketCreationMessage.class);
     registerDSFID(PREPARE_BACKUP_REQUEST, PrepareBackupRequest.class);
-    registerDSFID(PREPARE_BACKUP_RESPONSE, PrepareBackupResponse.class);
+    registerDSFID(BACKUP_RESPONSE, BackupResponse.class); // in older versions this was
+                                                          // FinishBackupResponse which is
+                                                          // compatible
     registerDSFID(FINISH_BACKUP_REQUEST, FinishBackupRequest.class);
-    registerDSFID(FINISH_BACKUP_RESPONSE, FinishBackupResponse.class);
+    registerDSFID(FINISH_BACKUP_RESPONSE, BackupResponse.class); // for backwards compatibility map
+                                                                 // FINISH_BACKUP_RESPONSE to
+                                                                 // BackupResponse
     registerDSFID(COMPACT_REQUEST, CompactRequest.class);
     registerDSFID(COMPACT_RESPONSE, CompactResponse.class);
     registerDSFID(FLOW_CONTROL_PERMIT_MESSAGE, FlowControlPermitMessage.class);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/DataSerializableFixedID.java b/geode-core/src/main/java/org/apache/geode/internal/DataSerializableFixedID.java
index e7a319e..d81f067 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/DataSerializableFixedID.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/DataSerializableFixedID.java
@@ -727,7 +727,7 @@ public interface DataSerializableFixedID extends SerializationVersions {
   public static final short FINISH_BACKUP_REQUEST = 2114;
   public static final short FINISH_BACKUP_RESPONSE = 2115;
   public static final short PREPARE_BACKUP_REQUEST = 2116;
-  public static final short PREPARE_BACKUP_RESPONSE = 2117;
+  public static final short BACKUP_RESPONSE = 2117;
   public static final short COMPACT_REQUEST = 2118;
   public static final short COMPACT_RESPONSE = 2119;
   public static final short FLOW_CONTROL_PERMIT_MESSAGE = 2120;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java b/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
index 2f1eda9..d6660b2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
@@ -75,8 +75,8 @@ import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.distributed.internal.tcpserver.TcpClient;
 import org.apache.geode.internal.admin.remote.TailLogResponse;
-import org.apache.geode.internal.cache.BackupUtil;
 import org.apache.geode.internal.cache.DiskStoreImpl;
+import org.apache.geode.internal.cache.backup.BackupUtil;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.DateFormatter;
 import org.apache.geode.internal.logging.MergeLogFiles;
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 795b00c..4ffe064 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(DistributionManager dm) {
+  protected AdminResponse createResponse(DM 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 3fda3f7..50aed79 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(DistributionManager dm,
-      InternalDistributedMember recipient, GemFireHealthConfig cfg) {
+  public static AddHealthListenerResponse create(DM 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 2165ee8..37111dc 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
@@ -47,9 +47,11 @@ public class AddStatListenerRequest extends AdminRequest {
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM 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 b5f52a9..19b430e 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(DistributionManager dm,
-      InternalDistributedMember recipient, long resourceId, String statName) {
+  public static AddStatListenerResponse create(DM 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/AdminFailureResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminFailureResponse.java
index fc5556b..b709cf5 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminFailureResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/AdminFailureResponse.java
@@ -12,40 +12,38 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
-
 package org.apache.geode.internal.admin.remote;
 
-import java.io.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 
-import org.apache.geode.*;
-import org.apache.geode.distributed.internal.*;
-import org.apache.geode.distributed.internal.membership.*;
+import org.apache.geode.DataSerializer;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 
 /**
  * A response to a failed request.
  */
 public class AdminFailureResponse extends AdminResponse {
-  // instance variables
-  Exception cause;
+
+  private Exception cause;
 
   /**
-   * Returns a <code>AdminFailureResponse</code> that will be returned to the specified recipient.
-   * The message will contains a copy of the local manager's system config.
+   * Returns a {@code AdminFailureResponse} that will be returned to the specified recipient. The
+   * message will contains a copy of the local manager's system config.
    */
-  public static AdminFailureResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, Exception cause) {
-    AdminFailureResponse m = new AdminFailureResponse();
-    m.setRecipient(recipient);
-    m.cause = cause;
-    return m;
+  public static AdminFailureResponse create(InternalDistributedMember recipient, Exception cause) {
+    AdminFailureResponse message = new AdminFailureResponse();
+    message.setRecipient(recipient);
+    message.cause = cause;
+    return message;
   }
 
-  // instance methods
   public Exception getCause() {
     return this.cause;
   }
 
+  @Override
   public int getDSFID() {
     return ADMIN_FAILURE_RESPONSE;
   }
@@ -59,11 +57,11 @@ public class AdminFailureResponse extends AdminResponse {
   @Override
   public void fromData(DataInput in) throws IOException, ClassNotFoundException {
     super.fromData(in);
-    this.cause = (Exception) DataSerializer.readObject(in);
+    this.cause = DataSerializer.readObject(in);
   }
 
   @Override
   public String toString() {
-    return "AdminFailureResponse from " + this.getRecipient() + " cause=" + this.cause;
+    return "AdminFailureResponse from " + getRecipient() + " cause=" + this.cause;
   }
 }
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 95efbe7..3a5cff9 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,21 +16,15 @@ package org.apache.geode.internal.admin.remote;
 
 import java.util.Collection;
 
-import org.apache.geode.CancelCriterion;
 import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionMessage;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ReplyException;
-import org.apache.geode.distributed.internal.ReplyMessage;
 import org.apache.geode.distributed.internal.ReplyProcessor21;
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-
 
 /**
  * TODO prpersist. This code really needs to be merged with the AdminReplyProcessor. However, we're
  * getting close to the release and I don't want to mess with all of the admin code right now. We
  * need this class to handle failures from admin messages that expect replies from multiple members.
- *
  */
 public class AdminMultipleReplyProcessor extends ReplyProcessor21 {
 
@@ -38,47 +32,16 @@ public class AdminMultipleReplyProcessor extends ReplyProcessor21 {
     super(dm, initMembers);
   }
 
-  public AdminMultipleReplyProcessor(DM dm, InternalDistributedMember member) {
-    super(dm, member);
-  }
-
-  public AdminMultipleReplyProcessor(DM dm, InternalDistributedSystem system,
-      Collection initMembers, CancelCriterion cancelCriterion, boolean register) {
-    super(dm, system, initMembers, cancelCriterion, register);
-  }
-
-  public AdminMultipleReplyProcessor(InternalDistributedSystem system, Collection initMembers,
-      CancelCriterion cancelCriterion) {
-    super(system, initMembers, cancelCriterion);
-  }
-
-  public AdminMultipleReplyProcessor(InternalDistributedSystem system, Collection initMembers) {
-    super(system, initMembers);
-  }
-
-  public AdminMultipleReplyProcessor(InternalDistributedSystem system,
-      InternalDistributedMember member, CancelCriterion cancelCriterion) {
-    super(system, member, cancelCriterion);
-  }
-
-  public AdminMultipleReplyProcessor(InternalDistributedSystem system,
-      InternalDistributedMember member) {
-    super(system, member);
-  }
-
   @Override
-  protected void process(DistributionMessage msg, boolean warn) {
-    if (msg instanceof AdminFailureResponse) {
-      Exception ex = ((AdminFailureResponse) msg).getCause();
+  protected void process(DistributionMessage message, boolean warn) {
+    if (message instanceof AdminFailureResponse) {
+      Exception ex = ((AdminFailureResponse) message).getCause();
       if (ex != null) {
-        ReplyException rex = new ReplyException(ex);
-        rex.setSenderIfNull(msg.getSender());
-        processException(msg, rex);
+        ReplyException replyException = new ReplyException(ex);
+        replyException.setSenderIfNull(message.getSender());
+        processException(message, replyException);
       }
     }
-    super.process(msg, warn);
+    super.process(message, warn);
   }
-
-
-
 }
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 6ea7de6..58c078b 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,6 +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.DistributionManager;
 import org.apache.geode.distributed.internal.PooledDistributionMessage;
 import org.apache.geode.distributed.internal.ReplyException;
@@ -134,7 +135,7 @@ public abstract class AdminRequest extends PooledDistributionMessage {
       cpMgr.jumpToModifiedClassLoader(modifiedClasspath);
       response = createResponse(dm);
     } catch (Exception ex) {
-      response = AdminFailureResponse.create(dm, this.getSender(), ex);
+      response = AdminFailureResponse.create(this.getSender(), ex);
     } finally {
       cpMgr.revertToOldClassLoader();
     }
@@ -150,7 +151,7 @@ public abstract class AdminRequest extends PooledDistributionMessage {
   /**
    * Must return a proper response to this request.
    */
-  protected abstract AdminResponse createResponse(DistributionManager dm);
+  protected abstract AdminResponse createResponse(DM dm);
 
   @Override
   public void toData(DataOutput out) throws IOException {
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 04c4233..853d5bf 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.DistributionManager;
+import org.apache.geode.distributed.internal.DM;
 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(DistributionManager dm) {
+  protected AdminResponse createResponse(DM 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 ddc102d..6e9b32d 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.DistributionManager;
+import org.apache.geode.distributed.internal.DM;
 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(DistributionManager dm, BridgeServerRequest request) {
+  static BridgeServerResponse create(DM 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 dba5810..d833507 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(DistributionManager dm) {
+  protected AdminResponse createResponse(DM 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 0aae5ff..098d3f1 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.DistributionManager;
+import org.apache.geode.distributed.internal.DM;
 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(DistributionManager dm,
-      InternalDistributedMember recipient, int cacheId, byte attributeCode, int newValue) {
+  public static CacheConfigResponse create(DM 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 db13737..2692d28 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
@@ -42,9 +42,11 @@ public class CacheInfoRequest extends AdminRequest {
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM 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 f6191c5..bc79498 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.DistributionManager;
+import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalCache;
 
@@ -37,8 +37,7 @@ public class CacheInfoResponse extends AdminResponse {
   /**
    * Returns a {@code CacheInfoResponse} that will be returned to the specified recipient.
    */
-  public static CacheInfoResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static CacheInfoResponse create(DM dm, InternalDistributedMember recipient) {
     CacheInfoResponse m = new CacheInfoResponse();
     m.setRecipient(recipient);
     try {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CancelStatListenerRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CancelStatListenerRequest.java
index 0602fc4..2ac6121 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CancelStatListenerRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CancelStatListenerRequest.java
@@ -47,7 +47,7 @@ public class CancelStatListenerRequest extends AdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return CancelStatListenerResponse.create(dm, this.getSender(), this.listenerId);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CancelStatListenerResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CancelStatListenerResponse.java
index d105fa3..73058bb 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CancelStatListenerResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CancelStatListenerResponse.java
@@ -34,8 +34,8 @@ public class CancelStatListenerResponse extends AdminResponse {
    * Returns a <code>CancelStatListenerResponse</code> that will be returned to the specified
    * recipient. The message will contains a copy of the local manager's system config.
    */
-  public static CancelStatListenerResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, int listenerId) {
+  public static CancelStatListenerResponse create(DM dm, InternalDistributedMember recipient,
+      int listenerId) {
     CancelStatListenerResponse m = new CancelStatListenerResponse();
     m.setRecipient(recipient);
     GemFireStatSampler sampler = null;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CliLegacyMessage.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CliLegacyMessage.java
index 3e0fc6e..47bf5ff 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CliLegacyMessage.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CliLegacyMessage.java
@@ -40,7 +40,7 @@ public abstract class CliLegacyMessage extends AdminRequest {
     } catch (Exception ex) {
       logger.error(
           LocalizedMessage.create(LocalizedStrings.CliLegacyMessage_ERROR, this.getClass()), ex);
-      response = AdminFailureResponse.create(dm, this.getSender(), ex);
+      response = AdminFailureResponse.create(this.getSender(), ex);
 
     }
     if (response != null) { // cancellations result in null response
@@ -51,5 +51,4 @@ public abstract class CliLegacyMessage extends AdminRequest {
           LocalizedStrings.AdminRequest_RESPONSE_TO__0__WAS_CANCELLED, this.getClass().getName()));
     }
   }
-
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CompactRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CompactRequest.java
index 96c87e2..df197f6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CompactRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/CompactRequest.java
@@ -35,7 +35,6 @@ 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.internal.cache.DiskStoreImpl;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.util.ArrayUtils;
@@ -81,7 +80,7 @@ public class CompactRequest extends CliLegacyMessage {
   }
 
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     InternalCache cache = dm.getCache();
     HashSet<PersistentID> compactedStores = new HashSet<>();
     if (cache != null && !cache.isClosed()) {
@@ -135,14 +134,14 @@ public class CompactRequest extends CliLegacyMessage {
     }
 
     @Override
-    protected void process(DistributionMessage msg, boolean warn) {
-      if (msg instanceof CompactResponse) {
-        final Set<PersistentID> persistentIds = ((CompactResponse) msg).getPersistentIds();
+    protected void process(DistributionMessage message, boolean warn) {
+      if (message instanceof CompactResponse) {
+        final Set<PersistentID> persistentIds = ((CompactResponse) message).getPersistentIds();
         if (persistentIds != null && !persistentIds.isEmpty()) {
-          this.results.put(msg.getSender(), persistentIds);
+          this.results.put(message.getSender(), persistentIds);
         }
       }
-      super.process(msg, warn);
+      super.process(message, warn);
     }
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DurableClientInfoRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DurableClientInfoRequest.java
index a079d25..5d8abd0 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DurableClientInfoRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DurableClientInfoRequest.java
@@ -55,8 +55,10 @@ public class DurableClientInfoRequest extends AdminRequest {
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return DurableClientInfoResponse.create(dm, this.getSender(), this);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DurableClientInfoResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DurableClientInfoResponse.java
index 98837c5..f44109c 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DurableClientInfoResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/DurableClientInfoResponse.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 
 import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.CacheServerImpl;
 import org.apache.geode.internal.cache.InternalCache;
@@ -40,8 +40,8 @@ public class DurableClientInfoResponse extends AdminResponse {
   /**
    * Returns a {@code DurableClientInfoResponse} that will be returned to the specified recipient.
    */
-  public static DurableClientInfoResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, DurableClientInfoRequest request) {
+  public static DurableClientInfoResponse create(DM dm, InternalDistributedMember recipient,
+      DurableClientInfoRequest request) {
     DurableClientInfoResponse m = new DurableClientInfoResponse();
     m.setRecipient(recipient);
     try {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchDistLockInfoRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchDistLockInfoRequest.java
index 0e394ae..5a3500f 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchDistLockInfoRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchDistLockInfoRequest.java
@@ -37,9 +37,11 @@ public class FetchDistLockInfoRequest extends AdminRequest {
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return FetchDistLockInfoResponse.create(dm, this.getSender());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchDistLockInfoResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchDistLockInfoResponse.java
index 4492fb8..5804e65 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchDistLockInfoResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchDistLockInfoResponse.java
@@ -35,8 +35,7 @@ public class FetchDistLockInfoResponse extends AdminResponse {
    * recipient. The message will contains a copy of the local manager's distributed lock service
    * information.
    */
-  public static FetchDistLockInfoResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static FetchDistLockInfoResponse create(DM dm, InternalDistributedMember recipient) {
     FetchDistLockInfoResponse m = new FetchDistLockInfoResponse();
     InternalDistributedMember id = dm.getDistributionManagerId();
     Set entries = DLockService.snapshotAllServices().entrySet();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHealthDiagnosisRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHealthDiagnosisRequest.java
index 37d0a33..05141b4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHealthDiagnosisRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHealthDiagnosisRequest.java
@@ -51,7 +51,7 @@ public class FetchHealthDiagnosisRequest extends AdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return FetchHealthDiagnosisResponse.create(dm, this.getSender(), this.id, this.healthCode);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHealthDiagnosisResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHealthDiagnosisResponse.java
index 1585907..e134f19 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHealthDiagnosisResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHealthDiagnosisResponse.java
@@ -36,8 +36,8 @@ public class FetchHealthDiagnosisResponse extends AdminResponse {
    * Returns a <code>FetchHealthDiagnosisResponse</code> that will be returned to the specified
    * recipient.
    */
-  public static FetchHealthDiagnosisResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, int id, GemFireHealth.Health healthCode) {
+  public static FetchHealthDiagnosisResponse create(DM dm, InternalDistributedMember recipient,
+      int id, GemFireHealth.Health healthCode) {
     FetchHealthDiagnosisResponse m = new FetchHealthDiagnosisResponse();
     m.setRecipient(recipient);
     {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHostRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHostRequest.java
index ee3d8b6..f27aa66 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHostRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHostRequest.java
@@ -41,7 +41,7 @@ public class FetchHostRequest extends AdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return FetchHostResponse.create(dm, this.getSender());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHostResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHostResponse.java
index f41f0ac..e568a47 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHostResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchHostResponse.java
@@ -27,8 +27,8 @@ import java.net.UnknownHostException;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
+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.GemFireVersion;
 import org.apache.geode.internal.cache.CacheServerLauncher;
@@ -59,8 +59,7 @@ public class FetchHostResponse extends AdminResponse {
    * Returns a <code>FetchHostResponse</code> that will be returned to the specified recipient. The
    * message will contains a copy of this vm's local host.
    */
-  public static FetchHostResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static FetchHostResponse create(DM dm, InternalDistributedMember recipient) {
     FetchHostResponse m = new FetchHostResponse();
     m.setRecipient(recipient);
     try {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchResourceAttributesRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchResourceAttributesRequest.java
index 308a759..ed1095a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchResourceAttributesRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchResourceAttributesRequest.java
@@ -38,7 +38,7 @@ public class FetchResourceAttributesRequest extends AdminRequest {
   }
 
   @Override
-  public AdminResponse createResponse(DistributionManager dm) {
+  public AdminResponse createResponse(DM dm) {
     return FetchResourceAttributesResponse.create(dm, this.getSender(), resourceUniqueId);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchResourceAttributesResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchResourceAttributesResponse.java
index 7f2be75..709a89d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchResourceAttributesResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchResourceAttributesResponse.java
@@ -12,23 +12,26 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
-
 package org.apache.geode.internal.admin.remote;
 
-import java.io.*;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 
-import org.apache.geode.*;
-import org.apache.geode.distributed.internal.*;
-import org.apache.geode.distributed.internal.membership.*;
+import org.apache.geode.DataSerializer;
+import org.apache.geode.StatisticDescriptor;
+import org.apache.geode.Statistics;
+import org.apache.geode.StatisticsType;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 
 public class FetchResourceAttributesResponse extends AdminResponse {
 
-  // instance variables
   private RemoteStat[] stats;
 
-  public static FetchResourceAttributesResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, long rsrcUniqueId) {
+  public static FetchResourceAttributesResponse create(DM dm, InternalDistributedMember recipient,
+      long rsrcUniqueId) {
     FetchResourceAttributesResponse m = new FetchResourceAttributesResponse();
     m.setRecipient(recipient);
     Statistics s = null;
@@ -53,9 +56,11 @@ public class FetchResourceAttributesResponse extends AdminResponse {
   }
 
   /**
-   * Constructor required by <code>DataSerializable</code>
+   * Constructor required by {@code DataSerializable}
    */
-  public FetchResourceAttributesResponse() {}
+  public FetchResourceAttributesResponse() {
+    // nothing
+  }
 
   public int getDSFID() {
     return FETCH_RESOURCE_ATTRIBUTES_RESPONSE;
@@ -75,6 +80,6 @@ public class FetchResourceAttributesResponse extends AdminResponse {
 
   @Override
   public String toString() {
-    return "FetchResourceAttributesResponse from " + this.getRecipient();
+    return "FetchResourceAttributesResponse from " + getRecipient();
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchStatsRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchStatsRequest.java
index dc98cae..8cd6d1c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchStatsRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchStatsRequest.java
@@ -40,7 +40,7 @@ public class FetchStatsRequest extends AdminRequest {
   }
 
   @Override
-  public AdminResponse createResponse(DistributionManager dm) {
+  public AdminResponse createResponse(DM dm) {
     return FetchStatsResponse.create(dm, this.getSender(), this.statisticsTypeName);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchStatsResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchStatsResponse.java
index f9a861c..665f6c4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchStatsResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchStatsResponse.java
@@ -18,23 +18,19 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.ConcurrentModificationException;
-import java.util.Iterator;
 import java.util.List;
 
 import org.apache.geode.DataSerializer;
 import org.apache.geode.Statistics;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.InternalDistributedSystem.StatisticsVisitor;
-import org.apache.geode.distributed.internal.membership.*;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 
 /**
- * Provides a response of remote statistic resources for a <code>FetchStatsRequest</code>
- *
+ * Provides a response of remote statistic resources for a {@code FetchStatsRequest}
  */
 public class FetchStatsResponse extends AdminResponse {
 
-  // instance variables
   private RemoteStatResource[] stats;
 
   /**
@@ -44,12 +40,11 @@ public class FetchStatsResponse extends AdminResponse {
    * @param recipient the recipient who made the original request
    * @return response containing all remote stat resources
    */
-  public static FetchStatsResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, final String statisticsTypeName) {
-    // LogWriterI18n log = dm.getLogger();
+  public static FetchStatsResponse create(DM dm, InternalDistributedMember recipient,
+      final String statisticsTypeName) {
     FetchStatsResponse m = new FetchStatsResponse();
     m.setRecipient(recipient);
-    final List<RemoteStatResource> statList = new ArrayList<RemoteStatResource>();
+    List<RemoteStatResource> statList = new ArrayList<RemoteStatResource>();
     // get vm-local stats
     // call visitStatistics to fix for bug 40358
     if (statisticsTypeName == null) {
@@ -72,7 +67,6 @@ public class FetchStatsResponse extends AdminResponse {
     return m;
   }
 
-
   @Override
   public boolean sendViaUDP() {
     return true;
@@ -130,7 +124,7 @@ public class FetchStatsResponse extends AdminResponse {
    */
   @Override
   public String toString() {
-    return "FetchStatsResponse from " + this.getRecipient() + " stats.length=" + stats.length;
+    return "FetchStatsResponse from " + getRecipient() + " stats.length=" + stats.length;
   }
 
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchSysCfgRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchSysCfgRequest.java
index fa1288f..33d78ff 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchSysCfgRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchSysCfgRequest.java
@@ -48,7 +48,7 @@ public class FetchSysCfgRequest extends AdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return FetchSysCfgResponse.create(dm, this.getSender());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchSysCfgResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchSysCfgResponse.java
index 57d58c1..e5405bc 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchSysCfgResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/FetchSysCfgResponse.java
@@ -34,8 +34,7 @@ public class FetchSysCfgResponse extends AdminResponse {
    * Returns a <code>FetchSysCfgResponse</code> that will be returned to the specified recipient.
    * The message will contains a copy of the local manager's config.
    */
-  public static FetchSysCfgResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static FetchSysCfgResponse create(DM dm, InternalDistributedMember recipient) {
     FetchSysCfgResponse m = new FetchSysCfgResponse();
     m.setRecipient(recipient);
     Config conf = dm.getSystem().getConfig();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/LicenseInfoRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/LicenseInfoRequest.java
index 22a5f67..11877d1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/LicenseInfoRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/LicenseInfoRequest.java
@@ -40,9 +40,11 @@ public class LicenseInfoRequest extends AdminRequest {
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return LicenseInfoResponse.create(dm, this.getSender());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/LicenseInfoResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/LicenseInfoResponse.java
index ba05780..edc63d8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/LicenseInfoResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/LicenseInfoResponse.java
@@ -22,26 +22,23 @@ import java.util.Properties;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
+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.logging.LogService;
 
-
 /**
  * A message that is sent in response to a {@link LicenseInfoRequest}.
  */
 public class LicenseInfoResponse extends AdminResponse {
   private static final Logger logger = LogService.getLogger();
 
-  // instance variables
   private Properties p;
 
-
   /**
-   * Returns a <code>LicenseInfoResponse</code> that will be returned to the specified recipient.
+   * Returns a {@code LicenseInfoResponse} that will be returned to the specified recipient.
    */
-  public static LicenseInfoResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static LicenseInfoResponse create(DM dm, InternalDistributedMember recipient) {
     LicenseInfoResponse m = new LicenseInfoResponse();
     m.setRecipient(recipient);
     m.p = new Properties();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/MissingPersistentIDsRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/MissingPersistentIDsRequest.java
index 68ac778..e273824 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/MissingPersistentIDsRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/MissingPersistentIDsRequest.java
@@ -30,7 +30,6 @@ 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.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.persistence.PersistentMemberID;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
@@ -78,7 +77,7 @@ public class MissingPersistentIDsRequest extends CliLegacyMessage {
   }
 
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     Set<PersistentID> missingIds = new HashSet<>();
     Set<PersistentID> localPatterns = new HashSet<>();
     InternalCache cache = dm.getCache();
@@ -124,12 +123,12 @@ public class MissingPersistentIDsRequest extends CliLegacyMessage {
     }
 
     @Override
-    protected void process(DistributionMessage msg, boolean warn) {
-      if (msg instanceof MissingPersistentIDsResponse) {
-        this.missing.addAll(((MissingPersistentIDsResponse) msg).getMissingIds());
-        this.existing.addAll(((MissingPersistentIDsResponse) msg).getLocalIds());
+    protected void process(DistributionMessage message, boolean warn) {
+      if (message instanceof MissingPersistentIDsResponse) {
+        this.missing.addAll(((MissingPersistentIDsResponse) message).getMissingIds());
+        this.existing.addAll(((MissingPersistentIDsResponse) message).getLocalIds());
       }
-      super.process(msg, warn);
+      super.process(message, warn);
     }
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectDetailsRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectDetailsRequest.java
index 36dba51..1dd5c26 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectDetailsRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectDetailsRequest.java
@@ -56,9 +56,11 @@ public class ObjectDetailsRequest extends RegionAdminRequest implements Cancella
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     CancellationRegistry.getInstance().registerMessage(this);
     resp = ObjectDetailsResponse.create(dm, this.getSender());
     if (cancelled) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectDetailsResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectDetailsResponse.java
index aeb33e5..490a3b1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectDetailsResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectDetailsResponse.java
@@ -38,8 +38,7 @@ public class ObjectDetailsResponse extends AdminResponse implements Cancellable
    * Returns a <code>ObjectValueResponse</code> that will be returned to the specified recipient.
    * The message will contains a copy of the local manager's system config.
    */
-  public static ObjectDetailsResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static ObjectDetailsResponse create(DM dm, InternalDistributedMember recipient) {
     ObjectDetailsResponse m = new ObjectDetailsResponse();
     m.setRecipient(recipient);
     return m;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectNamesRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectNamesRequest.java
index c51cb2b..5987340 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectNamesRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectNamesRequest.java
@@ -44,9 +44,11 @@ public class ObjectNamesRequest extends RegionAdminRequest implements Cancellabl
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     CancellationRegistry.getInstance().registerMessage(this);
     resp = ObjectNamesResponse.create(dm, this.getSender());
     if (cancelled) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectNamesResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectNamesResponse.java
index 3f02d0b..c916f0d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectNamesResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ObjectNamesResponse.java
@@ -36,8 +36,7 @@ public class ObjectNamesResponse extends AdminResponse implements Cancellable {
    * Returns a <code>ObjectNamesResponse</code> that will be returned to the specified recipient.
    * The message will contains a copy of the local manager's system config.
    */
-  public static ObjectNamesResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static ObjectNamesResponse create(DM dm, InternalDistributedMember recipient) {
     ObjectNamesResponse m = new ObjectNamesResponse();
     m.setRecipient(recipient);
     return m;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/PrepareRevokePersistentIDRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/PrepareRevokePersistentIDRequest.java
index ac9fdbc..6a84143 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/PrepareRevokePersistentIDRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/PrepareRevokePersistentIDRequest.java
@@ -27,7 +27,6 @@ import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.ReplyException;
 import org.apache.geode.internal.InternalDataSerializer;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
 import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
@@ -90,7 +89,7 @@ public class PrepareRevokePersistentIDRequest extends CliLegacyMessage {
   }
 
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     InternalCache cache = dm.getCache();
     if (cache != null && !cache.isClosed()) {
       PersistentMemberManager mm = cache.getPersistentMemberManager();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RefreshMemberSnapshotRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RefreshMemberSnapshotRequest.java
index 0de8b79..5d76bbb 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RefreshMemberSnapshotRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RefreshMemberSnapshotRequest.java
@@ -41,7 +41,7 @@ public class RefreshMemberSnapshotRequest extends AdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return RefreshMemberSnapshotResponse.create(dm, this.getSender());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RefreshMemberSnapshotResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RefreshMemberSnapshotResponse.java
index 9e973f9..bf0bd6c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RefreshMemberSnapshotResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RefreshMemberSnapshotResponse.java
@@ -22,7 +22,7 @@ import org.apache.geode.DataSerializer;
 import org.apache.geode.admin.GemFireMemberStatus;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalCache;
 
@@ -38,8 +38,7 @@ public class RefreshMemberSnapshotResponse extends AdminResponse {
    * Returns a {@code FetchSysCfgResponse} that will be returned to the specified recipient. The
    * message will contains a copy of the local manager's config.
    */
-  public static RefreshMemberSnapshotResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static RefreshMemberSnapshotResponse create(DM dm, InternalDistributedMember recipient) {
     RefreshMemberSnapshotResponse m = new RefreshMemberSnapshotResponse();
     m.setRecipient(recipient);
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionAttributesRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionAttributesRequest.java
index 9cf6d3c..16ce17c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionAttributesRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionAttributesRequest.java
@@ -42,9 +42,11 @@ public class RegionAttributesRequest extends RegionAdminRequest {
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return RegionAttributesResponse.create(dm, this.getSender(), this.getRegion(dm.getSystem()));
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionAttributesResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionAttributesResponse.java
index f618826..abd2b77 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionAttributesResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionAttributesResponse.java
@@ -34,8 +34,8 @@ public class RegionAttributesResponse extends AdminResponse {
    * Returns a <code>RegionAttributesResponse</code> that will be returned to the specified
    * recipient. The message will contains a copy of the local manager's system config.
    */
-  public static RegionAttributesResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, Region r) {
+  public static RegionAttributesResponse create(DM dm, InternalDistributedMember recipient,
+      Region r) {
     RegionAttributesResponse m = new RegionAttributesResponse();
     m.setRecipient(recipient);
     m.attributes = new RemoteRegionAttributes(r.getAttributes());
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionRequest.java
index ebd176e..596c59c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionRequest.java
@@ -111,9 +111,11 @@ public class RegionRequest extends AdminRequest {
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     // nothing needs to be done. If we got this far then a cache must exist.
     return RegionResponse.create(dm, this.getSender(), this);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionResponse.java
index e444be8..5faefe1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionResponse.java
@@ -48,7 +48,7 @@ public class RegionResponse extends AdminResponse {
    * Returns a <code>RegionResponse</code> that will be returned to the specified recipient. The
    * message will contains a copy of the local manager's system config.
    */
-  public static RegionResponse create(DistributionManager dm, InternalDistributedMember recipient,
+  public static RegionResponse create(DM dm, InternalDistributedMember recipient,
       RegionRequest request) {
     RegionResponse m = new RegionResponse();
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSizeRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSizeRequest.java
index 73a5b50..94902a4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSizeRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSizeRequest.java
@@ -40,9 +40,11 @@ public class RegionSizeRequest extends RegionAdminRequest implements Cancellable
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     Assert.assertTrue(this.getSender() != null);
     CancellationRegistry.getInstance().registerMessage(this);
     resp = RegionSizeResponse.create(dm, this.getSender());
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSizeResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSizeResponse.java
index 69fbec4..883c1d7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSizeResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSizeResponse.java
@@ -35,8 +35,7 @@ public class RegionSizeResponse extends AdminResponse implements Cancellable {
   /**
    * Returns a <code>RegionSizeResponse</code> that will be returned to the specified recipient.
    */
-  public static RegionSizeResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static RegionSizeResponse create(DM dm, InternalDistributedMember recipient) {
     RegionSizeResponse m = new RegionSizeResponse();
     m.setRecipient(recipient);
     return m;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionStatisticsRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionStatisticsRequest.java
index 4799b91..e8cc225 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionStatisticsRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionStatisticsRequest.java
@@ -44,7 +44,7 @@ public class RegionStatisticsRequest extends RegionAdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return RegionStatisticsResponse.create(dm, this.getSender(), this.getRegion(dm.getSystem()));
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionStatisticsResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionStatisticsResponse.java
index 9197eda..4778d15 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionStatisticsResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionStatisticsResponse.java
@@ -34,8 +34,8 @@ public class RegionStatisticsResponse extends AdminResponse {
    * Returns a <code>RegionStatisticsResponse</code> that will be returned to the specified
    * recipient. The message will contains a copy of the local manager's system config.
    */
-  public static RegionStatisticsResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, Region r) {
+  public static RegionStatisticsResponse create(DM dm, InternalDistributedMember recipient,
+      Region r) {
     RegionStatisticsResponse m = new RegionStatisticsResponse();
     m.setRecipient(recipient);
     m.regionStatistics = new RemoteCacheStatistics(r.getStatistics());
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSubRegionSizeRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSubRegionSizeRequest.java
index 2306bca..13123d9 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSubRegionSizeRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSubRegionSizeRequest.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.DM;
 
 /**
  * Admin request to transfer region info for a member
@@ -38,7 +38,7 @@ public class RegionSubRegionSizeRequest extends AdminRequest implements Cancella
   }
 
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     CancellationRegistry.getInstance().registerMessage(this);
 
     resp = RegionSubRegionsSizeResponse.create(dm, this.getSender());
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSubRegionsSizeResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSubRegionsSizeResponse.java
index 5e24f63..2ee08cd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSubRegionsSizeResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RegionSubRegionsSizeResponse.java
@@ -26,7 +26,7 @@ import org.apache.geode.admin.RegionSubRegionSnapshot;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.Region;
 import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogService;
@@ -54,8 +54,7 @@ public class RegionSubRegionsSizeResponse extends AdminResponse implements Cance
    * Returns a {@code RegionSubRegionsSizeResponse} that will be returned to the specified
    * recipient. The message will contains a copy of the region snapshot
    */
-  public static RegionSubRegionsSizeResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static RegionSubRegionsSizeResponse create(DM dm, InternalDistributedMember recipient) {
     RegionSubRegionsSizeResponse m = new RegionSubRegionsSizeResponse();
     m.setRecipient(recipient);
     m.snapshot = null;
@@ -64,7 +63,7 @@ public class RegionSubRegionsSizeResponse extends AdminResponse implements Cance
     return m;
   }
 
-  void populateSnapshot(DistributionManager dm) {
+  void populateSnapshot(DM dm) {
     if (this.cancelled) {
       return;
     }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoveHealthListenerRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoveHealthListenerRequest.java
index 3a615a5..2225bde 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoveHealthListenerRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoveHealthListenerRequest.java
@@ -48,7 +48,7 @@ public class RemoveHealthListenerRequest extends AdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return RemoveHealthListenerResponse.create(dm, this.getSender(), this.id);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoveHealthListenerResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoveHealthListenerResponse.java
index eeb85b2..9c44464 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoveHealthListenerResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RemoveHealthListenerResponse.java
@@ -33,8 +33,8 @@ public class RemoveHealthListenerResponse extends AdminResponse {
    * Returns a <code>RemoveHealthListenerResponse</code> that will be returned to the specified
    * recipient.
    */
-  public static RemoveHealthListenerResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, int id) {
+  public static RemoveHealthListenerResponse create(DM dm, InternalDistributedMember recipient,
+      int id) {
     RemoveHealthListenerResponse m = new RemoveHealthListenerResponse();
     m.setRecipient(recipient);
     dm.removeHealthMonitor(recipient, id);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ResetHealthStatusRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ResetHealthStatusRequest.java
index 709f069..7a75c36 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ResetHealthStatusRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ResetHealthStatusRequest.java
@@ -46,9 +46,11 @@ public class ResetHealthStatusRequest extends AdminRequest {
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return ResetHealthStatusResponse.create(dm, this.getSender(), this.id);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ResetHealthStatusResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ResetHealthStatusResponse.java
index 0659b9c..b5a5c22 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ResetHealthStatusResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ResetHealthStatusResponse.java
@@ -33,8 +33,8 @@ public class ResetHealthStatusResponse extends AdminResponse {
    * Returns a <code>ResetHealthStatusResponse</code> that will be returned to the specified
    * recipient.
    */
-  public static ResetHealthStatusResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, int id) {
+  public static ResetHealthStatusResponse create(DM dm, InternalDistributedMember recipient,
+      int id) {
     ResetHealthStatusResponse m = new ResetHealthStatusResponse();
     m.setRecipient(recipient);
     {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RevokePersistentIDRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RevokePersistentIDRequest.java
index f425a2b..feb638e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RevokePersistentIDRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RevokePersistentIDRequest.java
@@ -26,7 +26,6 @@ import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.ReplyException;
 import org.apache.geode.internal.InternalDataSerializer;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
 import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
@@ -74,7 +73,7 @@ public class RevokePersistentIDRequest extends CliLegacyMessage {
   }
 
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     InternalCache cache = dm.getCache();
     if (cache != null && !cache.isClosed()) {
       PersistentMemberManager mm = cache.getPersistentMemberManager();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RootRegionRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RootRegionRequest.java
index f6332dd..392fe6b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RootRegionRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RootRegionRequest.java
@@ -45,7 +45,7 @@ public class RootRegionRequest extends AdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     // nothing needs to be done. If we got this far then a cache must exist.
     return RootRegionResponse.create(dm, this.getSender());
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RootRegionResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RootRegionResponse.java
index 3dad8f7..64bd67d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RootRegionResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/RootRegionResponse.java
@@ -25,8 +25,8 @@ import org.apache.geode.CancelException;
 import org.apache.geode.DataSerializer;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.Region;
+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.GemFireVM;
 import org.apache.geode.internal.cache.InternalCache;
@@ -43,8 +43,7 @@ public class RootRegionResponse extends AdminResponse {
    * Returns a {@code RootRegionResponse} that will be returned to the specified recipient. The
    * message will contains a copy of the local manager's system config.
    */
-  public static RootRegionResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static RootRegionResponse create(DM dm, InternalDistributedMember recipient) {
     RootRegionResponse m = new RootRegionResponse();
     try {
       InternalCache cache = (InternalCache) CacheFactory.getInstance(dm.getSystem());
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ShutdownAllRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ShutdownAllRequest.java
index b7b2c30..3023a37 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ShutdownAllRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/ShutdownAllRequest.java
@@ -87,7 +87,7 @@ public class ShutdownAllRequest extends AdminRequest {
           if (logger.isDebugEnabled()) {
             logger.debug("caught exception while processing shutdownAll locally", ex);
           }
-          response = AdminFailureResponse.create(dism, myId, ex);
+          response = AdminFailureResponse.create(myId, ex);
         }
         response.setSender(myId);
         replyProcessor.process(response);
@@ -173,7 +173,7 @@ public class ShutdownAllRequest extends AdminRequest {
   }
 
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     boolean isToShutdown = hasCache();
     if (isToShutdown) {
       boolean isSuccess = false;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/StoreSysCfgRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/StoreSysCfgRequest.java
index f05cf9d..b3c6fd3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/StoreSysCfgRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/StoreSysCfgRequest.java
@@ -50,7 +50,7 @@ public class StoreSysCfgRequest extends AdminRequest {
    * Must return a proper response to this request.
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return StoreSysCfgResponse.create(dm, this.getSender(), this.sc);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/StoreSysCfgResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/StoreSysCfgResponse.java
index 6cbb08d..bf6c41a 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/StoreSysCfgResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/StoreSysCfgResponse.java
@@ -35,8 +35,7 @@ public class StoreSysCfgResponse extends AdminResponse {
    * Returns a <code>StoreSysCfgResponse</code> that states that a given set of distribution
    * managers are known by <code>dm</code> to be started.
    */
-  public static StoreSysCfgResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, Config sc) {
+  public static StoreSysCfgResponse create(DM dm, InternalDistributedMember recipient, Config sc) {
     StoreSysCfgResponse m = new StoreSysCfgResponse();
     m.setRecipient(recipient);
     InternalDistributedSystem sys = dm.getSystem();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/SubRegionRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/SubRegionRequest.java
index 6641941..8f0e88e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/SubRegionRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/SubRegionRequest.java
@@ -42,9 +42,11 @@ public class SubRegionRequest extends RegionAdminRequest {
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return SubRegionResponse.create(dm, this.getSender(), this.getRegion(dm.getSystem()));
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/SubRegionResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/SubRegionResponse.java
index 513a1fe..e049318 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/SubRegionResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/SubRegionResponse.java
@@ -37,8 +37,7 @@ public class SubRegionResponse extends AdminResponse {
    * Returns a <code>SubRegionResponse</code> that will be returned to the specified recipient. The
    * message will contains a copy of the local manager's system config.
    */
-  public static SubRegionResponse create(DistributionManager dm,
-      InternalDistributedMember recipient, Region r) {
+  public static SubRegionResponse create(DM dm, InternalDistributedMember recipient, Region r) {
     SubRegionResponse m = new SubRegionResponse();
     m.setRecipient(recipient);
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/TailLogRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/TailLogRequest.java
index e79b99c..d911b08 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/TailLogRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/TailLogRequest.java
@@ -28,7 +28,7 @@ public class TailLogRequest extends AdminRequest {
   }
 
   @Override
-  public AdminResponse createResponse(DistributionManager dm) {
+  public AdminResponse createResponse(DM dm) {
     return TailLogResponse.create(dm, this.getSender());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/TailLogResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/TailLogResponse.java
index 137c8ba..6e7ee03 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/TailLogResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/TailLogResponse.java
@@ -21,8 +21,8 @@ import java.io.*;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.DataSerializer;
+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.i18n.LocalizedStrings;
@@ -38,8 +38,7 @@ public class TailLogResponse extends AdminResponse {
   private String tail;
   private String childTail;
 
-  public static TailLogResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static TailLogResponse create(DM dm, InternalDistributedMember recipient) {
     TailLogResponse m = new TailLogResponse();
     m.setRecipient(recipient);
     try {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/VersionInfoRequest.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/VersionInfoRequest.java
index db4fa42..25ff383 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/VersionInfoRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/VersionInfoRequest.java
@@ -42,9 +42,11 @@ public class VersionInfoRequest extends AdminRequest {
 
   /**
    * Must return a proper response to this request.
+   *
+   * @param dm
    */
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     return VersionInfoResponse.create(dm, this.getSender());
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/VersionInfoResponse.java b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/VersionInfoResponse.java
index e8a87ed..f67f8e6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/admin/remote/VersionInfoResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/admin/remote/VersionInfoResponse.java
@@ -36,8 +36,7 @@ public class VersionInfoResponse extends AdminResponse {
   /**
    * Returns a <code>VersionInfoResponse</code> that will be returned to the specified recipient.
    */
-  public static VersionInfoResponse create(DistributionManager dm,
-      InternalDistributedMember recipient) {
+  public static VersionInfoResponse create(DM dm, InternalDistributedMember recipient) {
     VersionInfoResponse m = new VersionInfoResponse();
     m.setRecipient(recipient);
     m.verInfo = GemFireVersion.asString();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DirectoryHolder.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DirectoryHolder.java
index 16b3baf..335aa1a 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DirectoryHolder.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DirectoryHolder.java
@@ -91,7 +91,7 @@ public class DirectoryHolder {
     return dir;
   }
 
-  int getArrayIndex() {
+  public int getArrayIndex() {
     return this.index;
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreBackup.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreBackup.java
index 9b005ff..b4428e8 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreBackup.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreBackup.java
@@ -19,8 +19,6 @@ import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.geode.internal.cache.persistence.BackupInspector;
-
 /**
  * This class manages the state of the backup of an individual disk store. It holds the list of
  * oplogs that still need to be backed up, along with the lists of oplog files that should be
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreFactoryImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreFactoryImpl.java
index d09e70e..ce52220 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreFactoryImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreFactoryImpl.java
@@ -21,6 +21,7 @@ import org.apache.geode.GemFireIOException;
 import org.apache.geode.cache.DiskStore;
 import org.apache.geode.cache.DiskStoreFactory;
 import org.apache.geode.distributed.internal.ResourceEvent;
+import org.apache.geode.internal.cache.backup.BackupManager;
 import org.apache.geode.internal.cache.xmlcache.CacheCreation;
 import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.apache.geode.internal.cache.xmlcache.DiskStoreAttributesCreation;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
index cc66ca7..3bf13ea 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
@@ -80,6 +80,8 @@ import org.apache.geode.distributed.internal.membership.InternalDistributedMembe
 import org.apache.geode.i18n.StringId;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.ExportDiskRegion.ExportWriter;
+import org.apache.geode.internal.cache.backup.BackupLock;
+import org.apache.geode.internal.cache.backup.BackupManager;
 import org.apache.geode.internal.cache.entries.DiskEntry;
 import org.apache.geode.internal.cache.entries.DiskEntry.Helper.ValueWrapper;
 import org.apache.geode.internal.cache.entries.DiskEntry.RecoveredEntry;
@@ -2027,7 +2029,7 @@ public class DiskStoreImpl implements DiskStore {
     return this.directories[this.infoFileDirIndex];
   }
 
-  int getInforFileDirIndex() {
+  public int getInforFileDirIndex() {
     return this.infoFileDirIndex;
   }
 
@@ -2694,7 +2696,7 @@ public class DiskStoreImpl implements DiskStore {
   /**
    * Get all of the oplogs
    */
-  Oplog[] getAllOplogsForBackup() {
+  public Oplog[] getAllOplogsForBackup() {
     return persistentOplogs.getAllOplogs();
   }
 
@@ -2939,7 +2941,7 @@ public class DiskStoreImpl implements DiskStore {
   public static class KillCompactorException extends RuntimeException {
   }
 
-  DiskInitFile getDiskInitFile() {
+  public DiskInitFile getDiskInitFile() {
     return this.initFile;
   }
 
@@ -4325,7 +4327,7 @@ public class DiskStoreImpl implements DiskStore {
    * Wait for any current operations in the delayed write pool. Completion of this method ensures
    * that the writes have completed or the pool was shutdown
    */
-  protected void waitForDelayedWrites() {
+  public void waitForDelayedWrites() {
     Future<?> lastWriteTask = lastDelayedWrite;
     if (lastWriteTask != null) {
       try {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index e8aac76..252684e 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -134,9 +134,7 @@ import org.apache.geode.cache.query.QueryService;
 import org.apache.geode.cache.query.internal.DefaultQuery;
 import org.apache.geode.cache.query.internal.DefaultQueryService;
 import org.apache.geode.cache.query.internal.InternalQueryService;
-import org.apache.geode.cache.query.internal.MethodInvocationAuthorizer;
 import org.apache.geode.cache.query.internal.QueryMonitor;
-import org.apache.geode.cache.query.internal.RestrictedMethodInvocationAuthorizer;
 import org.apache.geode.cache.query.internal.cq.CqService;
 import org.apache.geode.cache.query.internal.cq.CqServiceProvider;
 import org.apache.geode.cache.server.CacheServer;
@@ -174,6 +172,7 @@ import org.apache.geode.i18n.LogWriterI18n;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.SystemTimer;
+import org.apache.geode.internal.cache.backup.BackupManager;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType;
 import org.apache.geode.internal.cache.control.ResourceAdvisor;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
index 58ab77d..9ec1fd0 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCache.java
@@ -52,6 +52,7 @@ import org.apache.geode.distributed.internal.DistributionAdvisor;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.SystemTimer;
+import org.apache.geode.internal.cache.backup.BackupManager;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.control.ResourceAdvisor;
 import org.apache.geode.internal.cache.event.EventTrackerExpiryTask;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java b/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java
index 9c3e52b..9b4ed10 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/Oplog.java
@@ -27,10 +27,8 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InterruptedIOException;
-import java.io.SyncFailedException;
 import java.nio.ByteBuffer;
 import java.nio.channels.ClosedChannelException;
-import java.nio.file.Files;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -55,7 +53,6 @@ import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
 import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
 import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
 import it.unimi.dsi.fastutil.objects.ObjectIterator;
-import org.apache.commons.io.FileUtils;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelException;
@@ -84,6 +81,7 @@ import org.apache.geode.internal.cache.DiskInitFile.DiskRegionFlag;
 import org.apache.geode.internal.cache.DiskStoreImpl.OplogCompactor;
 import org.apache.geode.internal.cache.DiskStoreImpl.OplogEntryIdSet;
 import org.apache.geode.internal.cache.DistributedRegion.DiskPosition;
+import org.apache.geode.internal.cache.backup.BackupManager;
 import org.apache.geode.internal.cache.entries.DiskEntry;
 import org.apache.geode.internal.cache.entries.DiskEntry.Helper.Flushable;
 import org.apache.geode.internal.cache.entries.DiskEntry.Helper.ValueWrapper;
@@ -688,6 +686,10 @@ public class Oplog implements CompactableOplog, Flushable {
     }
   }
 
+  public Object getLock() {
+    return lock;
+  }
+
   public void replaceIncompatibleEntry(DiskRegionView dr, DiskEntry old, DiskEntry repl) {
     boolean useNextOplog = false;
     // No need to get the backup lock prior to synchronizing (correct lock order) since the
@@ -1173,11 +1175,11 @@ public class Oplog implements CompactableOplog, Flushable {
     }
   }
 
-  File getCrfFile() {
+  public File getCrfFile() {
     return this.crf.f;
   }
 
-  File getDrfFile() {
+  public File getDrfFile() {
     return this.drf.f;
   }
 
@@ -1187,7 +1189,7 @@ public class Oplog implements CompactableOplog, Flushable {
    *
    * @param oplogFileNames a Set of operation log file names.
    */
-  Set<String> gatherMatchingOplogFiles(Set<String> oplogFileNames) {
+  public Set<String> gatherMatchingOplogFiles(Set<String> oplogFileNames) {
     Set<String> matchingFiles = new HashSet<>();
 
     // CRF match?
@@ -1221,7 +1223,7 @@ public class Oplog implements CompactableOplog, Flushable {
    * @return a map of baslineline oplog files to copy. May be empty if total current set for this
    *         oplog does not match the baseline.
    */
-  Map<File, File> mapBaseline(Collection<File> baselineOplogFiles) {
+  public Map<File, File> mapBaseline(Collection<File> baselineOplogFiles) {
     // Map of baseline oplog file name to oplog file
     Map<String, File> baselineOplogMap =
         TransformUtils.transformAndMap(baselineOplogFiles, TransformUtils.fileNameTransformer);
@@ -4233,7 +4235,7 @@ public class Oplog implements CompactableOplog, Flushable {
     }
   }
 
-  File getKrfFile() {
+  public File getKrfFile() {
     return new File(this.diskFile.getPath() + KRF_FILE_EXT);
   }
 
@@ -5822,7 +5824,7 @@ public class Oplog implements CompactableOplog, Flushable {
     return this.crf.channel;
   }
 
-  DirectoryHolder getDirectoryHolder() {
+  public DirectoryHolder getDirectoryHolder() {
     return this.dirHolder;
   }
 
@@ -7751,7 +7753,7 @@ public class Oplog implements CompactableOplog, Flushable {
     }
   }
 
-  void finishKrf() {
+  public void finishKrf() {
     createKrf(false);
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
index a4cf98a..33d03ba 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionDataStore.java
@@ -77,6 +77,7 @@ import org.apache.geode.internal.cache.BucketRegion.RawValue;
 import org.apache.geode.internal.cache.LocalRegion.RegionPerfStats;
 import org.apache.geode.internal.cache.PartitionedRegion.BucketLock;
 import org.apache.geode.internal.cache.PartitionedRegion.SizeEntry;
+import org.apache.geode.internal.cache.backup.BackupManager;
 import org.apache.geode.internal.cache.execute.BucketMovedException;
 import org.apache.geode.internal.cache.execute.FunctionStats;
 import org.apache.geode.internal.cache.execute.PartitionedRegionFunctionResultSender;
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/BackupDataStoreHelper.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupDataStoreHelper.java
similarity index 70%
rename from geode-core/src/main/java/org/apache/geode/admin/internal/BackupDataStoreHelper.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupDataStoreHelper.java
index 551aaa1..d5d5386 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/BackupDataStoreHelper.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupDataStoreHelper.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.admin.internal;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.File;
 import java.util.Map;
@@ -26,25 +26,28 @@ import org.apache.geode.internal.Assert;
 
 public class BackupDataStoreHelper {
 
-  public static String LOCK_SERVICE_NAME = BackupDataStoreHelper.class.getSimpleName();
-
-  private static String LOCK_NAME = LOCK_SERVICE_NAME + "_token";
-
-  private static Object LOCK_SYNC = new Object();
+  public static final String LOCK_SERVICE_NAME = BackupDataStoreHelper.class.getSimpleName();
+  private static final String LOCK_NAME = LOCK_SERVICE_NAME + "_token";
+  private static final Object LOCK_SYNC = new Object();
+  private static final FlushToDiskFactory flushToDiskFactory = new FlushToDiskFactory();
+  private static final PrepareBackupFactory prepareBackupFactory = new PrepareBackupFactory();
+  private static final FinishBackupFactory finishBackupFactory = new FinishBackupFactory();
 
   @SuppressWarnings("rawtypes")
   public static BackupDataStoreResult backupAllMembers(DM dm, Set recipients, File targetDir,
       File baselineDir) {
-    FlushToDiskRequest.send(dm, recipients);
+    new FlushToDiskOperation(dm, dm.getId(), dm.getCache(), recipients, flushToDiskFactory).send();
 
     boolean abort = true;
     Map<DistributedMember, Set<PersistentID>> successfulMembers;
     Map<DistributedMember, Set<PersistentID>> existingDataStores;
     try {
-      existingDataStores = PrepareBackupRequest.send(dm, recipients);
+      existingDataStores = new PrepareBackupOperation(dm, dm.getId(), dm.getCache(), recipients,
+          prepareBackupFactory).send();
       abort = false;
     } finally {
-      successfulMembers = FinishBackupRequest.send(dm, recipients, targetDir, baselineDir, abort);
+      successfulMembers = new FinishBackupOperation(dm, dm.getId(), dm.getCache(), recipients,
+          targetDir, baselineDir, abort, finishBackupFactory).send();
     }
     return new BackupDataStoreResult(existingDataStores, successfulMembers);
   }
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/BackupDataStoreResult.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupDataStoreResult.java
similarity index 97%
rename from geode-core/src/main/java/org/apache/geode/admin/internal/BackupDataStoreResult.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupDataStoreResult.java
index eae674b..294c71a 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/BackupDataStoreResult.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupDataStoreResult.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.admin.internal;
+package org.apache.geode.internal.cache.backup;
 
 import java.util.Map;
 import java.util.Set;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/BackupInspector.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupInspector.java
similarity index 99%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/persistence/BackupInspector.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupInspector.java
index 8c707f8..80d9673 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/BackupInspector.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupInspector.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.BufferedReader;
 import java.io.File;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/BackupLock.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupLock.java
similarity index 76%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/BackupLock.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupLock.java
index 1c130bf..29d6976 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/BackupLock.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupLock.java
@@ -12,8 +12,9 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.backup;
 
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 
@@ -21,29 +22,29 @@ import java.util.concurrent.locks.ReentrantLock;
  * A lock used for the backup process. This is a reentrant lock that provides a "backup" mode, where
  * the lock is held by a "backup thread" which can be assigned later than the time we lock.
  *
- * We need this because our backup process is two phase. In the first phase we aquire the lock and
+ * <p>
+ * We need this because our backup process is two phase. In the first phase we acquire the lock and
  * in the second phase we actually do the backup. During the second phase we need to reenter the
  * lock and release it with a different thread.
- *
  */
 public class BackupLock extends ReentrantLock {
 
-  private final ThreadLocal<Boolean> isBackupThread = new ThreadLocal<Boolean>();
-  boolean isBackingUp;
-  Condition backupDone = super.newCondition();
+  private final ThreadLocal<Boolean> isBackupThread = new ThreadLocal<>();
+  private boolean isBackingUp;
+  private Condition backupDone = super.newCondition();
 
   // test hook
-  private BackupLockTestHook hook = null;
+  private final AtomicReference<BackupLockTestHook> hook = new AtomicReference<>();
 
   public interface BackupLockTestHook {
     /**
      * Test hook called before the wait for backup to complete
      */
-    public void beforeWaitForBackupCompletion();
+    void beforeWaitForBackupCompletion();
   }
 
   public void setBackupLockTestHook(BackupLockTestHook testHook) {
-    hook = testHook;
+    hook.set(testHook);
   }
 
   public void lockForBackup() {
@@ -52,23 +53,37 @@ public class BackupLock extends ReentrantLock {
     super.unlock();
   }
 
-  public void setBackupThread() {
+  void setBackupThread() {
     isBackupThread.set(true);
   }
 
   public void unlockForBackup() {
     super.lock();
     isBackingUp = false;
-    isBackupThread.set(false);
+    isBackupThread.remove();
     backupDone.signalAll();
     super.unlock();
   }
 
-  public boolean isCurrentThreadDoingBackup() {
+  boolean isCurrentThreadDoingBackup() {
     Boolean result = isBackupThread.get();
     return (result != null) && result;
   }
 
+  /**
+   * For testing only
+   */
+  boolean isBackingUp() {
+    return isBackingUp;
+  }
+
+  /**
+   * For testing only
+   */
+  boolean hasThreadLocal() {
+    return isBackupThread.get() != null;
+  }
+
   @Override
   public void unlock() {
     // The backup thread does not need to unlock this lock since it never gets the lock. It is the
@@ -80,7 +95,6 @@ public class BackupLock extends ReentrantLock {
 
   /**
    * Acquire this lock, waiting for a backup to finish the first phase.
-   *
    */
   @Override
   public void lock() {
@@ -89,8 +103,9 @@ public class BackupLock extends ReentrantLock {
     if (!isCurrentThreadDoingBackup()) {
       super.lock();
       while (isBackingUp) {
-        if (hook != null) {
-          hook.beforeWaitForBackupCompletion();
+        BackupLockTestHook testHook = hook.get();
+        if (testHook != null) {
+          testHook.beforeWaitForBackupCompletion();
         }
         backupDone.awaitUninterruptibly();
       }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/BackupManager.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupManager.java
similarity index 96%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/BackupManager.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupManager.java
index 70a4345..f773b21 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/BackupManager.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupManager.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -42,8 +42,12 @@ import org.apache.geode.distributed.internal.membership.InternalDistributedMembe
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.DeployedJar;
 import org.apache.geode.internal.JarDeployer;
-import org.apache.geode.internal.cache.persistence.BackupInspector;
-import org.apache.geode.internal.cache.persistence.RestoreScript;
+import org.apache.geode.internal.cache.DirectoryHolder;
+import org.apache.geode.internal.cache.DiskStoreBackup;
+import org.apache.geode.internal.cache.DiskStoreImpl;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.Oplog;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 
@@ -51,7 +55,7 @@ import org.apache.geode.internal.logging.LogService;
  * This class manages the state an logic to backup a single cache.
  */
 public class BackupManager implements MembershipListener {
-  private static final Logger logger = LogService.getLogger(BackupManager.class);
+  private static final Logger logger = LogService.getLogger();
 
   static final String INCOMPLETE_BACKUP_FILE = "INCOMPLETE_BACKUP_FILE";
 
@@ -311,7 +315,7 @@ public class BackupManager implements MembershipListener {
         }
 
         // Get an appropriate lock object for each set of oplogs.
-        Object childLock = childOplog.lock;
+        Object childLock = childOplog.getLock();
 
         // TODO - We really should move this lock into the disk store, but
         // until then we need to do this magic to make sure we're actually
@@ -354,18 +358,16 @@ public class BackupManager implements MembershipListener {
           diskStore.getPersistentOplogSet().forceRoll(null);
 
           if (logger.isDebugEnabled()) {
-            logger.debug("done snaphotting for disk store {}", diskStore.getName());
+            logger.debug("done backing up disk store {}", diskStore.getName());
           }
           break;
         }
       }
       done = true;
     } finally {
-      if (!done) {
-        if (backup != null) {
-          backupByDiskStore.remove(diskStore);
-          backup.cleanup();
-        }
+      if (!done && backup != null) {
+        backupByDiskStore.remove(diskStore);
+        backup.cleanup();
       }
     }
     return backup;
@@ -583,13 +585,14 @@ public class BackupManager implements MembershipListener {
   }
 
   private void backupFile(File targetDir, File file) throws IOException {
-    if (file != null && file.exists())
+    if (file != null && file.exists()) {
       try {
         Files.createLink(targetDir.toPath().resolve(file.getName()), file.toPath());
       } catch (IOException | UnsupportedOperationException e) {
         logger.warn("Unable to create hard link for + {}. Reverting to file copy", targetDir);
         FileUtils.copyFileToDirectory(file, targetDir);
       }
+    }
   }
 
   private String cleanSpecialCharacters(String string) {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupOperation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupOperation.java
new file mode 100644
index 0000000..dc5f77b
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupOperation.java
@@ -0,0 +1,85 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.CancelException;
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.DistributedMember;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionMessage;
+import org.apache.geode.distributed.internal.ReplyException;
+import org.apache.geode.distributed.internal.ReplyProcessor21;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.logging.LogService;
+
+abstract class BackupOperation implements BackupResultCollector {
+  private static final Logger logger = LogService.getLogger();
+
+  private final DM dm;
+  private final Map<DistributedMember, Set<PersistentID>> results =
+      Collections.synchronizedMap(new HashMap<DistributedMember, Set<PersistentID>>());
+
+  protected BackupOperation(DM dm) {
+    this.dm = dm;
+  }
+
+  abstract ReplyProcessor21 createReplyProcessor();
+
+  abstract DistributionMessage createDistributionMessage(ReplyProcessor21 replyProcessor);
+
+  abstract void processLocally();
+
+  Map<DistributedMember, Set<PersistentID>> send() {
+    ReplyProcessor21 replyProcessor = createReplyProcessor();
+
+    dm.putOutgoing(createDistributionMessage(replyProcessor));
+
+    processLocally();
+
+    try {
+      replyProcessor.waitForReplies();
+    } catch (ReplyException e) {
+      if (!(e.getCause() instanceof CancelException)) {
+        throw e;
+      }
+    } catch (InterruptedException e) {
+      logger.warn(e.getMessage(), e);
+    }
+
+    return getResults();
+  }
+
+  @Override
+  public void addToResults(InternalDistributedMember member, Set<PersistentID> persistentIds) {
+    if (persistentIds != null && !persistentIds.isEmpty()) {
+      results.put(member, persistentIds);
+    }
+  }
+
+  Map<DistributedMember, Set<PersistentID>> getResults() {
+    return this.results;
+  }
+
+  protected DM getDistributionManager() {
+    return this.dm;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupReplyProcessor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupReplyProcessor.java
new file mode 100644
index 0000000..c42bfcc
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupReplyProcessor.java
@@ -0,0 +1,59 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.util.Set;
+
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionMessage;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.admin.remote.AdminMultipleReplyProcessor;
+
+class BackupReplyProcessor extends AdminMultipleReplyProcessor {
+
+  private final BackupResultCollector resultCollector;
+
+  BackupReplyProcessor(BackupResultCollector resultCollector, DM dm,
+      Set<InternalDistributedMember> recipients) {
+    super(dm, recipients);
+    this.resultCollector = resultCollector;
+  }
+
+  @Override
+  protected boolean stopBecauseOfExceptions() {
+    return false;
+  }
+
+  @Override
+  protected int getAckWaitThreshold() {
+    // Disable the 15 second warning if the backup is taking a long time
+    return 0;
+  }
+
+  @Override
+  public long getAckSevereAlertThresholdMS() {
+    // Don't log severe alerts for backups either
+    return Long.MAX_VALUE;
+  }
+
+  @Override
+  protected void process(DistributionMessage message, boolean warn) {
+    if (message instanceof BackupResponse) {
+      BackupResponse response = (BackupResponse) message;
+      resultCollector.addToResults(response.getSender(), response.getPersistentIds());
+    }
+    super.process(message, warn);
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupResponse.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupResponse.java
similarity index 75%
rename from geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupResponse.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupResponse.java
index 745e3ed..9f9243e 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/PrepareBackupResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupResponse.java
@@ -12,12 +12,13 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.admin.internal;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.HashSet;
+import java.util.Set;
 
 import org.apache.geode.DataSerializer;
 import org.apache.geode.cache.persistence.PersistentID;
@@ -25,25 +26,22 @@ import org.apache.geode.distributed.internal.membership.InternalDistributedMembe
 import org.apache.geode.internal.admin.remote.AdminResponse;
 
 /**
- * The response to the {@link PrepareBackupRequest}
- *
- *
+ * The response to a {@link PrepareBackupRequest} or {@link FinishBackupRequest}.
  */
-public class PrepareBackupResponse extends AdminResponse {
+public class BackupResponse extends AdminResponse {
 
   private HashSet<PersistentID> persistentIds;
 
-  public PrepareBackupResponse() {
+  public BackupResponse() {
     super();
   }
 
-  public PrepareBackupResponse(InternalDistributedMember sender,
-      HashSet<PersistentID> persistentIds) {
-    this.setRecipient(sender);
+  public BackupResponse(InternalDistributedMember sender, HashSet<PersistentID> persistentIds) {
+    setRecipient(sender);
     this.persistentIds = persistentIds;
   }
 
-  public HashSet<PersistentID> getPersistentIds() {
+  public Set<PersistentID> getPersistentIds() {
     return persistentIds;
   }
 
@@ -53,24 +51,15 @@ public class PrepareBackupResponse extends AdminResponse {
     persistentIds = DataSerializer.readHashSet(in);
   }
 
-
-
   @Override
   public void toData(DataOutput out) throws IOException {
     super.toData(out);
     DataSerializer.writeHashSet(persistentIds, out);
   }
 
-
-
   @Override
-  protected Object clone() throws CloneNotSupportedException {
-    // TODO Auto-generated method stub
-    return super.clone();
-  }
-
   public int getDSFID() {
-    return PREPARE_BACKUP_RESPONSE;
+    return BACKUP_RESPONSE;
   }
 
   @Override
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupResultCollector.java
similarity index 64%
copy from geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
copy to geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupResultCollector.java
index 3f0ba7d..d4ab322 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupResultCollector.java
@@ -12,17 +12,13 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.RegionShortcut;
+import java.util.Set;
 
-public class ReplicateBackupPrepareAndFinishMsgDUnitTest
-    extends BackupPrepareAndFinishMsgDUnitTest {
-  private static final RegionShortcut REGION_TYPE = RegionShortcut.REPLICATE_PERSISTENT;
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 
-  @Override
-  public Region<Integer, Integer> createRegion() {
-    return createRegion(REGION_TYPE);
-  }
+interface BackupResultCollector {
+  void addToResults(InternalDistributedMember member, Set<PersistentID> persistentIds);
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/BackupUtil.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupUtil.java
similarity index 94%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/BackupUtil.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupUtil.java
index e9ae635..264879d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/BackupUtil.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/BackupUtil.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.File;
 import java.text.SimpleDateFormat;
@@ -20,8 +20,6 @@ import java.util.Date;
 import java.util.Set;
 
 import org.apache.geode.admin.internal.AdminDistributedSystemImpl;
-import org.apache.geode.admin.internal.BackupDataStoreHelper;
-import org.apache.geode.admin.internal.BackupDataStoreResult;
 import org.apache.geode.cache.persistence.PersistentID;
 import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.internal.i18n.LocalizedStrings;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackup.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackup.java
new file mode 100644
index 0000000..f9d5813
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackup.java
@@ -0,0 +1,47 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.internal.cache.InternalCache;
+
+class FinishBackup {
+
+  private final InternalCache cache;
+  private final File targetDir;
+  private final File baselineDir;
+  private final boolean abort;
+
+  FinishBackup(InternalCache cache, File targetDir, File baselineDir, boolean abort) {
+    this.cache = cache;
+    this.targetDir = targetDir;
+    this.baselineDir = baselineDir;
+    this.abort = abort;
+  }
+
+  HashSet<PersistentID> run() throws IOException {
+    HashSet<PersistentID> persistentIds;
+    if (cache == null || cache.getBackupManager() == null) {
+      persistentIds = new HashSet<>();
+    } else {
+      persistentIds = cache.getBackupManager().doBackup(targetDir, baselineDir, abort);
+    }
+    return persistentIds;
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackupFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackupFactory.java
new file mode 100644
index 0000000..b4b40d9
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackupFactory.java
@@ -0,0 +1,49 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.io.File;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+
+class FinishBackupFactory {
+
+  BackupReplyProcessor createReplyProcessor(BackupResultCollector resultCollector, DM dm,
+      Set<InternalDistributedMember> recipients) {
+    return new BackupReplyProcessor(resultCollector, dm, recipients);
+  }
+
+  FinishBackupRequest createRequest(InternalDistributedMember sender,
+      Set<InternalDistributedMember> recipients, int processorId, File targetDir, File baselineDir,
+      boolean abort) {
+    return new FinishBackupRequest(sender, recipients, processorId, targetDir, baselineDir, abort,
+        this);
+  }
+
+  FinishBackup createFinishBackup(InternalCache cache, File targetDir, File baselineDir,
+      boolean abort) {
+    return new FinishBackup(cache, targetDir, baselineDir, abort);
+  }
+
+  BackupResponse createBackupResponse(InternalDistributedMember sender,
+      HashSet<PersistentID> persistentIds) {
+    return new BackupResponse(sender, persistentIds);
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackupOperation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackupOperation.java
new file mode 100644
index 0000000..468dba8
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackupOperation.java
@@ -0,0 +1,75 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionMessage;
+import org.apache.geode.distributed.internal.ReplyProcessor21;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.logging.LogService;
+
+class FinishBackupOperation extends BackupOperation {
+  private static final Logger logger = LogService.getLogger();
+
+  private final InternalDistributedMember member;
+  private final InternalCache cache;
+  private final Set<InternalDistributedMember> recipients;
+  private final File targetDir;
+  private final File baselineDir;
+  private final boolean abort;
+  private final FinishBackupFactory finishBackupFactory;
+
+  FinishBackupOperation(DM dm, InternalDistributedMember member, InternalCache cache,
+      Set<InternalDistributedMember> recipients, File targetDir, File baselineDir, boolean abort,
+      FinishBackupFactory FinishBackupFactory) {
+    super(dm);
+    this.member = member;
+    this.cache = cache;
+    this.recipients = recipients;
+    this.targetDir = targetDir;
+    this.baselineDir = baselineDir;
+    this.abort = abort;
+    this.finishBackupFactory = FinishBackupFactory;
+  }
+
+  @Override
+  ReplyProcessor21 createReplyProcessor() {
+    return finishBackupFactory.createReplyProcessor(this, getDistributionManager(), recipients);
+  }
+
+  @Override
+  DistributionMessage createDistributionMessage(ReplyProcessor21 replyProcessor) {
+    return finishBackupFactory.createRequest(member, recipients, replyProcessor.getProcessorId(),
+        targetDir, baselineDir, abort);
+  }
+
+  @Override
+  void processLocally() {
+    try {
+      addToResults(member,
+          finishBackupFactory.createFinishBackup(cache, targetDir, baselineDir, abort).run());
+    } catch (IOException e) {
+      logger.fatal("Failed to FinishBackup in " + member, e);
+    }
+  }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackupRequest.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackupRequest.java
new file mode 100644
index 0000000..32cc12a
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FinishBackupRequest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.admin.remote.AdminFailureResponse;
+import org.apache.geode.internal.admin.remote.AdminResponse;
+import org.apache.geode.internal.admin.remote.CliLegacyMessage;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.logging.log4j.LocalizedMessage;
+
+/**
+ * A request send from an admin VM to all of the peers to indicate that that should complete the
+ * backup operation.
+ */
+public class FinishBackupRequest extends CliLegacyMessage {
+  private static final Logger logger = LogService.getLogger();
+
+  private final transient FinishBackupFactory finishBackupFactory;
+
+  private File targetDir;
+  private File baselineDir;
+  private boolean abort;
+
+  public FinishBackupRequest() {
+    this.finishBackupFactory = new FinishBackupFactory();
+  }
+
+  FinishBackupRequest(InternalDistributedMember sender, Set<InternalDistributedMember> recipients,
+      int processorId, File targetDir, File baselineDir, boolean abort,
+      FinishBackupFactory finishBackupFactory) {
+    setSender(sender);
+    setRecipients(recipients);
+    this.msgId = processorId;
+    this.finishBackupFactory = finishBackupFactory;
+    this.targetDir = targetDir;
+    this.baselineDir = baselineDir;
+    this.abort = abort;
+  }
+
+  @Override
+  protected AdminResponse createResponse(DM dm) {
+    HashSet<PersistentID> persistentIds;
+    try {
+      persistentIds = finishBackupFactory
+          .createFinishBackup(dm.getCache(), this.targetDir, this.baselineDir, this.abort).run();
+    } catch (IOException e) {
+      logger.error(LocalizedMessage.create(LocalizedStrings.CliLegacyMessage_ERROR, getClass()), e);
+      return AdminFailureResponse.create(getSender(), e);
+    }
+    return finishBackupFactory.createBackupResponse(getSender(), persistentIds);
+  }
+
+  @Override
+  public int getDSFID() {
+    return FINISH_BACKUP_REQUEST;
+  }
+
+  @Override
+  public void fromData(DataInput in) throws IOException, ClassNotFoundException {
+    super.fromData(in);
+    targetDir = DataSerializer.readFile(in);
+    baselineDir = DataSerializer.readFile(in);
+    abort = DataSerializer.readBoolean(in);
+  }
+
+  @Override
+  public void toData(DataOutput out) throws IOException {
+    super.toData(out);
+    DataSerializer.writeFile(targetDir, out);
+    DataSerializer.writeFile(baselineDir, out);
+    DataSerializer.writeBoolean(abort, out);
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDisk.java
similarity index 64%
copy from geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
copy to geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDisk.java
index 3f0ba7d..8149d34 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDisk.java
@@ -12,17 +12,22 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.DiskStore;
+import org.apache.geode.internal.cache.InternalCache;
 
-public class ReplicateBackupPrepareAndFinishMsgDUnitTest
-    extends BackupPrepareAndFinishMsgDUnitTest {
-  private static final RegionShortcut REGION_TYPE = RegionShortcut.REPLICATE_PERSISTENT;
+class FlushToDisk {
 
-  @Override
-  public Region<Integer, Integer> createRegion() {
-    return createRegion(REGION_TYPE);
+  private final InternalCache cache;
+
+  FlushToDisk(InternalCache cache) {
+    this.cache = cache;
+  }
+
+  void run() {
+    if (cache != null) {
+      cache.listDiskStoresIncludingRegionOwned().forEach(DiskStore::flush);
+    }
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskFactory.java
similarity index 51%
copy from geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java
copy to geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskFactory.java
index a85a6aa..9d91997 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskFactory.java
@@ -12,32 +12,33 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.admin.internal;
+package org.apache.geode.internal.cache.backup;
 
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.admin.remote.AdminResponse;
+import org.apache.geode.internal.cache.InternalCache;
 
-/**
- * The response to the {@link FlushToDiskRequest}
- *
- *
- */
-public class FlushToDiskResponse extends AdminResponse {
+class FlushToDiskFactory {
 
-  public FlushToDiskResponse() {
-    super();
+  FlushToDiskProcessor createReplyProcessor(DM dm, Set<InternalDistributedMember> recipients) {
+    return new FlushToDiskProcessor(dm, recipients);
   }
 
-  public FlushToDiskResponse(InternalDistributedMember sender) {
-    this.setRecipient(sender);
+  FlushToDiskRequest createRequest(InternalDistributedMember sender,
+      Set<InternalDistributedMember> recipients, int processorId) {
+    return new FlushToDiskRequest(sender, recipients, processorId, this);
   }
 
-  public int getDSFID() {
-    return FLUSH_TO_DISK_RESPONSE;
+  FlushToDisk createFlushToDisk(InternalCache cache) {
+    return new FlushToDisk(cache);
   }
 
-  @Override
-  public String toString() {
-    return getClass().getName();
+  FlushToDiskResponse createResponse(InternalDistributedMember sender) {
+    return new FlushToDiskResponse(sender);
   }
+
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskOperation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskOperation.java
new file mode 100644
index 0000000..afeda08
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskOperation.java
@@ -0,0 +1,87 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.util.Set;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.CancelException;
+import org.apache.geode.cache.DiskStore;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionMessage;
+import org.apache.geode.distributed.internal.ReplyException;
+import org.apache.geode.distributed.internal.ReplyProcessor21;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.admin.remote.AdminResponse;
+import org.apache.geode.internal.admin.remote.CliLegacyMessage;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.logging.LogService;
+
+/**
+ * A Operation to from an admin VM to all non admin members to start a backup. In the prepare phase
+ * of the backup, the members will suspend bucket destroys to make sure buckets aren't missed during
+ * the backup.
+ */
+public class FlushToDiskOperation {
+  private static final Logger logger = LogService.getLogger();
+
+  private final DM dm;
+  private final InternalDistributedMember member;
+  private final InternalCache cache;
+  private final Set<InternalDistributedMember> recipients;
+  private final FlushToDiskFactory flushToDiskFactory;
+
+  FlushToDiskOperation(DM dm, InternalDistributedMember member, InternalCache cache,
+      Set<InternalDistributedMember> recipients, FlushToDiskFactory flushToDiskFactory) {
+    this.flushToDiskFactory = flushToDiskFactory;
+    this.dm = dm;
+    this.member = member;
+    this.recipients = recipients;
+    this.cache = cache;
+  }
+
+  void send() {
+    ReplyProcessor21 replyProcessor = createReplyProcessor();
+
+    dm.putOutgoing(createDistributionMessage(replyProcessor));
+
+    processLocally();
+
+    try {
+      replyProcessor.waitForReplies();
+    } catch (ReplyException e) {
+      if (!(e.getCause() instanceof CancelException)) {
+        throw e;
+      }
+    } catch (InterruptedException e) {
+      logger.warn(e.getMessage(), e);
+    }
+  }
+
+  private ReplyProcessor21 createReplyProcessor() {
+    return this.flushToDiskFactory.createReplyProcessor(dm, recipients);
+  }
+
+  private DistributionMessage createDistributionMessage(ReplyProcessor21 replyProcessor) {
+    return this.flushToDiskFactory.createRequest(member, recipients,
+        replyProcessor.getProcessorId());
+  }
+
+  private void processLocally() {
+    flushToDiskFactory.createFlushToDisk(cache).run();
+  }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskProcessor.java
similarity index 57%
copy from geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java
copy to geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskProcessor.java
index a85a6aa..0839ccc 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskProcessor.java
@@ -12,32 +12,27 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.admin.internal;
+package org.apache.geode.internal.cache.backup;
 
-import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.admin.remote.AdminResponse;
+import java.util.Collection;
 
-/**
- * The response to the {@link FlushToDiskRequest}
- *
- *
- */
-public class FlushToDiskResponse extends AdminResponse {
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionMessage;
+import org.apache.geode.internal.admin.remote.AdminMultipleReplyProcessor;
 
-  public FlushToDiskResponse() {
-    super();
-  }
+class FlushToDiskProcessor extends AdminMultipleReplyProcessor {
 
-  public FlushToDiskResponse(InternalDistributedMember sender) {
-    this.setRecipient(sender);
+  FlushToDiskProcessor(DM dm, Collection initMembers) {
+    super(dm, initMembers);
   }
 
-  public int getDSFID() {
-    return FLUSH_TO_DISK_RESPONSE;
+  @Override
+  protected boolean stopBecauseOfExceptions() {
+    return false;
   }
 
   @Override
-  public String toString() {
-    return getClass().getName();
+  protected void process(DistributionMessage message, boolean warn) {
+    super.process(message, warn);
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskRequest.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskRequest.java
new file mode 100644
index 0000000..f421447
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskRequest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.util.Set;
+
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.admin.remote.AdminResponse;
+import org.apache.geode.internal.admin.remote.CliLegacyMessage;
+
+/**
+ * A request to from an admin VM to all non admin members to start a backup. In the prepare phase of
+ * the backup, the members will suspend bucket destroys to make sure buckets aren't missed during
+ * the backup.
+ */
+public class FlushToDiskRequest extends CliLegacyMessage {
+
+  private final FlushToDiskFactory flushToDiskFactory;
+
+  public FlushToDiskRequest() {
+    super();
+    this.flushToDiskFactory = new FlushToDiskFactory();
+  }
+
+  FlushToDiskRequest(InternalDistributedMember sender, Set<InternalDistributedMember> recipients,
+      int processorId, FlushToDiskFactory flushToDiskFactory) {
+    this.setSender(sender);
+    setRecipients(recipients);
+    this.msgId = processorId;
+    this.flushToDiskFactory = flushToDiskFactory;
+  }
+
+  @Override
+  public int getDSFID() {
+    return FLUSH_TO_DISK_REQUEST;
+  }
+
+  @Override
+  protected AdminResponse createResponse(DM dm) {
+    flushToDiskFactory.createFlushToDisk(dm.getCache()).run();
+    return flushToDiskFactory.createResponse(getSender());
+  }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskResponse.java
similarity index 96%
copy from geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java
copy to geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskResponse.java
index a85a6aa..89b1b71 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/FlushToDiskResponse.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.admin.internal;
+package org.apache.geode.internal.cache.backup;
 
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.admin.remote.AdminResponse;
diff --git a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackup.java
similarity index 55%
rename from geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackup.java
index a85a6aa..de71173 100644
--- a/geode-core/src/main/java/org/apache/geode/admin/internal/FlushToDiskResponse.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackup.java
@@ -12,32 +12,33 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.admin.internal;
+package org.apache.geode.internal.cache.backup;
 
+import java.io.IOException;
+import java.util.HashSet;
+
+import org.apache.geode.cache.persistence.PersistentID;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.admin.remote.AdminResponse;
+import org.apache.geode.internal.cache.InternalCache;
 
-/**
- * The response to the {@link FlushToDiskRequest}
- *
- *
- */
-public class FlushToDiskResponse extends AdminResponse {
+class PrepareBackup {
 
-  public FlushToDiskResponse() {
-    super();
-  }
+  private final InternalDistributedMember member;
+  private final InternalCache cache;
 
-  public FlushToDiskResponse(InternalDistributedMember sender) {
-    this.setRecipient(sender);
+  PrepareBackup(InternalDistributedMember member, InternalCache cache) {
+    this.member = member;
+    this.cache = cache;
   }
 
-  public int getDSFID() {
-    return FLUSH_TO_DISK_RESPONSE;
+  HashSet<PersistentID> run() throws IOException {
+    HashSet<PersistentID> persistentIds;
+    if (cache == null) {
+      persistentIds = new HashSet<>();
+    } else {
+      persistentIds = cache.startBackup(member).prepareForBackup();
+    }
+    return persistentIds;
   }
 
-  @Override
-  public String toString() {
-    return getClass().getName();
-  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackupFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackupFactory.java
new file mode 100644
index 0000000..eebc439
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackupFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+
+class PrepareBackupFactory {
+
+  BackupReplyProcessor createReplyProcessor(BackupResultCollector resultCollector, DM dm,
+      Set<InternalDistributedMember> recipients) {
+    return new BackupReplyProcessor(resultCollector, dm, recipients);
+  }
+
+  PrepareBackupRequest createRequest(InternalDistributedMember sender,
+      Set<InternalDistributedMember> recipients, int processorId) {
+    return new PrepareBackupRequest(sender, recipients, processorId, this);
+  }
+
+  PrepareBackup createPrepareBackup(InternalDistributedMember member, InternalCache cache) {
+    return new PrepareBackup(member, cache);
+  }
+
+  BackupResponse createBackupResponse(InternalDistributedMember sender,
+      HashSet<PersistentID> persistentIds) {
+    return new BackupResponse(sender, persistentIds);
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackupOperation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackupOperation.java
new file mode 100644
index 0000000..b9d377c
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackupOperation.java
@@ -0,0 +1,64 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionMessage;
+import org.apache.geode.distributed.internal.ReplyProcessor21;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.logging.LogService;
+
+class PrepareBackupOperation extends BackupOperation {
+  private static final Logger logger = LogService.getLogger();
+
+  private final InternalDistributedMember member;
+  private final InternalCache cache;
+  private final Set<InternalDistributedMember> recipients;
+  private final PrepareBackupFactory prepareBackupFactory;
+
+  PrepareBackupOperation(DM dm, InternalDistributedMember member, InternalCache cache,
+      Set<InternalDistributedMember> recipients, PrepareBackupFactory prepareBackupFactory) {
+    super(dm);
+    this.member = member;
+    this.cache = cache;
+    this.recipients = recipients;
+    this.prepareBackupFactory = prepareBackupFactory;
+  }
+
+  @Override
+  ReplyProcessor21 createReplyProcessor() {
+    return prepareBackupFactory.createReplyProcessor(this, getDistributionManager(), recipients);
+  }
+
+  @Override
+  DistributionMessage createDistributionMessage(ReplyProcessor21 replyProcessor) {
+    return prepareBackupFactory.createRequest(member, recipients, replyProcessor.getProcessorId());
+  }
+
+  @Override
+  void processLocally() {
+    try {
+      addToResults(member, prepareBackupFactory.createPrepareBackup(member, cache).run());
+    } catch (IOException e) {
+      logger.fatal("Failed to PrepareBackup in " + member, e);
+    }
+  }
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackupRequest.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackupRequest.java
new file mode 100644
index 0000000..ccbf1d1
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/PrepareBackupRequest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.internal.cache.backup;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.admin.remote.AdminFailureResponse;
+import org.apache.geode.internal.admin.remote.AdminResponse;
+import org.apache.geode.internal.admin.remote.CliLegacyMessage;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.logging.log4j.LocalizedMessage;
+
+/**
+ * A request to from an admin VM to all non admin members to start a backup. In the prepare phase of
+ * the backup, the members will suspend bucket destroys to make sure buckets aren't missed during
+ * the backup.
+ */
+public class PrepareBackupRequest extends CliLegacyMessage {
+  private static final Logger logger = LogService.getLogger();
+
+  private final transient PrepareBackupFactory prepareBackupFactory;
+
+  public PrepareBackupRequest() {
+    this.prepareBackupFactory = new PrepareBackupFactory();
+  }
+
+  PrepareBackupRequest(InternalDistributedMember sender, Set<InternalDistributedMember> recipients,
+      int msgId, PrepareBackupFactory prepareBackupFactory) {
+    setSender(sender);
+    setRecipients(recipients);
+    this.msgId = msgId;
+    this.prepareBackupFactory = prepareBackupFactory;
+  }
+
+  @Override
+  protected AdminResponse createResponse(DM dm) {
+    HashSet<PersistentID> persistentIds;
+    try {
+      persistentIds = prepareBackupFactory
+          .createPrepareBackup(dm.getDistributionManagerId(), dm.getCache()).run();
+    } catch (IOException e) {
+      logger.error(LocalizedMessage.create(LocalizedStrings.CliLegacyMessage_ERROR, getClass()), e);
+      return AdminFailureResponse.create(getSender(), e);
+    }
+    return prepareBackupFactory.createBackupResponse(getSender(), persistentIds);
+  }
+
+  @Override
+  public int getDSFID() {
+    return PREPARE_BACKUP_REQUEST;
+  }
+
+}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/RestoreScript.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/RestoreScript.java
similarity index 99%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/persistence/RestoreScript.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/RestoreScript.java
index a3bd455..cd9d160 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/RestoreScript.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/RestoreScript.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.BufferedWriter;
 import java.io.File;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/ScriptGenerator.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/ScriptGenerator.java
similarity index 96%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/persistence/ScriptGenerator.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/ScriptGenerator.java
index 286caa1..7804d13 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/ScriptGenerator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/ScriptGenerator.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.BufferedWriter;
 import java.io.File;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/UnixBackupInspector.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/UnixBackupInspector.java
similarity index 97%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/persistence/UnixBackupInspector.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/UnixBackupInspector.java
index 2e0921b..cc14cb4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/UnixBackupInspector.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/UnixBackupInspector.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.BufferedReader;
 import java.io.File;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/UnixScriptGenerator.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/UnixScriptGenerator.java
similarity index 93%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/persistence/UnixScriptGenerator.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/UnixScriptGenerator.java
index a7969e1..ee9a53d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/UnixScriptGenerator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/UnixScriptGenerator.java
@@ -12,12 +12,15 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.IOException;
 
+import org.apache.geode.internal.cache.backup.RestoreScript;
+import org.apache.geode.internal.cache.backup.ScriptGenerator;
+
 class UnixScriptGenerator implements ScriptGenerator {
 
   private static final String SCRIPT_FILE_NAME = "restore.sh";
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/WindowsBackupInspector.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/WindowsBackupInspector.java
similarity index 98%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/persistence/WindowsBackupInspector.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/WindowsBackupInspector.java
index 13d4959..2d5adf7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/WindowsBackupInspector.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/WindowsBackupInspector.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.BufferedReader;
 import java.io.File;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/WindowsScriptGenerator.java b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/WindowsScriptGenerator.java
similarity index 97%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/persistence/WindowsScriptGenerator.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/backup/WindowsScriptGenerator.java
index 5327ae0..7f35691 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/WindowsScriptGenerator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/backup/WindowsScriptGenerator.java
@@ -12,12 +12,11 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.IOException;
-import java.io.PrintWriter;
 
 class WindowsScriptGenerator implements ScriptGenerator {
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/PersistentMemberManager.java b/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/PersistentMemberManager.java
index e0f6173..34b1ecb 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/PersistentMemberManager.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/persistence/PersistentMemberManager.java
@@ -24,7 +24,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.cache.persistence.RevokedPersistentDataException;
-import org.apache.geode.distributed.internal.DistributionManager;
+import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.MembershipListener;
 import org.apache.geode.distributed.internal.ProfileListener;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
@@ -164,7 +164,7 @@ public class PersistentMemberManager {
    * @return true if this member is not currently running the chosen disk store. false if the revoke
    *         should be aborted because the disk store is already running.
    */
-  public boolean prepareRevoke(PersistentMemberPattern pattern, DistributionManager dm,
+  public boolean prepareRevoke(PersistentMemberPattern pattern, DM dm,
       InternalDistributedMember sender) {
     if (logger.isDebugEnabled()) {
       logger.debug("Preparing revoke if pattern {}", pattern);
@@ -232,10 +232,10 @@ public class PersistentMemberManager {
   public class PendingRevokeListener implements MembershipListener {
     InternalDistributedMember sender;
     private PersistentMemberPattern pattern;
-    private DistributionManager dm;
+    private DM dm;
 
     public PendingRevokeListener(PersistentMemberPattern pattern, InternalDistributedMember sender,
-        DistributionManager dm) {
+        DM dm) {
       this.dm = dm;
       this.pattern = pattern;
       this.sender = sender;
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
index 2319776..e9267ad 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/xmlcache/CacheCreation.java
@@ -103,7 +103,6 @@ import org.apache.geode.distributed.internal.membership.InternalDistributedMembe
 import org.apache.geode.i18n.LogWriterI18n;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.SystemTimer;
-import org.apache.geode.internal.cache.BackupManager;
 import org.apache.geode.internal.cache.CacheConfig;
 import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.CacheServerImpl;
@@ -128,6 +127,7 @@ import org.apache.geode.internal.cache.RegionListener;
 import org.apache.geode.internal.cache.TXEntryStateFactory;
 import org.apache.geode.internal.cache.TXManagerImpl;
 import org.apache.geode.internal.cache.TombstoneService;
+import org.apache.geode.internal.cache.backup.BackupManager;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.control.ResourceAdvisor;
 import org.apache.geode.internal.cache.event.EventTrackerExpiryTask;
diff --git a/geode-core/src/main/java/org/apache/geode/management/BackupStatus.java b/geode-core/src/main/java/org/apache/geode/management/BackupStatus.java
index 9ac5b50..fb78607 100644
--- a/geode-core/src/main/java/org/apache/geode/management/BackupStatus.java
+++ b/geode-core/src/main/java/org/apache/geode/management/BackupStatus.java
@@ -19,14 +19,11 @@ import java.util.Set;
 
 import org.apache.geode.cache.persistence.PersistentID;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.DM;
-import org.apache.geode.internal.cache.BackupUtil;
 
 /**
- * The status of a backup operation, returned by
- * {@link BackupUtil#backupAllMembers(DM, java.io.File,java.io.File)}.
+ * The status of a backup operation.
  *
- * @since GemFire 6.5
+ * @since Geode 1.4
  */
 public interface BackupStatus {
 
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/DistributedSystemBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/DistributedSystemBridge.java
index 1bf5215..7cd6665 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/DistributedSystemBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/DistributedSystemBridge.java
@@ -44,8 +44,6 @@ import javax.management.ObjectName;
 import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.Logger;
 
-import org.apache.geode.admin.internal.BackupDataStoreHelper;
-import org.apache.geode.admin.internal.BackupDataStoreResult;
 import org.apache.geode.cache.persistence.PersistentID;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.DM;
@@ -58,6 +56,8 @@ import org.apache.geode.internal.admin.remote.RevokePersistentIDRequest;
 import org.apache.geode.internal.admin.remote.ShutdownAllRequest;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.backup.BackupDataStoreHelper;
+import org.apache.geode.internal.cache.backup.BackupDataStoreResult;
 import org.apache.geode.internal.cache.persistence.PersistentMemberPattern;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
index 0afdfd1..2a6565a 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/beans/MemberMBeanBridge.java
@@ -63,7 +63,6 @@ import org.apache.geode.distributed.internal.locks.DLockStats;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.internal.PureJavaMode;
-import org.apache.geode.internal.cache.BackupManager;
 import org.apache.geode.internal.cache.CachePerfStats;
 import org.apache.geode.internal.cache.DirectoryHolder;
 import org.apache.geode.internal.cache.DiskDirectoryStats;
@@ -74,6 +73,7 @@ import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PartitionedRegionStats;
+import org.apache.geode.internal.cache.backup.BackupManager;
 import org.apache.geode.internal.cache.control.ResourceManagerStats;
 import org.apache.geode.internal.cache.execute.FunctionServiceStats;
 import org.apache.geode.internal.cache.lru.LRUStatistics;
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/BackupDiskStoreCommand.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/BackupDiskStoreCommand.java
index 26f7d8b..8eb9bf8 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/BackupDiskStoreCommand.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/BackupDiskStoreCommand.java
@@ -25,8 +25,8 @@ import org.springframework.shell.core.annotation.CliOption;
 import org.apache.geode.cache.persistence.PersistentID;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.DM;
-import org.apache.geode.internal.cache.BackupUtil;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.backup.BackupUtil;
 import org.apache.geode.management.BackupStatus;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.Result;
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/messages/CompactRequest.java b/geode-core/src/main/java/org/apache/geode/management/internal/messages/CompactRequest.java
index 2b22d27..72cb1d8 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/messages/CompactRequest.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/messages/CompactRequest.java
@@ -95,7 +95,7 @@ public class CompactRequest extends AdminRequest {
   }
 
   @Override
-  protected AdminResponse createResponse(DistributionManager dm) {
+  protected AdminResponse createResponse(DM dm) {
     PersistentID compactedDiskStore = compactDiskStore(this.diskStoreName);
 
     return new CompactResponse(this.getSender(), compactedDiskStore);
@@ -159,14 +159,14 @@ public class CompactRequest extends AdminRequest {
     }
 
     @Override
-    protected void process(DistributionMessage msg, boolean warn) {
-      if (msg instanceof CompactResponse) {
-        final PersistentID persistentId = ((CompactResponse) msg).getPersistentId();
+    protected void process(DistributionMessage message, boolean warn) {
+      if (message instanceof CompactResponse) {
+        final PersistentID persistentId = ((CompactResponse) message).getPersistentId();
         if (persistentId != null) {
-          results.put(msg.getSender(), persistentId);
+          results.put(message.getSender(), persistentId);
         }
       }
-      super.process(msg, warn);
+      super.process(message, warn);
     }
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/BackupDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupDistributedTest.java
similarity index 59%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/BackupDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupDistributedTest.java
index 9c45e2d..ec869b6 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/BackupDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupDistributedTest.java
@@ -12,8 +12,13 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.backup;
 
+import static org.apache.geode.test.dunit.Host.getHost;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.apache.geode.test.dunit.Invoke.invokeInEveryVM;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -25,25 +30,26 @@ import java.io.FileNotFoundException;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.PrintStream;
-import java.io.Serializable;
+import java.io.UncheckedIOException;
 import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.TimeUnit;
+import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
 
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
 import org.apache.commons.io.FileUtils;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
 
-import org.apache.geode.admin.internal.PrepareBackupRequest;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.DiskStore;
@@ -61,46 +67,62 @@ import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.DistributionMessageObserver;
 import org.apache.geode.distributed.internal.ReplyMessage;
 import org.apache.geode.internal.admin.remote.AdminFailureResponse;
+import org.apache.geode.internal.cache.DestroyRegionOperation.DestroyRegionMessage;
+import org.apache.geode.internal.cache.DiskRegion;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.partitioned.PersistentPartitionedRegionTestBase;
 import org.apache.geode.management.BackupStatus;
-import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.DUnitEnv;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.IgnoredException;
-import org.apache.geode.test.dunit.Invoke;
 import org.apache.geode.test.dunit.SerializableCallable;
 import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
+/**
+ * Additional tests to consider adding:
+ * <ul>
+ * <li>Test default disk store.
+ * <li>Test backing up and recovering while a bucket move is in progress.
+ * <li>Test backing up and recovering while ops are in progress?
+ * </ul>
+ */
 @Category(DistributedTest.class)
-public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
-  Logger logger = LogManager.getLogger(BackupDUnitTest.class);
+@RunWith(JUnitParamsRunner.class)
+@SuppressWarnings("serial")
+public class BackupDistributedTest extends PersistentPartitionedRegionTestBase {
 
-  private static final long MAX_WAIT_SECONDS = 30;
   private VM vm0;
   private VM vm1;
+  private VM vm2;
+  private VM vm3;
+
+  @Before
+  public void setUp() throws Exception {
+    vm0 = getHost(0).getVM(0);
+    vm1 = getHost(0).getVM(1);
+    vm2 = getHost(0).getVM(2);
+    vm3 = getHost(0).getVM(3);
+  }
 
   @Override
   public final void preTearDownCacheTestCase() throws Exception {
+    vm0.invoke(() -> {
+      DistributionMessageObserver.setInstance(null);
+      disconnectFromDS();
+    });
+
     StringBuilder failures = new StringBuilder();
     delete(getBackupDir(), failures);
     if (failures.length() > 0) {
-      logger.error(failures.toString());
+      // logger.error(failures.toString());
     }
   }
 
   @Test
-  public void testBackupPR() throws Throwable {
-    Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-    vm1 = host.getVM(1);
-    VM vm2 = host.getVM(2);
-
-    logger.info("Creating region in VM0");
+  public void testBackupPR() throws Exception {
     createPersistentRegion(vm0);
-    logger.info("Creating region in VM1");
     createPersistentRegion(vm1);
 
     long lm0 = setBackupFiles(vm0);
@@ -110,11 +132,12 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     createData(vm0, 0, 5, "B", "region2");
 
     BackupStatus status = backup(vm2);
-    assertEquals(2, status.getBackedUpDiskStores().size());
-    assertEquals(Collections.emptySet(), status.getOfflineDiskStores());
+    assertThat(status.getBackedUpDiskStores()).hasSize(2);
+    assertThat(status.getOfflineDiskStores()).isEmpty();
 
     Collection<File> files = FileUtils.listFiles(getBackupDir(), new String[] {"txt"}, true);
-    assertEquals(4, files.size());
+    assertThat(files).hasSize(4);
+
     deleteOldUserUserFile(vm0);
     deleteOldUserUserFile(vm1);
     validateBackupComplete();
@@ -122,22 +145,14 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     createData(vm0, 0, 5, "C", "region1");
     createData(vm0, 0, 5, "C", "region2");
 
-    assertEquals(2, status.getBackedUpDiskStores().size());
-    assertEquals(Collections.emptySet(), status.getOfflineDiskStores());
+    assertThat(status.getBackedUpDiskStores()).hasSize(2);
+    assertThat(status.getOfflineDiskStores()).isEmpty();
 
     closeCache(vm0);
     closeCache(vm1);
 
-    // Destroy the current data
-    Invoke.invokeInEveryVM(new SerializableRunnable("Clean disk dirs") {
-      public void run() {
-        try {
-          cleanDiskDirs();
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
+    // destroy the current data
+    cleanDiskDirsInEveryVM();
 
     restoreBackup(2);
 
@@ -150,44 +165,35 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
   }
 
   /**
-   * Test of bug 42419
+   * Test of bug 42419.
+   *
+   * <p>
+   * TRAC 42419: backed up disk stores map contains null key instead of member; cannot restore
+   * backup files
    */
   @Test
-  public void testBackupFromMemberWithDiskStore() throws Throwable {
-    Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-    vm1 = host.getVM(1);
-
-    logger.info("Creating region in VM0");
+  public void testBackupFromMemberWithDiskStore() throws Exception {
     createPersistentRegion(vm0);
-    logger.info("Creating region in VM1");
     createPersistentRegion(vm1);
 
     createData(vm0, 0, 5, "A", "region1");
     createData(vm0, 0, 5, "B", "region2");
 
     BackupStatus status = backup(vm1);
-    assertEquals(2, status.getBackedUpDiskStores().size());
+    assertThat(status.getBackedUpDiskStores()).hasSize(2);
+
     for (DistributedMember key : status.getBackedUpDiskStores().keySet()) {
-      assertNotNull(key);
+      assertThat(key).isNotNull();
     }
-    assertEquals(Collections.emptySet(), status.getOfflineDiskStores());
+    assertThat(status.getOfflineDiskStores()).isEmpty();
 
     validateBackupComplete();
 
     closeCache(vm0);
     closeCache(vm1);
 
-    // Destroy the current data
-    Invoke.invokeInEveryVM(new SerializableRunnable("Clean disk dirs") {
-      public void run() {
-        try {
-          cleanDiskDirs();
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
+    // destroy the current data
+    invokeInEveryVM("cleanDiskDirs", () -> cleanDiskDirs());
 
     restoreBackup(2);
 
@@ -197,34 +203,21 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     checkData(vm0, 0, 5, "B", "region2");
   }
 
-  private void createPersistentRegionsAsync() throws java.util.concurrent.ExecutionException,
-      InterruptedException, java.util.concurrent.TimeoutException {
-    logger.info("Creating region in VM0");
-    AsyncInvocation async0 = createPersistentRegionAsync(vm0);
-    logger.info("Creating region in VM1");
-    AsyncInvocation async1 = createPersistentRegionAsync(vm1);
-    async0.get(MAX_WAIT_SECONDS, TimeUnit.SECONDS);
-    async1.get(MAX_WAIT_SECONDS, TimeUnit.SECONDS);
-  }
-
   /**
    * Test for bug 42419
+   *
+   * <p>
+   * TRAC 42419: backed up disk stores map contains null key instead of member; cannot restore
+   * backup files
    */
   @Test
-  public void testBackupWhileBucketIsCreated() throws Throwable {
-    Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-    vm1 = host.getVM(1);
-    final VM vm2 = host.getVM(2);
-
-    logger.info("Creating region in VM0");
+  public void testBackupWhileBucketIsCreated() throws Exception {
     createPersistentRegion(vm0);
 
     // create a bucket on vm0
     createData(vm0, 0, 1, "A", "region1");
 
     // create the pr on vm1, which won't have any buckets
-    logger.info("Creating region in VM1");
     createPersistentRegion(vm1);
 
     CompletableFuture<BackupStatus> backupStatusFuture =
@@ -234,29 +227,21 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     CompletableFuture.allOf(backupStatusFuture, createDataFuture);
 
     BackupStatus status = backupStatusFuture.get();
-    assertEquals(2, status.getBackedUpDiskStores().size());
-    assertEquals(Collections.emptySet(), status.getOfflineDiskStores());
+    assertThat(status.getBackedUpDiskStores()).hasSize(2);
+    assertThat(status.getOfflineDiskStores()).isEmpty();
 
     validateBackupComplete();
 
     createData(vm0, 0, 5, "C", "region1");
 
-    assertEquals(2, status.getBackedUpDiskStores().size());
-    assertEquals(Collections.emptySet(), status.getOfflineDiskStores());
+    assertThat(status.getBackedUpDiskStores()).hasSize(2);
+    assertThat(status.getOfflineDiskStores()).isEmpty();
 
     closeCache(vm0);
     closeCache(vm1);
 
-    // Destroy the current data
-    Invoke.invokeInEveryVM(new SerializableRunnable("Clean disk dirs") {
-      public void run() {
-        try {
-          cleanDiskDirs();
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
+    // destroy the current data
+    invokeInEveryVM("cleanDiskDirs", () -> cleanDiskDirs());
 
     restoreBackup(2);
 
@@ -265,93 +250,82 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     checkData(vm0, 0, 1, "A", "region1");
   }
 
+  /**
+   * Test for bug 42420. Invoke a backup when a bucket is in the middle of being moved.
+   *
+   * <p>
+   * TRAC 42420: Online backup files sometimes cannot be restored
+   */
   @Test
-  public void testBackupWhileBucketIsMovedBackupBeforeSendDestroy() throws Throwable {
-    Host host = Host.getHost(0);
-    final VM vm2 = host.getVM(2);
+  @Parameters({"BEFORE_SENDING_DESTROYREGIONMESSAGE", "BEFORE_PROCESSING_REPLYMESSAGE"})
+  @TestCaseName("{method}({params})")
+  public void testWhileBucketIsMovedBackup(final WhenToInvokeBackup whenToInvokeBackup)
+      throws Exception {
+    vm0.invoke("Add listener to invoke backup", () -> {
+      disconnectFromDS();
 
-    DistributionMessageObserver observer = new SerializableDistributionMessageObserver() {
-      private volatile boolean done;
+      // This listener will wait for a response to the
+      // destroy region message, and then trigger a backup.
+      // That will backup before this member has finished destroying
+      // a bucket, but after the peer has removed the bucket.
+      DistributionMessageObserver.setInstance(createTestHookToBackup(whenToInvokeBackup));
+    });
 
-      @Override
-      public void beforeSendMessage(DistributionManager dm, DistributionMessage msg) {
+    createPersistentRegion(vm0);
 
-        // The bucket move will send a destroy region message.
-        if (msg instanceof DestroyRegionOperation.DestroyRegionMessage && !done) {
-          backup(vm2);
-          done = true;
-        }
-      }
-    };
+    // create twos bucket on vm0
+    createData(vm0, 0, 2, "A", "region1");
 
-    backupWhileBucketIsMoved(observer);
-  }
+    // create the pr on vm1, which won't have any buckets
+    createPersistentRegion(vm1);
 
-  @Test
-  public void testBackupWhileBucketIsMovedBackupAfterSendDestroy() throws Throwable {
-    Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-    vm1 = host.getVM(1);
-    final VM vm2 = host.getVM(2);
+    // Perform a rebalance. This will trigger the backup in the middle of the bucket move.
+    vm0.invoke("Do rebalance", () -> {
+      RebalanceOperation op = getCache().getResourceManager().createRebalanceFactory().start();
+      RebalanceResults results;
+      try {
+        results = op.getResults();
+        assertEquals(1, results.getTotalBucketTransfersCompleted());
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    });
 
-    DistributionMessageObserver observer = new SerializableDistributionMessageObserver() {
-      private volatile boolean done;
-      private AtomicInteger count = new AtomicInteger();
-      private volatile int replyId = -0xBAD;
+    validateBackupComplete();
 
-      @Override
-      public void beforeSendMessage(DistributionManager dm, DistributionMessage msg) {
+    createData(vm0, 0, 5, "C", "region1");
 
-        // The bucket move will send a destroy region message.
-        if (msg instanceof DestroyRegionOperation.DestroyRegionMessage && !done) {
-          this.replyId = msg.getProcessorId();
-        }
-      }
+    closeCache(vm0);
+    closeCache(vm1);
 
-      @Override
-      public void beforeProcessMessage(DistributionManager dm, DistributionMessage message) {
-        if (message instanceof ReplyMessage && replyId != -0xBAD
-            && replyId == message.getProcessorId() && !done
-        // we need two replies
-            && count.incrementAndGet() == 2) {
-          backup(vm2);
-          done = true;
-        }
+    // Destroy the current data
+    invokeInEveryVM("Clean disk dirs", () -> {
+      try {
+        cleanDiskDirs();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
       }
-    };
+    });
+
+    restoreBackup(2);
 
-    backupWhileBucketIsMoved(observer);
+    createPersistentRegionsAsync();
+
+    checkData(vm0, 0, 2, "A", "region1");
   }
 
   @Test
-  public void testBackupStatusCleanedUpAfterFailureOnOneMember() throws Throwable {
-    IgnoredException.addIgnoredException("Uncaught exception");
-    IgnoredException.addIgnoredException("Stop processing");
-    Host host = Host.getHost(0);
-    final VM vm0 = host.getVM(0);
-    VM vm1 = host.getVM(1);
-    final VM vm2 = host.getVM(2);
-
-    // Create an observer that will fail a backup
-    // When this member receives a prepare
-    DistributionMessageObserver observer = new SerializableDistributionMessageObserver() {
-      @Override
-      public void beforeProcessMessage(DistributionManager dm, DistributionMessage message) {
-        if (message instanceof PrepareBackupRequest) {
-          DistributionMessageObserver.setInstance(null);
-          IOException exception = new IOException("Backup in progess");
-          AdminFailureResponse response =
-              AdminFailureResponse.create(dm, message.getSender(), exception);
-          response.setMsgId(((PrepareBackupRequest) message).getMsgId());
-          dm.putOutgoing(response);
-          throw new RuntimeException("Stop processing");
-        }
-      }
-    };
+  public void testBackupStatusCleanedUpAfterFailureOnOneMember() throws Exception {
+    addIgnoredException("Uncaught exception");
+    addIgnoredException("Stop processing");
+
+    String exceptionMessage = "Backup in progress";
 
     vm0.invoke(() -> {
       disconnectFromDS();
-      DistributionMessageObserver.setInstance(observer);
+      // create an observer that will fail a backup when this member receives a prepare
+      DistributionMessageObserver.setInstance(
+          createTestHookToThrowIOExceptionBeforeProcessingPrepareBackupRequest(exceptionMessage));
     });
 
     createPersistentRegion(vm0);
@@ -360,149 +334,37 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     createData(vm0, 0, 5, "A", "region1");
     createData(vm0, 0, 5, "B", "region2");
 
-    try {
-      backup(vm2);
-      fail("Backup should have failed with in progress exception");
-    } catch (Exception expected) {
-      // that's ok, hte backup should have failed
-    }
+    assertThatThrownBy(() -> backup(vm2)).hasRootCauseInstanceOf(IOException.class);
 
-    // A second backup should succeed because the observer
-    // has been cleared and the backup state should be cleared.
+    // second backup should succeed because the observer and backup state has been cleared
     BackupStatus status = backup(vm2);
-    assertEquals(2, status.getBackedUpDiskStores().size());
-    assertEquals(Collections.emptySet(), status.getOfflineDiskStores());
-
-
-  }
-
-  /**
-   * Test for bug 42420. Invoke a backup when a bucket is in the middle of being moved.
-   *
-   * @param observer - a message observer that triggers at the backup at the correct time.
-   */
-  private void backupWhileBucketIsMoved(final DistributionMessageObserver observer)
-      throws Throwable {
-    Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-    vm1 = host.getVM(1);
-
-    vm0.invoke(new SerializableRunnable("Add listener to invoke backup") {
-
-      public void run() {
-        disconnectFromDS();
-
-        // This listener will wait for a response to the
-        // destroy region message, and then trigger a backup.
-        // That will backup before this member has finished destroying
-        // a bucket, but after the peer has removed the bucket.
-        DistributionMessageObserver.setInstance(observer);
-      }
-    });
-    try {
-
-      logger.info("Creating region in VM0");
-      createPersistentRegion(vm0);
-
-      // create twos bucket on vm0
-      createData(vm0, 0, 2, "A", "region1");
-
-      // create the pr on vm1, which won't have any buckets
-      logger.info("Creating region in VM1");
-
-      createPersistentRegion(vm1);
-
-      // Perform a rebalance. This will trigger the backup in the middle
-      // of the bucket move.
-      vm0.invoke(new SerializableRunnable("Do rebalance") {
-
-        public void run() {
-          Cache cache = getCache();
-          RebalanceOperation op = cache.getResourceManager().createRebalanceFactory().start();
-          RebalanceResults results;
-          try {
-            results = op.getResults();
-            assertEquals(1, results.getTotalBucketTransfersCompleted());
-          } catch (Exception e) {
-            Assert.fail("interupted", e);
-          }
-        }
-      });
-
-      validateBackupComplete();
-
-      createData(vm0, 0, 5, "C", "region1");
-
-      closeCache(vm0);
-      closeCache(vm1);
-
-      // Destroy the current data
-      Invoke.invokeInEveryVM(new SerializableRunnable("Clean disk dirs") {
-        public void run() {
-          try {
-            cleanDiskDirs();
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-        }
-      });
-
-      restoreBackup(2);
-
-      createPersistentRegionsAsync();
-
-      checkData(vm0, 0, 2, "A", "region1");
-    } finally {
-      // cleanup the distribution message observer
-      vm0.invoke(new SerializableRunnable() {
-        public void run() {
-          DistributionMessageObserver.setInstance(null);
-          disconnectFromDS();
-        }
-      });
-    }
+    assertThat(status.getBackedUpDiskStores()).hasSize(2);
+    assertThat(status.getOfflineDiskStores()).isEmpty();
   }
 
   /**
    * Make sure we don't report members without persistent data as backed up.
    */
   @Test
-  public void testBackupOverflow() throws Throwable {
-    Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-    vm1 = host.getVM(1);
-    VM vm2 = host.getVM(2);
-
-    logger.info("Creating region in VM0");
+  public void testBackupOverflow() throws Exception {
     createPersistentRegion(vm0);
-    logger.info("Creating region in VM1");
     createOverflowRegion(vm1);
 
     createData(vm0, 0, 5, "A", "region1");
     createData(vm0, 0, 5, "B", "region2");
 
     BackupStatus status = backup(vm2);
-    assertEquals("Backed up disk stores  " + status, 1, status.getBackedUpDiskStores().size());
-    assertEquals(2, status.getBackedUpDiskStores().values().iterator().next().size());
-    assertEquals(Collections.emptySet(), status.getOfflineDiskStores());
+    assertThat(status.getBackedUpDiskStores()).hasSize(1);
+    assertThat(status.getBackedUpDiskStores().values().iterator().next()).hasSize(2);
+    assertThat(status.getOfflineDiskStores()).isEmpty();
 
     validateBackupComplete();
-
   }
 
   @Test
-  public void testBackupPRWithOfflineMembers() throws Throwable {
-    Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-    vm1 = host.getVM(1);
-    VM vm2 = host.getVM(2);
-    VM vm3 = host.getVM(3);
-
-    logger.info("Creating region in VM0");
+  public void testBackupPRWithOfflineMembers() throws Exception {
     createPersistentRegion(vm0);
-    logger.info("Creating region in VM1");
     createPersistentRegion(vm1);
-    logger.info("Creating region in VM2");
     createPersistentRegion(vm2);
 
     createData(vm0, 0, 5, "A", "region1");
@@ -511,15 +373,98 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     closeCache(vm2);
 
     BackupStatus status = backup(vm3);
-    assertEquals(2, status.getBackedUpDiskStores().size());
-    assertEquals(2, status.getOfflineDiskStores().size());
+    assertThat(status.getBackedUpDiskStores()).hasSize(2);
+    assertThat(status.getOfflineDiskStores()).hasSize(2);
+  }
+
+  private DistributionMessageObserver createTestHookToBackup(
+      WhenToInvokeBackup backupInvocationTestHook) {
+    switch (backupInvocationTestHook) {
+      case BEFORE_SENDING_DESTROYREGIONMESSAGE:
+        return createTestHookToBackupBeforeSendingDestroyRegionMessage(() -> backup(vm2));
+      case BEFORE_PROCESSING_REPLYMESSAGE:
+        return createTestHookToBackupBeforeProcessingReplyMessage(() -> backup(vm2));
+      default:
+        throw new AssertionError("Invalid backupInvocationTestHook " + backupInvocationTestHook);
+    }
+  }
+
+  private DistributionMessageObserver createTestHookToBackupBeforeProcessingReplyMessage(
+      Runnable task) {
+    return new DistributionMessageObserver() {
+      private volatile boolean done;
+      private final AtomicInteger count = new AtomicInteger();
+      private volatile int replyId = -0xBAD;
+
+      @Override
+      public void beforeSendMessage(DistributionManager dm, DistributionMessage message) {
+        // the bucket move will send a destroy region message.
+        if (message instanceof DestroyRegionMessage && !done) {
+          this.replyId = message.getProcessorId();
+        }
+      }
+
+      @Override
+      public void beforeProcessMessage(DistributionManager dm, DistributionMessage message) {
+        if (message instanceof ReplyMessage && replyId != -0xBAD
+            && replyId == message.getProcessorId() && !done && count.incrementAndGet() == 2) {
+          task.run();
+          done = true;
+        }
+      }
+    };
+  }
+
+  private DistributionMessageObserver createTestHookToBackupBeforeSendingDestroyRegionMessage(
+      Runnable task) {
+    return new DistributionMessageObserver() {
+      private volatile boolean done;
+
+      @Override
+      public void beforeSendMessage(DistributionManager dm, DistributionMessage message) {
+        // the bucket move will send a destroy region message.
+        if (message instanceof DestroyRegionMessage && !done) {
+          task.run();
+          done = true;
+        }
+      }
+    };
   }
 
-  // TODO
-  // Test default disk store.
-  // Test backing up and recovering while a bucket move is in progress.
-  // Test backing up and recovering while ops are in progress?
+  private void cleanDiskDirsInEveryVM() {
+    invokeInEveryVM("cleanDiskDirsInEveryVM", () -> {
+      try {
+        cleanDiskDirs();
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+    });
+  }
 
+  private DistributionMessageObserver createTestHookToThrowIOExceptionBeforeProcessingPrepareBackupRequest(
+      final String exceptionMessage) {
+    return new DistributionMessageObserver() {
+      @Override
+      public void beforeProcessMessage(DistributionManager dm, DistributionMessage message) {
+        if (message instanceof PrepareBackupRequest) {
+          DistributionMessageObserver.setInstance(null);
+          IOException exception = new IOException(exceptionMessage);
+          AdminFailureResponse response =
+              AdminFailureResponse.create(message.getSender(), exception);
+          response.setMsgId(((PrepareBackupRequest) message).getMsgId());
+          dm.putOutgoing(response);
+          throw new RuntimeException("Stop processing"); // TODO:KIRK: what is this?
+        }
+      }
+    };
+  }
+
+  private void createPersistentRegionsAsync() throws ExecutionException, InterruptedException {
+    AsyncInvocation async0 = createPersistentRegionAsync(vm0);
+    AsyncInvocation async1 = createPersistentRegionAsync(vm1);
+    async0.await();
+    async1.await();
+  }
 
   private void validateBackupComplete() {
     File backupDir = getBackupDir();
@@ -529,19 +474,13 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     assertTrue(files.length == 0);
   }
 
-  private void createPersistentRegion(VM vm) throws Throwable {
-    AsyncInvocation future = createPersistentRegionAsync(vm);
-    future.get(MAX_WAIT_SECONDS, TimeUnit.SECONDS);
-    if (future.isAlive()) {
-      fail("Region not created within" + MAX_WAIT_SECONDS);
-    }
-    if (future.exceptionOccurred()) {
-      throw new RuntimeException(future.getException());
-    }
+  private void createPersistentRegion(VM vm) throws Exception {
+    createPersistentRegionAsync(vm).await();
   }
 
   private void deleteOldUserUserFile(final VM vm) {
     SerializableRunnable validateUserFileBackup = new SerializableRunnable("set user backups") {
+      @Override
       public void run() {
         try {
           FileUtils.deleteDirectory(new File("userbackup_" + vm.getId()));
@@ -555,6 +494,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
 
   private long setBackupFiles(final VM vm) {
     SerializableCallable setUserBackups = new SerializableCallable("set user backups") {
+      @Override
       public Object call() {
         final int pid = DUnitEnv.get().getPid();
         File vmdir = new File("userbackup_" + pid);
@@ -587,6 +527,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
 
   private void verifyUserFileRestored(VM vm, final long lm) {
     vm.invoke(new SerializableRunnable() {
+      @Override
       public void run() {
         final int pid = DUnitEnv.get().getPid();
         File vmdir = new File("userbackup_" + pid);
@@ -615,6 +556,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
 
   private AsyncInvocation createPersistentRegionAsync(final VM vm) {
     SerializableRunnable createRegion = new SerializableRunnable("Create persistent region") {
+      @Override
       public void run() {
         Cache cache = getCache();
         DiskStoreFactory dsf = cache.createDiskStoreFactory();
@@ -644,6 +586,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
 
   private void createOverflowRegion(final VM vm) {
     SerializableRunnable createRegion = new SerializableRunnable("Create persistent region") {
+      @Override
       public void run() {
         Cache cache = getCache();
         DiskStoreFactory dsf = cache.createDiskStoreFactory();
@@ -663,14 +606,17 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     vm.invoke(createRegion);
   }
 
+  @Override
   protected void createData(VM vm, final int startKey, final int endKey, final String value) {
-    createData(vm, startKey, endKey, value, PR_REGION_NAME);
+    createData(vm, startKey, endKey, value, getPartitionedRegionName());
   }
 
+  @Override
   protected void createData(VM vm, final int startKey, final int endKey, final String value,
       final String regionName) {
     SerializableRunnable createData = new SerializableRunnable() {
 
+      @Override
       public void run() {
         Cache cache = getCache();
         Region region = cache.getRegion(regionName);
@@ -683,14 +629,17 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     vm.invoke(createData);
   }
 
-  protected void checkData(VM vm0, final int startKey, final int endKey, final String value) {
-    checkData(vm0, startKey, endKey, value, PR_REGION_NAME);
+  @Override
+  protected void checkData(VM vm, final int startKey, final int endKey, final String value) {
+    checkData(vm, startKey, endKey, value, getPartitionedRegionName());
   }
 
-  protected void checkData(VM vm0, final int startKey, final int endKey, final String value,
+  @Override
+  protected void checkData(VM vm, final int startKey, final int endKey, final String value,
       final String regionName) {
     SerializableRunnable checkData = new SerializableRunnable() {
 
+      @Override
       public void run() {
         Cache cache = getCache();
         Region region = cache.getRegion(regionName);
@@ -701,11 +650,13 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
       }
     };
 
-    vm0.invoke(checkData);
+    vm.invoke(checkData);
   }
 
+  @Override
   protected void closeCache(final VM vm) {
     SerializableRunnable closeCache = new SerializableRunnable("close cache") {
+      @Override
       public void run() {
         Cache cache = getCache();
         cache.close();
@@ -714,13 +665,16 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     vm.invoke(closeCache);
   }
 
-  protected Set<Integer> getBucketList(VM vm0) {
-    return getBucketList(vm0, PR_REGION_NAME);
+  @Override
+  protected Set<Integer> getBucketList(VM vm) {
+    return getBucketList(vm, getPartitionedRegionName());
   }
 
-  protected Set<Integer> getBucketList(VM vm0, final String regionName) {
+  @Override
+  protected Set<Integer> getBucketList(VM vm, final String regionName) {
     SerializableCallable getBuckets = new SerializableCallable("get buckets") {
 
+      @Override
       public Object call() throws Exception {
         Cache cache = getCache();
         PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionName);
@@ -728,7 +682,7 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
       }
     };
 
-    return (Set<Integer>) vm0.invoke(getBuckets);
+    return (Set<Integer>) vm.invoke(getBuckets);
   }
 
   private File[] getDiskDirs(String dsName) {
@@ -743,9 +697,22 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
     return DataPolicy.PERSISTENT_PARTITION;
   }
 
-  private static class SerializableDistributionMessageObserver extends DistributionMessageObserver
-      implements Serializable {
-
+  void checkRecoveredFromDisk(VM vm, final int bucketId, final boolean recoveredLocally) {
+    vm.invoke(new SerializableRunnable("check recovered from disk") {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        PartitionedRegion region = (PartitionedRegion) cache.getRegion(getPartitionedRegionName());
+        DiskRegion disk = region.getRegionAdvisor().getBucket(bucketId).getDiskRegion();
+        if (recoveredLocally) {
+          assertEquals(0, disk.getStats().getRemoteInitializations());
+          assertEquals(1, disk.getStats().getLocalInitializations());
+        } else {
+          assertEquals(1, disk.getStats().getRemoteInitializations());
+          assertEquals(0, disk.getStats().getLocalInitializations());
+        }
+      }
+    });
   }
 
   /**
@@ -754,8 +721,9 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
    * much as possible.
    */
   public static void delete(File file, StringBuilder failures) {
-    if (!file.exists())
+    if (!file.exists()) {
       return;
+    }
 
     if (file.isDirectory()) {
       File[] fileList = file.listFiles();
@@ -775,4 +743,8 @@ public class BackupDUnitTest extends PersistentPartitionedRegionTestBase {
       }
     }
   }
+
+  enum WhenToInvokeBackup {
+    BEFORE_SENDING_DESTROYREGIONMESSAGE, BEFORE_PROCESSING_REPLYMESSAGE
+  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupInspectorIntegrationTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupInspectorIntegrationTest.java
similarity index 99%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupInspectorIntegrationTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupInspectorIntegrationTest.java
index 704a6e1..1a2be04 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupInspectorIntegrationTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupInspectorIntegrationTest.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/BackupJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupIntegrationTest.java
similarity index 93%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/BackupJUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupIntegrationTest.java
index af85980..ee58738 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/BackupJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupIntegrationTest.java
@@ -12,13 +12,16 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.backup;
 
 import static org.apache.geode.distributed.ConfigurationProperties.CACHE_XML_FILE;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -51,10 +54,12 @@ import org.apache.geode.cache.EvictionAction;
 import org.apache.geode.cache.EvictionAttributes;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
+import org.apache.geode.internal.cache.DiskStoreImpl;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
 @Category(IntegrationTest.class)
-public class BackupJUnitTest {
+public class BackupIntegrationTest {
 
   private static final String DISK_STORE_NAME = "diskStore";
   private GemFireCacheImpl cache = null;
@@ -68,7 +73,7 @@ public class BackupJUnitTest {
   private final Random random = new Random();
 
   private String getName() {
-    return "BackupJUnitTest_" + System.identityHashCode(this);
+    return "BackupIntegrationTest_" + System.identityHashCode(this);
   }
 
   @Before
@@ -79,7 +84,7 @@ public class BackupJUnitTest {
       String tmpDirName = System.getProperty("java.io.tmpdir");
       tmpDir = new File(tmpDirName == null ? "" : tmpDirName);
       try {
-        URL url = BackupJUnitTest.class.getResource("BackupJUnitTest.cache.xml");
+        URL url = BackupIntegrationTest.class.getResource("BackupIntegrationTest.cache.xml");
         cacheXmlFile = new File(url.toURI().getPath());
       } catch (URISyntaxException e) {
         throw new ExceptionInInitializerError(e);
@@ -99,7 +104,7 @@ public class BackupJUnitTest {
     diskDirs[1].mkdir();
   }
 
-  private void createCache() throws IOException {
+  private void createCache() {
     cache = (GemFireCacheImpl) new CacheFactory(props).create();
   }
 
@@ -119,15 +124,15 @@ public class BackupJUnitTest {
   }
 
   @Test
-  public void testBackupAndRecover() throws IOException, InterruptedException {
+  public void testBackupAndRecover() throws Exception {
     backupAndRecover(() -> {
       createDiskStore();
-      return BackupJUnitTest.this.createRegion();
+      return createRegion();
     });
   }
 
   @Test
-  public void testBackupAndRecoverOldConfig() throws IOException, InterruptedException {
+  public void testBackupAndRecoverOldConfig() throws Exception {
     backupAndRecover(() -> {
       createDiskStore();
       RegionFactory regionFactory = cache.createRegionFactory();
@@ -227,7 +232,7 @@ public class BackupJUnitTest {
 
 
   @Test
-  public void testBackupEmptyDiskStore() throws IOException, InterruptedException {
+  public void testBackupEmptyDiskStore() throws Exception {
     createDiskStore();
 
     BackupManager backup =
@@ -239,7 +244,7 @@ public class BackupJUnitTest {
   }
 
   @Test
-  public void testBackupOverflowOnlyDiskStore() throws IOException, InterruptedException {
+  public void testBackupOverflowOnlyDiskStore() throws Exception {
     createDiskStore();
     Region region = createOverflowRegion();
     // Put another key to make sure we restore
@@ -258,7 +263,7 @@ public class BackupJUnitTest {
 
 
   @Test
-  public void testCompactionDuringBackup() throws IOException, InterruptedException {
+  public void testCompactionDuringBackup() throws Exception {
     DiskStoreFactory dsf = cache.createDiskStoreFactory();
     dsf.setDiskDirs(diskDirs);
     dsf.setMaxOplogSize(1);
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupLockTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupLockTest.java
new file mode 100644
index 0000000..9f1c82b
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupLockTest.java
@@ -0,0 +1,105 @@
+/*
+ * 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.internal.cache.backup;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class BackupLockTest {
+
+  private BackupLock backupLock;
+  private ExecutorService executor;
+
+  @Before
+  public void setUp() throws Exception {
+    backupLock = new BackupLock();
+    executor = Executors.newSingleThreadExecutor();
+  }
+
+  @Test
+  public void lockShouldBlockUntilLockForBackup() throws Exception {
+    backupLock.lockForBackup();
+    backupLock.setBackupThread();
+
+    AtomicBoolean beforeLock = new AtomicBoolean();
+    AtomicBoolean afterLock = new AtomicBoolean();
+
+    backupLock.setBackupLockTestHook(() -> beforeLock.set(true));
+
+    executor.submit(() -> {
+      backupLock.lock(); // beforeLock is set inside lock() method
+      afterLock.set(true);
+    });
+
+    await().atMost(10, SECONDS).until(() -> assertThat(beforeLock).isTrue());
+    assertThat(afterLock).isFalse();
+
+    backupLock.unlockForBackup();
+    await().atMost(10, SECONDS).until(() -> assertThat(afterLock).isTrue());
+  }
+
+  @Test
+  public void otherThreadShouldBeAbleToUnlockForBackup() throws Exception {
+    backupLock.lockForBackup();
+    backupLock.setBackupThread();
+
+    await().atMost(10, SECONDS).until(() -> assertThat(backupLock.isBackingUp()).isTrue());
+    assertThat(backupLock.isCurrentThreadDoingBackup()).isTrue();
+
+    executor.submit(() -> {
+      assertThat(backupLock.isCurrentThreadDoingBackup()).isFalse();
+      backupLock.unlockForBackup();
+    });
+
+    await().atMost(10, SECONDS).until(() -> assertThat(backupLock.isBackingUp()).isFalse());
+  }
+
+  @Test
+  public void isCurrentThreadDoingBackupShouldBeSetAndUnset() throws Exception {
+    backupLock.lockForBackup();
+    backupLock.setBackupThread();
+
+    assertThat(backupLock.isCurrentThreadDoingBackup()).isTrue();
+
+    backupLock.unlockForBackup();
+
+    assertThat(backupLock.isCurrentThreadDoingBackup()).isFalse();
+  }
+
+  @Test
+  public void threadLocalShouldNotLeak() throws Exception {
+    backupLock.lockForBackup();
+    backupLock.setBackupThread();
+
+    assertThat(backupLock.hasThreadLocal()).isTrue();
+
+    backupLock.unlockForBackup();
+
+    assertThat(backupLock.hasThreadLocal()).isFalse();
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupPrepareAndFinishMsgDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupPrepareAndFinishMsgDUnitTest.java
similarity index 93%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupPrepareAndFinishMsgDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupPrepareAndFinishMsgDUnitTest.java
index 2dfc16c..f48f7f3 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/BackupPrepareAndFinishMsgDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupPrepareAndFinishMsgDUnitTest.java
@@ -12,13 +12,11 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.util.ArrayList;
@@ -40,8 +38,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.admin.internal.FinishBackupRequest;
-import org.apache.geode.admin.internal.PrepareBackupRequest;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.DiskStore;
 import org.apache.geode.cache.DiskStoreFactory;
@@ -55,13 +51,13 @@ import org.apache.geode.cache.query.QueryInvocationTargetException;
 import org.apache.geode.cache.query.TypeMismatchException;
 import org.apache.geode.cache30.CacheTestCase;
 import org.apache.geode.distributed.internal.DM;
-import org.apache.geode.internal.cache.BackupLock;
 import org.apache.geode.internal.cache.DiskStoreImpl;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 @Category({DistributedTest.class})
 public abstract class BackupPrepareAndFinishMsgDUnitTest extends CacheTestCase {
+
   // Although this test does not make use of other members, the current member needs to be
   // a distributed member (rather than local) because it sends prepare and finish backup messages
   private static final String TEST_REGION_NAME = "TestRegion";
@@ -141,24 +137,28 @@ public abstract class BackupPrepareAndFinishMsgDUnitTest extends CacheTestCase {
     DM dm = GemFireCacheImpl.getInstance().getDistributionManager();
     Set recipients = dm.getOtherDistributionManagerIds();
     Future<Void> future = null;
-    PrepareBackupRequest.send(dm, recipients);
+    new PrepareBackupOperation(dm, dm.getId(), dm.getCache(), recipients,
+        new PrepareBackupFactory()).send();
     waitingForBackupLockCount = 0;
     future = CompletableFuture.runAsync(function);
     Awaitility.await().atMost(5, TimeUnit.SECONDS)
         .until(() -> assertTrue(waitingForBackupLockCount == 1));
-    FinishBackupRequest.send(dm, recipients, diskDirs[0], null, false);
+    new FinishBackupOperation(dm, dm.getId(), dm.getCache(), recipients, diskDirs[0], null, false,
+        new FinishBackupFactory()).send();
     future.get(5, TimeUnit.SECONDS);
   }
 
   private void doReadActionsAndVerifyCompletion() {
     DM dm = GemFireCacheImpl.getInstance().getDistributionManager();
     Set recipients = dm.getOtherDistributionManagerIds();
-    PrepareBackupRequest.send(dm, recipients);
+    new PrepareBackupOperation(dm, dm.getId(), dm.getCache(), recipients,
+        new PrepareBackupFactory()).send();
     waitingForBackupLockCount = 0;
     List<CompletableFuture<?>> futureList = doReadActions();
     CompletableFuture.allOf(futureList.toArray(new CompletableFuture<?>[futureList.size()]));
     assertTrue(waitingForBackupLockCount == 0);
-    FinishBackupRequest.send(dm, recipients, diskDirs[0], null, false);
+    new FinishBackupOperation(dm, dm.getId(), dm.getCache(), recipients, diskDirs[0], null, false,
+        new FinishBackupFactory()).send();
   }
 
   private void verifyKeyValuePair(Integer key, Integer expectedValue) {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupReplyProcessorTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupReplyProcessorTest.java
new file mode 100644
index 0000000..e014852
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/BackupReplyProcessorTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.internal.cache.backup;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.DistributionMessage;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class BackupReplyProcessorTest {
+
+  private BackupReplyProcessor backupReplyProcessor;
+
+  private BackupResultCollector resultCollector;
+  private DM dm;
+  private InternalDistributedSystem system;
+  private InternalDistributedMember sender;
+
+  private Set<InternalDistributedMember> recipients;
+  private Set<PersistentID> persistentIds;
+
+  private BackupResponse backupResponse;
+  private DistributionMessage nonBackupResponse;
+
+  @Before
+  public void setUp() throws Exception {
+    resultCollector = mock(BackupResultCollector.class);
+    dm = mock(DM.class);
+    system = mock(InternalDistributedSystem.class);
+    backupResponse = mock(BackupResponse.class);
+    nonBackupResponse = mock(DistributionMessage.class);
+    sender = mock(InternalDistributedMember.class);
+
+    recipients = new HashSet<>();
+    persistentIds = new HashSet<>();
+
+    when(dm.getSystem()).thenReturn(system);
+    when(dm.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
+    when(backupResponse.getSender()).thenReturn(sender);
+    when(backupResponse.getPersistentIds()).thenReturn(persistentIds);
+    when(nonBackupResponse.getSender()).thenReturn(sender);
+
+    backupReplyProcessor = new BackupReplyProcessor(resultCollector, dm, recipients);
+  }
+
+  @Test
+  public void stopBecauseOfExceptionsReturnsFalse() throws Exception {
+    assertThat(backupReplyProcessor.stopBecauseOfExceptions()).isFalse();
+  }
+
+  @Test
+  public void processBackupResponseAddsSenderToResults() throws Exception {
+    backupReplyProcessor.process(backupResponse, false);
+
+    verify(resultCollector, times(1)).addToResults(eq(sender), eq(persistentIds));
+  }
+
+  @Test
+  public void processNonBackupResponseDoesNotAddSenderToResults() throws Exception {
+    backupReplyProcessor.process(nonBackupResponse, false);
+
+    verify(resultCollector, times(0)).addToResults(eq(sender), eq(persistentIds));
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FinishBackupFactoryTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FinishBackupFactoryTest.java
new file mode 100644
index 0000000..ce2e7f4
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FinishBackupFactoryTest.java
@@ -0,0 +1,87 @@
+/*
+ * 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.internal.cache.backup;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class FinishBackupFactoryTest {
+
+  private FinishBackupFactory finishBackupFactory;
+
+  private BackupResultCollector resultCollector;
+  private DM dm;
+  private InternalDistributedMember sender;
+  private Set<InternalDistributedMember> recipients;
+  private InternalDistributedMember member;
+  private InternalCache cache;
+
+  @Before
+  public void setUp() throws Exception {
+    resultCollector = mock(BackupResultCollector.class);
+    dm = mock(DM.class);
+    sender = mock(InternalDistributedMember.class);
+    member = mock(InternalDistributedMember.class);
+    cache = mock(InternalCache.class);
+
+    recipients = new HashSet<>();
+
+    when(dm.getSystem()).thenReturn(mock(InternalDistributedSystem.class));
+    when(dm.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
+
+    finishBackupFactory = new FinishBackupFactory();
+  }
+
+  @Test
+  public void createReplyProcessorReturnsBackupReplyProcessor() throws Exception {
+    assertThat(finishBackupFactory.createReplyProcessor(resultCollector, dm, recipients))
+        .isInstanceOf(BackupReplyProcessor.class);
+  }
+
+  @Test
+  public void createRequestReturnsFinishBackupRequest() throws Exception {
+    assertThat(finishBackupFactory.createRequest(sender, recipients, 1, new File("targetDir"),
+        new File("baselineDir"), false)).isInstanceOf(FinishBackupRequest.class);
+  }
+
+  @Test
+  public void createFinishBackupReturnsFinishBackup() throws Exception {
+    assertThat(finishBackupFactory.createFinishBackup(cache, new File("targetDir"),
+        new File("baselineDir"), false)).isInstanceOf(FinishBackup.class);
+  }
+
+  @Test
+  public void createBackupResponseReturnsBackupResponse() {
+    assertThat(finishBackupFactory.createBackupResponse(member, null))
+        .isInstanceOf(BackupResponse.class);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FinishBackupOperationTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FinishBackupOperationTest.java
new file mode 100644
index 0000000..c0972e8
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FinishBackupOperationTest.java
@@ -0,0 +1,236 @@
+/*
+ * 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.internal.cache.backup;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.InOrder;
+import org.mockito.stubbing.Answer;
+
+import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ReplyException;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class FinishBackupOperationTest {
+
+  private DM dm;
+  private InternalCache cache;
+  private Set<InternalDistributedMember> recipients;
+
+  private InternalDistributedMember sender;
+  private InternalDistributedMember member1;
+  private InternalDistributedMember member2;
+
+  private File targetDir = new File("targetDir");
+  private File baselineDir = new File("baselineDir");
+  private boolean abort = false;
+
+  private FinishBackupFactory finishBackupFactory;
+  private BackupReplyProcessor finishBackupReplyProcessor;
+  private FinishBackupRequest finishBackupRequest;
+  private FinishBackup finishBackup;
+
+  private FinishBackupOperation finishBackupOperation;
+
+  @Before
+  public void setUp() throws Exception {
+    dm = mock(DM.class);
+    cache = mock(InternalCache.class);
+
+    finishBackupReplyProcessor = mock(BackupReplyProcessor.class);
+    finishBackupRequest = mock(FinishBackupRequest.class);
+    finishBackup = mock(FinishBackup.class);
+
+    finishBackupFactory = mock(FinishBackupFactory.class);
+
+    sender = mock(InternalDistributedMember.class, "sender");
+    member1 = mock(InternalDistributedMember.class, "member1");
+    member2 = mock(InternalDistributedMember.class, "member2");
+    recipients = new HashSet<>();
+
+    finishBackupOperation = new FinishBackupOperation(dm, sender, cache, recipients, targetDir,
+        baselineDir, abort, finishBackupFactory);
+
+    when(finishBackupReplyProcessor.getProcessorId()).thenReturn(42);
+
+    when(
+        finishBackupFactory.createReplyProcessor(eq(finishBackupOperation), eq(dm), eq(recipients)))
+            .thenReturn(finishBackupReplyProcessor);
+    when(finishBackupFactory.createRequest(eq(sender), eq(recipients), eq(42), eq(targetDir),
+        eq(baselineDir), eq(abort))).thenReturn(finishBackupRequest);
+    when(finishBackupFactory.createFinishBackup(eq(cache), eq(targetDir), eq(baselineDir),
+        eq(abort))).thenReturn(finishBackup);
+  }
+
+  @Test
+  public void sendShouldSendfinishBackupMessage() throws Exception {
+    finishBackupOperation.send();
+
+    verify(dm, times(1)).putOutgoing(finishBackupRequest);
+  }
+
+  @Test
+  public void sendReturnsResultsForRemoteRecipient() throws Exception {
+    HashSet<PersistentID> persistentIdsForMember1 = new HashSet<>();
+    persistentIdsForMember1.add(mock(PersistentID.class));
+    doAnswer(invokeAddToResults(new MemberWithPersistentIds(member1, persistentIdsForMember1)))
+        .when(finishBackupReplyProcessor).waitForReplies();
+
+    assertThat(finishBackupOperation.send()).containsOnlyKeys(member1)
+        .containsValues(persistentIdsForMember1);
+  }
+
+  @Test
+  public void sendReturnsResultsForLocalMember() throws Exception {
+    HashSet<PersistentID> persistentIdsForSender = new HashSet<>();
+    persistentIdsForSender.add(mock(PersistentID.class));
+    when(finishBackup.run()).thenReturn(persistentIdsForSender);
+
+    assertThat(finishBackupOperation.send()).containsOnlyKeys(sender)
+        .containsValue(persistentIdsForSender);
+  }
+
+  @Test
+  public void sendReturnsResultsForAllMembers() throws Exception {
+    HashSet<PersistentID> persistentIdsForMember1 = new HashSet<>();
+    persistentIdsForMember1.add(mock(PersistentID.class));
+
+    HashSet<PersistentID> persistentIdsForMember2 = new HashSet<>();
+    persistentIdsForMember2.add(mock(PersistentID.class));
+
+    MemberWithPersistentIds[] ids = new MemberWithPersistentIds[] {
+        new MemberWithPersistentIds(member1, persistentIdsForMember1),
+        new MemberWithPersistentIds(member2, persistentIdsForMember2)};
+
+    doAnswer(invokeAddToResults(ids)).when(finishBackupReplyProcessor).waitForReplies();
+
+    HashSet<PersistentID> persistentIdsForSender = new HashSet<>();
+    persistentIdsForSender.add(mock(PersistentID.class));
+    when(finishBackup.run()).thenReturn(persistentIdsForSender);
+
+    assertThat(finishBackupOperation.send()).containsOnlyKeys(member1, member2, sender)
+        .containsValues(persistentIdsForSender, persistentIdsForMember1, persistentIdsForMember2);
+  }
+
+  @Test
+  public void getResultsShouldReturnEmptyMapByDefault() throws Exception {
+    assertThat(finishBackupOperation.getResults()).isEmpty();
+  }
+
+  @Test
+  public void addToResultsWithNullShouldBeNoop() throws Exception {
+    finishBackupOperation.addToResults(member1, null);
+    assertThat(finishBackupOperation.getResults()).isEmpty();
+  }
+
+  @Test
+  public void addToResultsWithEmptySetShouldBeNoop() throws Exception {
+    finishBackupOperation.addToResults(member1, new HashSet<>());
+    assertThat(finishBackupOperation.getResults()).isEmpty();
+  }
+
+  @Test
+  public void addToResultsShouldShowUpInGetResults() throws Exception {
+    HashSet<PersistentID> persistentIdsForMember1 = new HashSet<>();
+    persistentIdsForMember1.add(mock(PersistentID.class));
+    finishBackupOperation.addToResults(member1, persistentIdsForMember1);
+    assertThat(finishBackupOperation.getResults()).containsOnlyKeys(member1)
+        .containsValue(persistentIdsForMember1);
+  }
+
+  @Test
+  public void sendShouldHandleIOExceptionThrownFromRun() throws Exception {
+    when(finishBackup.run()).thenThrow(new IOException("expected exception"));
+    finishBackupOperation.send();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void sendShouldThrowNonIOExceptionThrownFromRun() throws Exception {
+    when(finishBackup.run()).thenThrow(new RuntimeException("expected exception"));
+    finishBackupOperation.send();
+  }
+
+  @Test
+  public void sendShouldHandleCancelExceptionFromWaitForReplies() throws Exception {
+    ReplyException replyException =
+        new ReplyException("expected exception", new CacheClosedException("expected exception"));
+    doThrow(replyException).when(finishBackupReplyProcessor).waitForReplies();
+    finishBackupOperation.send();
+  }
+
+  @Test
+  public void sendShouldHandleInterruptedExceptionFromWaitForReplies() throws Exception {
+    doThrow(new InterruptedException("expected exception")).when(finishBackupReplyProcessor)
+        .waitForReplies();
+    finishBackupOperation.send();
+  }
+
+  @Test(expected = ReplyException.class)
+  public void sendShouldThrowReplyExceptionWithNoCauseFromWaitForReplies() throws Exception {
+    doThrow(new ReplyException("expected exception")).when(finishBackupReplyProcessor)
+        .waitForReplies();
+    finishBackupOperation.send();
+  }
+
+  @Test(expected = ReplyException.class)
+  public void sendShouldThrowReplyExceptionWithCauseThatIsNotACancelFromWaitForReplies()
+      throws Exception {
+    doThrow(new ReplyException("expected exception", new RuntimeException("expected")))
+        .when(finishBackupReplyProcessor).waitForReplies();
+    finishBackupOperation.send();
+  }
+
+  @Test
+  public void sendShouldfinishForBackupInLocalMemberBeforeWaitingForReplies() throws Exception {
+    InOrder inOrder = inOrder(finishBackup, finishBackupReplyProcessor);
+    finishBackupOperation.send();
+
+    inOrder.verify(finishBackup, times(1)).run();
+    inOrder.verify(finishBackupReplyProcessor, times(1)).waitForReplies();
+  }
+
+  private Answer<Object> invokeAddToResults(MemberWithPersistentIds... memberWithPersistentIds) {
+    return invocation -> {
+      for (MemberWithPersistentIds ids : memberWithPersistentIds) {
+        finishBackupOperation.addToResults(ids.member, ids.persistentIds);
+      }
+      return null;
+    };
+  }
+
+  private static class MemberWithPersistentIds {
+    InternalDistributedMember member;
+    HashSet<PersistentID> persistentIds;
+
+    MemberWithPersistentIds(InternalDistributedMember member, HashSet<PersistentID> persistentIds) {
+      this.member = member;
+      this.persistentIds = persistentIds;
+    }
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FinishBackupRequestTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FinishBackupRequestTest.java
new file mode 100644
index 0000000..5e1a5a1
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FinishBackupRequestTest.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.cache.backup;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.admin.remote.AdminFailureResponse;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class FinishBackupRequestTest {
+
+  private FinishBackupRequest finishBackupRequest;
+
+  private DM dm;
+  private InternalCache cache;
+  private BackupManager backupManager;
+  private int processorId = 79;
+  private File targetDir;
+  private File baselineDir;
+  private boolean abort;
+  private FinishBackupFactory finishBackupFactory;
+  private InternalDistributedMember sender;
+  private Set<InternalDistributedMember> recipients;
+  private HashSet<PersistentID> persistentIds;
+  private FinishBackup finishBackup;
+
+  @Before
+  public void setUp() throws Exception {
+    // mocks here
+    dm = mock(DM.class);
+    cache = mock(InternalCache.class);
+    backupManager = mock(BackupManager.class);
+    targetDir = mock(File.class);
+    baselineDir = mock(File.class);
+    abort = false;
+
+    when(dm.getCache()).thenReturn(cache);
+    when(dm.getDistributionManagerId()).thenReturn(sender);
+    when(cache.getBackupManager()).thenReturn(backupManager);
+
+    sender = mock(InternalDistributedMember.class);
+
+    recipients = new HashSet<>();
+    persistentIds = new HashSet<>();
+
+    finishBackup = mock(FinishBackup.class);
+    when(finishBackup.run()).thenReturn(persistentIds);
+
+    finishBackupFactory = mock(FinishBackupFactory.class);
+    when(finishBackupFactory.createFinishBackup(eq(cache), eq(targetDir), eq(baselineDir),
+        eq(abort))).thenReturn(finishBackup);
+    when(finishBackupFactory.createBackupResponse(eq(sender), eq(persistentIds)))
+        .thenReturn(mock(BackupResponse.class));
+
+
+    finishBackupRequest = new FinishBackupRequest(sender, recipients, processorId, targetDir,
+        baselineDir, false, finishBackupFactory);
+  }
+
+  @Test
+  public void usesFactoryToCreateFinishBackup() throws Exception {
+    finishBackupRequest.createResponse(dm);
+
+    verify(finishBackupFactory, times(1)).createFinishBackup(eq(cache), eq(targetDir),
+        eq(baselineDir), eq(abort));
+  }
+
+  @Test
+  public void usesFactoryToCreateBackupResponse() throws Exception {
+    finishBackupRequest.createResponse(dm);
+
+    verify(finishBackupFactory, times(1)).createBackupResponse(eq(sender), eq(persistentIds));
+  }
+
+  @Test
+  public void returnsBackupResponse() throws Exception {
+    assertThat(finishBackupRequest.createResponse(dm)).isInstanceOf(BackupResponse.class);
+  }
+
+  @Test
+  public void returnsAdminFailureResponseWhenFinishBackupThrowsIOException() throws Exception {
+    when(finishBackup.run()).thenThrow(new IOException());
+
+    assertThat(finishBackupRequest.createResponse(dm)).isInstanceOf(AdminFailureResponse.class);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FlushToDiskFactoryTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FlushToDiskFactoryTest.java
new file mode 100644
index 0000000..cfb62ab
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FlushToDiskFactoryTest.java
@@ -0,0 +1,83 @@
+/*
+ * 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.internal.cache.backup;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.io.File;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class FlushToDiskFactoryTest {
+
+  private FlushToDiskFactory flushToDiskFactory;
+
+  private DM dm;
+  private InternalDistributedMember sender;
+  private Set<InternalDistributedMember> recipients;
+  private InternalDistributedMember member;
+  private InternalCache cache;
+
+  @Before
+  public void setUp() throws Exception {
+    dm = mock(DM.class);
+    sender = mock(InternalDistributedMember.class);
+    member = mock(InternalDistributedMember.class);
+    cache = mock(InternalCache.class);
+
+    recipients = new HashSet<>();
+
+    when(dm.getSystem()).thenReturn(mock(InternalDistributedSystem.class));
+    when(dm.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
+
+    flushToDiskFactory = new FlushToDiskFactory();
+  }
+
+  @Test
+  public void createReplyProcessorReturnsFlushToDiskReplyProcessor() throws Exception {
+    assertThat(flushToDiskFactory.createReplyProcessor(dm, recipients))
+        .isInstanceOf(FlushToDiskProcessor.class);
+  }
+
+  @Test
+  public void createRequestReturnsFlushToDiskRequest() throws Exception {
+    assertThat(flushToDiskFactory.createRequest(sender, recipients, 1))
+        .isInstanceOf(FlushToDiskRequest.class);
+  }
+
+  @Test
+  public void createFlushToDiskReturnsFlushToDisk() throws Exception {
+    assertThat(flushToDiskFactory.createFlushToDisk(cache)).isInstanceOf(FlushToDisk.class);
+  }
+
+  @Test
+  public void createResponseReturnsFlushToDiskResponse() {
+    assertThat(flushToDiskFactory.createResponse(member)).isInstanceOf(FlushToDiskResponse.class);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FlushToDiskOperationTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FlushToDiskOperationTest.java
new file mode 100644
index 0000000..a79b43f
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FlushToDiskOperationTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.internal.cache.backup;
+
+import static org.mockito.Mockito.*;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.InOrder;
+
+import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ReplyException;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class FlushToDiskOperationTest {
+
+  private DM dm;
+  private InternalCache cache;
+  private Set<InternalDistributedMember> recipients;
+
+  private InternalDistributedMember sender;
+
+  private FlushToDiskFactory flushToDiskFactory;
+  private FlushToDiskProcessor flushToDiskReplyProcessor;
+  private FlushToDiskRequest flushToDiskRequest;
+  private FlushToDisk flushToDisk;
+
+  private FlushToDiskOperation flushToDiskOperation;
+
+  @Before
+  public void setUp() throws Exception {
+    dm = mock(DM.class);
+    cache = mock(InternalCache.class);
+
+    flushToDiskReplyProcessor = mock(FlushToDiskProcessor.class);
+    flushToDiskRequest = mock(FlushToDiskRequest.class);
+    flushToDisk = mock(FlushToDisk.class);
+
+    flushToDiskFactory = mock(FlushToDiskFactory.class);
+
+    sender = mock(InternalDistributedMember.class, "sender");
+    recipients = new HashSet<>();
+
+    flushToDiskOperation =
+        new FlushToDiskOperation(dm, sender, cache, recipients, flushToDiskFactory);
+
+    when(flushToDiskReplyProcessor.getProcessorId()).thenReturn(42);
+
+    when(flushToDiskFactory.createReplyProcessor(eq(dm), eq(recipients)))
+        .thenReturn(flushToDiskReplyProcessor);
+    when(flushToDiskFactory.createRequest(eq(sender), eq(recipients), eq(42)))
+        .thenReturn(flushToDiskRequest);
+    when(flushToDiskFactory.createFlushToDisk(eq(cache))).thenReturn(flushToDisk);
+  }
+
+  @Test
+  public void sendShouldSendFlushToDiskMessage() throws Exception {
+    flushToDiskOperation.send();
+
+    verify(dm, times(1)).putOutgoing(flushToDiskRequest);
+  }
+
+  @Test
+  public void sendShouldHandleCancelExceptionFromWaitForReplies() throws Exception {
+    ReplyException replyException =
+        new ReplyException("expected exception", new CacheClosedException("expected exception"));
+    doThrow(replyException).when(flushToDiskReplyProcessor).waitForReplies();
+    flushToDiskOperation.send();
+  }
+
+  @Test
+  public void sendShouldHandleInterruptedExceptionFromWaitForReplies() throws Exception {
+    doThrow(new InterruptedException("expected exception")).when(flushToDiskReplyProcessor)
+        .waitForReplies();
+    flushToDiskOperation.send();
+  }
+
+  @Test(expected = ReplyException.class)
+  public void sendShouldThrowReplyExceptionWithNoCauseFromWaitForReplies() throws Exception {
+    doThrow(new ReplyException("expected exception")).when(flushToDiskReplyProcessor)
+        .waitForReplies();
+    flushToDiskOperation.send();
+  }
+
+  @Test(expected = ReplyException.class)
+  public void sendShouldThrowReplyExceptionWithCauseThatIsNotACancelFromWaitForReplies()
+      throws Exception {
+    doThrow(new ReplyException("expected exception", new RuntimeException("expected")))
+        .when(flushToDiskReplyProcessor).waitForReplies();
+    flushToDiskOperation.send();
+  }
+
+  @Test
+  public void sendShouldProcessLocallyBeforeWaitingForReplies() throws Exception {
+    InOrder inOrder = inOrder(flushToDisk, flushToDiskReplyProcessor);
+    flushToDiskOperation.send();
+
+    inOrder.verify(flushToDisk, times(1)).run();
+    inOrder.verify(flushToDiskReplyProcessor, times(1)).waitForReplies();
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FlushToDiskRequestTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FlushToDiskRequestTest.java
new file mode 100644
index 0000000..113bb70
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/FlushToDiskRequestTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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.internal.cache.backup;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class FlushToDiskRequestTest {
+
+  private FlushToDiskRequest flushToDiskRequest;
+
+  private DM dm;
+  private Set<InternalDistributedMember> recipients;
+  private int msgId;
+  private FlushToDiskFactory flushToDiskFactory;
+  private InternalDistributedMember sender;
+  private InternalCache cache;
+  private FlushToDisk flushToDisk;
+
+  @Before
+  public void setUp() throws Exception {
+    dm = mock(DM.class);
+    sender = mock(InternalDistributedMember.class);
+    cache = mock(InternalCache.class);
+    flushToDiskFactory = mock(FlushToDiskFactory.class);
+    flushToDisk = mock(FlushToDisk.class);
+
+    msgId = 42;
+    recipients = new HashSet<>();
+
+    when(dm.getCache()).thenReturn(cache);
+    when(dm.getDistributionManagerId()).thenReturn(sender);
+    when(flushToDiskFactory.createFlushToDisk(eq(cache))).thenReturn(flushToDisk);
+    when(flushToDiskFactory.createResponse(eq(sender))).thenReturn(mock(FlushToDiskResponse.class));
+
+    flushToDiskRequest = new FlushToDiskRequest(sender, recipients, msgId, flushToDiskFactory);
+  }
+
+  @Test
+  public void usesFactoryToCreateFlushToDisk() throws Exception {
+    flushToDiskRequest.createResponse(dm);
+
+    verify(flushToDiskFactory, times(1)).createFlushToDisk(eq(cache));
+  }
+
+  @Test
+  public void usesFactoryToCreateResponse() throws Exception {
+    flushToDiskRequest.createResponse(dm);
+
+    verify(flushToDiskFactory, times(1)).createResponse(eq(sender));
+  }
+
+  @Test
+  public void returnsFlushToDiskResponse() throws Exception {
+    assertThat(flushToDiskRequest.createResponse(dm)).isInstanceOf(FlushToDiskResponse.class);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/IncrementalBackupDistributedTest.java
similarity index 98%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/backup/IncrementalBackupDistributedTest.java
index cc245f6..69b6478 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/IncrementalBackupDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/IncrementalBackupDistributedTest.java
@@ -12,10 +12,13 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache;
+package org.apache.geode.internal.cache.backup;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -51,6 +54,8 @@ import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.DeployedJar;
+import org.apache.geode.internal.cache.DiskStoreImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.util.IOUtils;
 import org.apache.geode.internal.util.TransformUtils;
 import org.apache.geode.management.BackupStatus;
@@ -71,7 +76,7 @@ import org.apache.geode.test.junit.categories.DistributedTest;
  */
 @Category(DistributedTest.class)
 @SuppressWarnings("serial")
-public class IncrementalBackupDUnitTest extends JUnit4CacheTestCase {
+public class IncrementalBackupDistributedTest extends JUnit4CacheTestCase {
   /**
    * Data load increment.
    */
@@ -124,7 +129,7 @@ public class IncrementalBackupDUnitTest extends JUnit4CacheTestCase {
    * @param message a message to log.
    */
   private void log(String message) {
-    LogWriterUtils.getLogWriter().info("[IncrementalBackupDUnitTest] " + message);
+    LogWriterUtils.getLogWriter().info("[IncrementalBackupDistributedTest] " + message);
   }
 
   /**
@@ -354,7 +359,7 @@ public class IncrementalBackupDUnitTest extends JUnit4CacheTestCase {
 
       @Override
       public String description() {
-        return "[IncrementalBackupDUnitTest] Waiting for missing member " + id;
+        return "[IncrementalBackupDistributedTest] Waiting for missing member " + id;
       }
     }, 10000, 500, false);
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PartitionedBackupPrepareAndFinishMsgDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PartitionedBackupPrepareAndFinishMsgDUnitTest.java
similarity index 85%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PartitionedBackupPrepareAndFinishMsgDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/backup/PartitionedBackupPrepareAndFinishMsgDUnitTest.java
index 4b42c21..b734286 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PartitionedBackupPrepareAndFinishMsgDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PartitionedBackupPrepareAndFinishMsgDUnitTest.java
@@ -12,17 +12,16 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionShortcut;
 
 public class PartitionedBackupPrepareAndFinishMsgDUnitTest
     extends BackupPrepareAndFinishMsgDUnitTest {
-  private static final RegionShortcut REGION_TYPE = RegionShortcut.PARTITION_PERSISTENT;
 
   @Override
   public Region<Integer, Integer> createRegion() {
-    return createRegion(REGION_TYPE);
+    return createRegion(RegionShortcut.PARTITION_PERSISTENT);
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PrepareBackupFactoryTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PrepareBackupFactoryTest.java
new file mode 100644
index 0000000..41431a3
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PrepareBackupFactoryTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.internal.cache.backup;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.CancelCriterion;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class PrepareBackupFactoryTest {
+
+  private PrepareBackupFactory prepareBackupFactory;
+
+  private BackupResultCollector resultCollector;
+  private DM dm;
+  private InternalDistributedMember sender;
+  private Set<InternalDistributedMember> recipients;
+  private InternalDistributedMember member;
+  private InternalCache cache;
+
+  @Before
+  public void setUp() throws Exception {
+    resultCollector = mock(BackupResultCollector.class);
+    dm = mock(DM.class);
+    sender = mock(InternalDistributedMember.class);
+    member = mock(InternalDistributedMember.class);
+    cache = mock(InternalCache.class);
+
+    recipients = new HashSet<>();
+
+    when(dm.getSystem()).thenReturn(mock(InternalDistributedSystem.class));
+    when(dm.getCancelCriterion()).thenReturn(mock(CancelCriterion.class));
+
+    prepareBackupFactory = new PrepareBackupFactory();
+  }
+
+  @Test
+  public void createReplyProcessorReturnsBackupReplyProcessor() throws Exception {
+    assertThat(prepareBackupFactory.createReplyProcessor(resultCollector, dm, recipients))
+        .isInstanceOf(BackupReplyProcessor.class);
+  }
+
+  @Test
+  public void createRequestReturnsPrepareBackupRequest() throws Exception {
+    assertThat(prepareBackupFactory.createRequest(sender, recipients, 1))
+        .isInstanceOf(PrepareBackupRequest.class);
+  }
+
+  @Test
+  public void createPrepareBackupReturnsPrepareBackup() throws Exception {
+    assertThat(prepareBackupFactory.createPrepareBackup(member, cache))
+        .isInstanceOf(PrepareBackup.class);
+  }
+
+  @Test
+  public void createBackupResponseReturnsBackupResponse() {
+    assertThat(prepareBackupFactory.createBackupResponse(member, null))
+        .isInstanceOf(BackupResponse.class);
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PrepareBackupOperationTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PrepareBackupOperationTest.java
new file mode 100644
index 0000000..2e4246a
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PrepareBackupOperationTest.java
@@ -0,0 +1,232 @@
+/*
+ * 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.internal.cache.backup;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.InOrder;
+import org.mockito.stubbing.Answer;
+
+import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.ReplyException;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class PrepareBackupOperationTest {
+
+  private DM dm;
+  private InternalCache cache;
+  private Set<InternalDistributedMember> recipients;
+
+  private InternalDistributedMember sender;
+  private InternalDistributedMember member1;
+  private InternalDistributedMember member2;
+
+  private PrepareBackupFactory prepareBackupFactory;
+  private BackupReplyProcessor prepareBackupReplyProcessor;
+  private PrepareBackupRequest prepareBackupRequest;
+  private PrepareBackup prepareBackup;
+
+  private PrepareBackupOperation prepareBackupOperation;
+
+  @Before
+  public void setUp() throws Exception {
+    dm = mock(DM.class);
+    cache = mock(InternalCache.class);
+
+    prepareBackupReplyProcessor = mock(BackupReplyProcessor.class);
+    prepareBackupRequest = mock(PrepareBackupRequest.class);
+    prepareBackup = mock(PrepareBackup.class);
+
+    prepareBackupFactory = mock(PrepareBackupFactory.class);
+
+    sender = mock(InternalDistributedMember.class, "sender");
+    member1 = mock(InternalDistributedMember.class, "member1");
+    member2 = mock(InternalDistributedMember.class, "member2");
+    recipients = new HashSet<>();
+
+    prepareBackupOperation =
+        new PrepareBackupOperation(dm, sender, cache, recipients, prepareBackupFactory);
+
+    when(prepareBackupReplyProcessor.getProcessorId()).thenReturn(42);
+
+    when(prepareBackupFactory.createReplyProcessor(eq(prepareBackupOperation), eq(dm),
+        eq(recipients))).thenReturn(prepareBackupReplyProcessor);
+    when(prepareBackupFactory.createRequest(eq(sender), eq(recipients), eq(42)))
+        .thenReturn(prepareBackupRequest);
+    when(prepareBackupFactory.createPrepareBackup(eq(sender), eq(cache))).thenReturn(prepareBackup);
+  }
+
+  @Test
+  public void sendShouldSendPrepareBackupMessage() throws Exception {
+    prepareBackupOperation.send();
+
+    verify(dm, times(1)).putOutgoing(prepareBackupRequest);
+  }
+
+  @Test
+  public void sendReturnsResultsForRemoteRecipient() throws Exception {
+    HashSet<PersistentID> persistentIdsForMember1 = new HashSet<>();
+    persistentIdsForMember1.add(mock(PersistentID.class));
+    doAnswer(invokeAddToResults(new MemberWithPersistentIds(member1, persistentIdsForMember1)))
+        .when(prepareBackupReplyProcessor).waitForReplies();
+
+    assertThat(prepareBackupOperation.send()).containsOnlyKeys(member1)
+        .containsValues(persistentIdsForMember1);
+  }
+
+  @Test
+  public void sendReturnsResultsForLocalMember() throws Exception {
+    HashSet<PersistentID> persistentIdsForSender = new HashSet<>();
+    persistentIdsForSender.add(mock(PersistentID.class));
+    when(prepareBackup.run()).thenReturn(persistentIdsForSender);
+
+    assertThat(prepareBackupOperation.send()).containsOnlyKeys(sender)
+        .containsValue(persistentIdsForSender);
+  }
+
+  @Test
+  public void sendReturnsResultsForAllMembers() throws Exception {
+    HashSet<PersistentID> persistentIdsForMember1 = new HashSet<>();
+    persistentIdsForMember1.add(mock(PersistentID.class));
+
+    HashSet<PersistentID> persistentIdsForMember2 = new HashSet<>();
+    persistentIdsForMember2.add(mock(PersistentID.class));
+
+    MemberWithPersistentIds[] ids = new MemberWithPersistentIds[] {
+        new MemberWithPersistentIds(member1, persistentIdsForMember1),
+        new MemberWithPersistentIds(member2, persistentIdsForMember2)};
+
+    doAnswer(invokeAddToResults(ids)).when(prepareBackupReplyProcessor).waitForReplies();
+
+    // prepareBackupOperation.addToResults(ids[0].member, ids[0].persistentIds);
+    // prepareBackupOperation.addToResults(ids[1].member, ids[1].persistentIds);
+
+    HashSet<PersistentID> persistentIdsForSender = new HashSet<>();
+    persistentIdsForSender.add(mock(PersistentID.class));
+    when(prepareBackup.run()).thenReturn(persistentIdsForSender);
+
+    assertThat(prepareBackupOperation.send()).containsOnlyKeys(member1, member2, sender)
+        .containsValues(persistentIdsForSender, persistentIdsForMember1, persistentIdsForMember2);
+  }
+
+  @Test
+  public void getResultsShouldReturnEmptyMapByDefault() throws Exception {
+    assertThat(prepareBackupOperation.getResults()).isEmpty();
+  }
+
+  @Test
+  public void addToResultsWithNullShouldBeNoop() throws Exception {
+    prepareBackupOperation.addToResults(member1, null);
+    assertThat(prepareBackupOperation.getResults()).isEmpty();
+  }
+
+  @Test
+  public void addToResultsWithEmptySetShouldBeNoop() throws Exception {
+    prepareBackupOperation.addToResults(member1, new HashSet<>());
+    assertThat(prepareBackupOperation.getResults()).isEmpty();
+  }
+
+  @Test
+  public void addToResultsShouldShowUpInGetResults() throws Exception {
+    HashSet<PersistentID> persistentIdsForMember1 = new HashSet<>();
+    persistentIdsForMember1.add(mock(PersistentID.class));
+    prepareBackupOperation.addToResults(member1, persistentIdsForMember1);
+    assertThat(prepareBackupOperation.getResults()).containsOnlyKeys(member1)
+        .containsValue(persistentIdsForMember1);
+  }
+
+  @Test
+  public void sendShouldHandleIOExceptionThrownFromRun() throws Exception {
+    when(prepareBackup.run()).thenThrow(new IOException("expected exception"));
+    prepareBackupOperation.send();
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void sendShouldThrowNonIOExceptionThrownFromRun() throws Exception {
+    when(prepareBackup.run()).thenThrow(new RuntimeException("expected exception"));
+    prepareBackupOperation.send();
+  }
+
+  @Test
+  public void sendShouldHandleCancelExceptionFromWaitForReplies() throws Exception {
+    ReplyException replyException =
+        new ReplyException("expected exception", new CacheClosedException("expected exception"));
+    doThrow(replyException).when(prepareBackupReplyProcessor).waitForReplies();
+    prepareBackupOperation.send();
+  }
+
+  @Test
+  public void sendShouldHandleInterruptedExceptionFromWaitForReplies() throws Exception {
+    doThrow(new InterruptedException("expected exception")).when(prepareBackupReplyProcessor)
+        .waitForReplies();
+    prepareBackupOperation.send();
+  }
+
+  @Test(expected = ReplyException.class)
+  public void sendShouldThrowReplyExceptionWithNoCauseFromWaitForReplies() throws Exception {
+    doThrow(new ReplyException("expected exception")).when(prepareBackupReplyProcessor)
+        .waitForReplies();
+    prepareBackupOperation.send();
+  }
+
+  @Test(expected = ReplyException.class)
+  public void sendShouldThrowReplyExceptionWithCauseThatIsNotACancelFromWaitForReplies()
+      throws Exception {
+    doThrow(new ReplyException("expected exception", new RuntimeException("expected")))
+        .when(prepareBackupReplyProcessor).waitForReplies();
+    prepareBackupOperation.send();
+  }
+
+  @Test
+  public void sendShouldPrepareForBackupInLocalMemberBeforeWaitingForReplies() throws Exception {
+    InOrder inOrder = inOrder(prepareBackup, prepareBackupReplyProcessor);
+    prepareBackupOperation.send();
+
+    inOrder.verify(prepareBackup, times(1)).run();
+    inOrder.verify(prepareBackupReplyProcessor, times(1)).waitForReplies();
+  }
+
+  private Answer<Object> invokeAddToResults(MemberWithPersistentIds... memberWithPersistentIds) {
+    return invocation -> {
+      for (MemberWithPersistentIds ids : memberWithPersistentIds) {
+        prepareBackupOperation.addToResults(ids.member, ids.persistentIds);
+      }
+      return null;
+    };
+  }
+
+  private static class MemberWithPersistentIds {
+    InternalDistributedMember member;
+    HashSet<PersistentID> persistentIds;
+
+    MemberWithPersistentIds(InternalDistributedMember member, HashSet<PersistentID> persistentIds) {
+      this.member = member;
+      this.persistentIds = persistentIds;
+    }
+  }
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PrepareBackupRequestTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PrepareBackupRequestTest.java
new file mode 100644
index 0000000..f1700bc
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/PrepareBackupRequestTest.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.cache.backup;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.mockito.Mockito.*;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.persistence.PersistentID;
+import org.apache.geode.distributed.internal.DM;
+import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
+import org.apache.geode.internal.admin.remote.AdminFailureResponse;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class PrepareBackupRequestTest {
+
+  private PrepareBackupRequest prepareBackupRequest;
+
+  private DM dm;
+  private Set<InternalDistributedMember> recipients;
+  private int msgId;
+  private PrepareBackupFactory prepareBackupFactory;
+  private InternalDistributedMember sender;
+  private InternalCache cache;
+  private HashSet<PersistentID> persistentIds;
+  private PrepareBackup prepareBackup;
+
+  @Before
+  public void setUp() throws Exception {
+    dm = mock(DM.class);
+    sender = mock(InternalDistributedMember.class);
+    cache = mock(InternalCache.class);
+    prepareBackupFactory = mock(PrepareBackupFactory.class);
+    prepareBackup = mock(PrepareBackup.class);
+
+    msgId = 42;
+    recipients = new HashSet<>();
+    persistentIds = new HashSet<>();
+
+    when(dm.getCache()).thenReturn(cache);
+    when(dm.getDistributionManagerId()).thenReturn(sender);
+    when(prepareBackupFactory.createPrepareBackup(eq(sender), eq(cache))).thenReturn(prepareBackup);
+    when(prepareBackupFactory.createBackupResponse(eq(sender), eq(persistentIds)))
+        .thenReturn(mock(BackupResponse.class));
+    when(prepareBackup.run()).thenReturn(persistentIds);
+
+    prepareBackupRequest =
+        new PrepareBackupRequest(sender, recipients, msgId, prepareBackupFactory);
+  }
+
+  @Test
+  public void usesFactoryToCreatePrepareBackup() throws Exception {
+    prepareBackupRequest.createResponse(dm);
+
+    verify(prepareBackupFactory, times(1)).createPrepareBackup(eq(sender), eq(cache));
+  }
+
+  @Test
+  public void usesFactoryToCreateBackupResponse() throws Exception {
+    prepareBackupRequest.createResponse(dm);
+
+    verify(prepareBackupFactory, times(1)).createBackupResponse(eq(sender), eq(persistentIds));
+  }
+
+  @Test
+  public void returnsBackupResponse() throws Exception {
+    assertThat(prepareBackupRequest.createResponse(dm)).isInstanceOf(BackupResponse.class);
+  }
+
+  @Test
+  public void returnsAdminFailureResponseWhenPrepareBackupThrowsIOException() throws Exception {
+    when(prepareBackup.run()).thenThrow(new IOException());
+
+    assertThat(prepareBackupRequest.createResponse(dm)).isInstanceOf(AdminFailureResponse.class);
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
similarity index 85%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/backup/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
index 3f0ba7d..aa097c1 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/ReplicateBackupPrepareAndFinishMsgDUnitTest.java
@@ -12,17 +12,16 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionShortcut;
 
 public class ReplicateBackupPrepareAndFinishMsgDUnitTest
     extends BackupPrepareAndFinishMsgDUnitTest {
-  private static final RegionShortcut REGION_TYPE = RegionShortcut.REPLICATE_PERSISTENT;
 
   @Override
   public Region<Integer, Integer> createRegion() {
-    return createRegion(REGION_TYPE);
+    return createRegion(RegionShortcut.REPLICATE_PERSISTENT);
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/UnixScriptGeneratorTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/UnixScriptGeneratorTest.java
similarity index 98%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/persistence/UnixScriptGeneratorTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/backup/UnixScriptGeneratorTest.java
index b665b64..1b13cbf 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/UnixScriptGeneratorTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/UnixScriptGeneratorTest.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/WindowsScriptGeneratorTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/WindowsScriptGeneratorTest.java
similarity index 98%
rename from geode-core/src/test/java/org/apache/geode/internal/cache/persistence/WindowsScriptGeneratorTest.java
rename to geode-core/src/test/java/org/apache/geode/internal/cache/backup/WindowsScriptGeneratorTest.java
index fba97bc..59abacf 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/WindowsScriptGeneratorTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/backup/WindowsScriptGeneratorTest.java
@@ -12,7 +12,7 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.persistence;
+package org.apache.geode.internal.cache.backup;
 
 import static org.assertj.core.api.Assertions.assertThat;
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistPRKRFDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistPRKRFDUnitTest.java
index edc068f..e8fb004 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistPRKRFDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistPRKRFDUnitTest.java
@@ -21,6 +21,7 @@ import java.util.Properties;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.CacheException;
 import org.apache.geode.cache.Declarable;
@@ -28,16 +29,17 @@ import org.apache.geode.cache.EntryEvent;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.util.CacheWriterAdapter;
 import org.apache.geode.cache30.CacheSerializableRunnable;
+import org.apache.geode.internal.cache.DiskRegion;
 import org.apache.geode.internal.cache.DiskStoreImpl;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.Host;
 import org.apache.geode.test.dunit.IgnoredException;
+import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.ThreadUtils;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.Wait;
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 /**
@@ -66,7 +68,7 @@ public class PersistPRKRFDUnitTest extends PersistentPartitionedRegionTestBase {
     createData(vm0, 0, 10, "a");
     vm0.invoke(new CacheSerializableRunnable(title + "server add writer") {
       public void run2() throws CacheException {
-        Region region = getRootRegion(PR_REGION_NAME);
+        Region region = getRootRegion(getPartitionedRegionName());
         // let the region to hold on the put until diskstore is closed
         if (!DiskStoreImpl.KRF_DEBUG) {
           region.getAttributesMutator().setCacheWriter(new MyWriter());
@@ -77,7 +79,7 @@ public class PersistPRKRFDUnitTest extends PersistentPartitionedRegionTestBase {
     // create test
     AsyncInvocation async1 = vm0.invokeAsync(new CacheSerializableRunnable(title + "async create") {
       public void run2() throws CacheException {
-        Region region = getRootRegion(PR_REGION_NAME);
+        Region region = getRootRegion(getPartitionedRegionName());
         IgnoredException expect = IgnoredException.addIgnoredException("CacheClosedException");
         try {
           region.put(10, "b");
@@ -111,7 +113,7 @@ public class PersistPRKRFDUnitTest extends PersistentPartitionedRegionTestBase {
     createPR(vm0, 0);
     vm0.invoke(new CacheSerializableRunnable(title + "server add writer") {
       public void run2() throws CacheException {
-        Region region = getRootRegion(PR_REGION_NAME);
+        Region region = getRootRegion(getPartitionedRegionName());
         // let the region to hold on the put until diskstore is closed
         if (!DiskStoreImpl.KRF_DEBUG) {
           region.getAttributesMutator().setCacheWriter(new MyWriter());
@@ -120,7 +122,7 @@ public class PersistPRKRFDUnitTest extends PersistentPartitionedRegionTestBase {
     });
     async1 = vm0.invokeAsync(new CacheSerializableRunnable(title + "async update") {
       public void run2() throws CacheException {
-        Region region = getRootRegion(PR_REGION_NAME);
+        Region region = getRootRegion(getPartitionedRegionName());
         IgnoredException expect = IgnoredException.addIgnoredException("CacheClosedException");
         try {
           region.put(1, "b");
@@ -154,7 +156,7 @@ public class PersistPRKRFDUnitTest extends PersistentPartitionedRegionTestBase {
     createPR(vm0, 0);
     vm0.invoke(new CacheSerializableRunnable(title + "server add writer") {
       public void run2() throws CacheException {
-        Region region = getRootRegion(PR_REGION_NAME);
+        Region region = getRootRegion(getPartitionedRegionName());
         // let the region to hold on the put until diskstore is closed
         if (!DiskStoreImpl.KRF_DEBUG) {
           region.getAttributesMutator().setCacheWriter(new MyWriter());
@@ -163,7 +165,7 @@ public class PersistPRKRFDUnitTest extends PersistentPartitionedRegionTestBase {
     });
     async1 = vm0.invokeAsync(new CacheSerializableRunnable(title + "async destroy") {
       public void run2() throws CacheException {
-        Region region = getRootRegion(PR_REGION_NAME);
+        Region region = getRootRegion(getPartitionedRegionName());
         IgnoredException expect = IgnoredException.addIgnoredException("CacheClosedException");
         try {
           region.destroy(2, "b");
@@ -197,6 +199,24 @@ public class PersistPRKRFDUnitTest extends PersistentPartitionedRegionTestBase {
     closeCache(vm0);
   }
 
+  void checkRecoveredFromDisk(VM vm, final int bucketId, final boolean recoveredLocally) {
+    vm.invoke(new SerializableRunnable("check recovered from disk") {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        PartitionedRegion region = (PartitionedRegion) cache.getRegion(getPartitionedRegionName());
+        DiskRegion disk = region.getRegionAdvisor().getBucket(bucketId).getDiskRegion();
+        if (recoveredLocally) {
+          assertEquals(0, disk.getStats().getRemoteInitializations());
+          assertEquals(1, disk.getStats().getLocalInitializations());
+        } else {
+          assertEquals(1, disk.getStats().getRemoteInitializations());
+          assertEquals(0, disk.getStats().getLocalInitializations());
+        }
+      }
+    });
+  }
+
   private static class MyWriter extends CacheWriterAdapter implements Declarable {
     public MyWriter() {}
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentColocatedPartitionedRegionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentColocatedPartitionedRegionDUnitTest.java
index c526fc2..d66bb84 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentColocatedPartitionedRegionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentColocatedPartitionedRegionDUnitTest.java
@@ -64,6 +64,7 @@ import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.DistributionMessageObserver;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.cache.ColocationLogger;
+import org.apache.geode.internal.cache.DiskRegion;
 import org.apache.geode.internal.cache.InitialImageOperation.RequestImageMessage;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PartitionedRegionHelper;
@@ -192,9 +193,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
 
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setPartitionAttributes(paf.create());
         cache.createRegion("region2", af.create());
         paf.setColocatedWith("region2");
@@ -212,13 +213,13 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
     createData(vm0, 0, NUM_BUCKETS, "c", "region3");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertEquals(vm0Buckets, getBucketList(vm0, "region2"));
     assertEquals(vm0Buckets, getBucketList(vm0, "region3"));
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1Buckets, getBucketList(vm1, "region2"));
     assertEquals(vm1Buckets, getBucketList(vm1, "region3"));
-    Set<Integer> vm2Buckets = getBucketList(vm2, PR_REGION_NAME);
+    Set<Integer> vm2Buckets = getBucketList(vm2, getPartitionedRegionName());
     assertEquals(vm2Buckets, getBucketList(vm2, "region2"));
     assertEquals(vm2Buckets, getBucketList(vm2, "region3"));
 
@@ -236,9 +237,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
 
     // The secondary buckets can be recovered asynchronously,
     // so wait for them to come back.
-    waitForBuckets(vm0, vm0Buckets, PR_REGION_NAME);
+    waitForBuckets(vm0, vm0Buckets, getPartitionedRegionName());
     waitForBuckets(vm0, vm0Buckets, "region2");
-    waitForBuckets(vm1, vm1Buckets, PR_REGION_NAME);
+    waitForBuckets(vm1, vm1Buckets, getPartitionedRegionName());
     waitForBuckets(vm1, vm1Buckets, "region2");
 
     checkData(vm0, 0, NUM_BUCKETS, "a");
@@ -294,8 +295,8 @@ public class PersistentColocatedPartitionedRegionDUnitTest
   private SerializableRunnable createPRsColocatedPairThread =
       new SerializableRunnable("create2PRs") {
         public void run() {
-          createPR(PR_REGION_NAME, true);
-          createPR("region2", PR_REGION_NAME, true);
+          createPR(getPartitionedRegionName(), true);
+          createPR("region2", getPartitionedRegionName(), true);
         }
       };
 
@@ -303,9 +304,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
       new SerializableRunnable("create multiple child PRs") {
         @Override
         public void run() throws Exception {
-          createPR(PR_REGION_NAME, true);
+          createPR(getPartitionedRegionName(), true);
           for (int i = 2; i < numChildPRs + 2; ++i) {
-            createPR("region" + i, PR_REGION_NAME, true);
+            createPR("region" + i, getPartitionedRegionName(), true);
           }
         }
       };
@@ -314,8 +315,8 @@ public class PersistentColocatedPartitionedRegionDUnitTest
       new SerializableRunnable("create PR colocation hierarchy") {
         @Override
         public void run() throws Exception {
-          createPR(PR_REGION_NAME, true);
-          createPR("region2", PR_REGION_NAME, true);
+          createPR(getPartitionedRegionName(), true);
+          createPR("region2", getPartitionedRegionName(), true);
           for (int i = 3; i < numChildPRGenerations + 2; ++i) {
             createPR("region" + i, "region" + (i - 1), true);
           }
@@ -330,7 +331,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           try {
             // Skip creation of first region - expect region2 creation to fail
             // createPR(PR_REGION_NAME, true);
-            createPR("region2", PR_REGION_NAME, true);
+            createPR("region2", getPartitionedRegionName(), true);
           } catch (Exception e) {
             ex = e;
             exClass = e.getClass().toString();
@@ -362,7 +363,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           try {
             // Skip creation of first region - expect region2 creation to fail
             // createPR(PR_REGION_NAME, true);
-            createPR("region2", PR_REGION_NAME, true);
+            createPR("region2", getPartitionedRegionName(), true);
           } catch (Exception e) {
             ex = e;
             exClass = e.getClass().toString();
@@ -394,7 +395,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
 
           AtomicBoolean isDone = new AtomicBoolean(false);
           try {
-            createPR(PR_REGION_NAME, true);
+            createPR(getPartitionedRegionName(), true);
             // Let this thread continue running long enough for the missing region to be logged a
             // couple times.
             // Child regions do not get created by this thread.
@@ -437,14 +438,14 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           List<LogEvent> logEvents = Collections.emptyList();
 
           try {
-            createPR(PR_REGION_NAME, true);
+            createPR(getPartitionedRegionName(), true);
             // Delay creation of second (i.e child) region to see missing colocated region log
             // message (logInterval/2 < delay < logInterval)
             await().atMost(MAX_WAIT, TimeUnit.MILLISECONDS).until(() -> {
               verify(mockAppender, times(1)).append(loggingEventCaptor.capture());
             });
             logEvents = loggingEventCaptor.getAllValues();
-            createPR("region2", PR_REGION_NAME, true);
+            createPR("region2", getPartitionedRegionName(), true);
             // Another delay before exiting the thread to make sure that missing region logging
             // doesn't continue after missing region is created (delay > logInterval)
             await().atMost(logInterval * 2, TimeUnit.MILLISECONDS).until(() -> {
@@ -485,13 +486,13 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           List<LogEvent> logEvents = Collections.emptyList();
           int numLogEvents = 0;
 
-          createPR(PR_REGION_NAME, true);
+          createPR(getPartitionedRegionName(), true);
           // Delay creation of child generation regions to see missing colocated region log message
           // parent region is generation 1, child region is generation 2, grandchild is 3, etc.
           for (int generation = 2; generation < (numChildPRGenerations + 2); ++generation) {
             String childPRName = "region" + generation;
             String colocatedWithRegionName =
-                generation == 2 ? PR_REGION_NAME : "region" + (generation - 1);
+                generation == 2 ? getPartitionedRegionName() : "region" + (generation - 1);
             loggingEventCaptor = ArgumentCaptor.forClass(LogEvent.class);
 
             // delay between starting generations of child regions until the expected missing
@@ -551,7 +552,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           List<LogEvent> logEvents = Collections.emptyList();
           int numLogEvents = 0;
 
-          createPR(PR_REGION_NAME, true);
+          createPR(getPartitionedRegionName(), true);
           // Delay creation of child generation regions to see missing colocated region log message
           for (int regionNum = 2; regionNum < (numChildPRs + 2); ++regionNum) {
             String childPRName = "region" + regionNum;
@@ -570,7 +571,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
             assertEquals("Expected warning messages to be logged.", regionNum - 1, numLogEvents);
 
             // Start the child region
-            createPR(childPRName, PR_REGION_NAME, true);
+            createPR(childPRName, getPartitionedRegionName(), true);
           }
           String logMsg = "";
           logEvents = loggingEventCaptor.getAllValues();
@@ -616,8 +617,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           List<LogEvent> logEvents = Collections.emptyList();
 
           try {
-            createPR(PR_REGION_NAME, true);
-            createPR("region2", PR_REGION_NAME, true); // This child region is never created
+            createPR(getPartitionedRegionName(), true);
+            createPR("region2", getPartitionedRegionName(), true); // This child region is never
+                                                                   // created
             // Let this thread continue running long enough for the missing region to be logged a
             // couple times.
             // Grandchild region does not get created by this thread. (1.5*logInterval < delay <
@@ -640,6 +642,24 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         }
       };
 
+  void checkRecoveredFromDisk(VM vm, final int bucketId, final boolean recoveredLocally) {
+    vm.invoke(new SerializableRunnable("check recovered from disk") {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        PartitionedRegion region = (PartitionedRegion) cache.getRegion(getPartitionedRegionName());
+        DiskRegion disk = region.getRegionAdvisor().getBucket(bucketId).getDiskRegion();
+        if (recoveredLocally) {
+          assertEquals(0, disk.getStats().getRemoteInitializations());
+          assertEquals(1, disk.getStats().getLocalInitializations());
+        } else {
+          assertEquals(1, disk.getStats().getRemoteInitializations());
+          assertEquals(0, disk.getStats().getLocalInitializations());
+        }
+      }
+    });
+  }
+
   private class ColocationLoggerIntervalSetter extends SerializableRunnable {
     private int logInterval;
 
@@ -742,10 +762,10 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "a");
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertFalse(vm0Buckets.isEmpty());
     assertEquals(vm0Buckets, getBucketList(vm0, "region2"));
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1Buckets, getBucketList(vm1, "region2"));
 
     closeCache(vm0);
@@ -778,10 +798,10 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "a");
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertFalse(vm0Buckets.isEmpty());
     assertEquals(vm0Buckets, getBucketList(vm0, "region2"));
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1Buckets, getBucketList(vm1, "region2"));
 
     closeCache(vm0);
@@ -836,10 +856,10 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "a");
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertFalse(vm0Buckets.isEmpty());
     assertEquals(vm0Buckets, getBucketList(vm0, "region2"));
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1Buckets, getBucketList(vm1, "region2"));
 
     closeCache(vm0);
@@ -881,10 +901,10 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "a");
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertFalse(vm0Buckets.isEmpty());
     assertEquals(vm0Buckets, getBucketList(vm0, "region2"));
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1Buckets, getBucketList(vm1, "region2"));
 
     closeCache(vm0);
@@ -931,9 +951,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
     createData(vm0, 0, NUM_BUCKETS, "c", "region2");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertFalse(vm0Buckets.isEmpty());
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertFalse(vm1Buckets.isEmpty());
     for (int i = 2; i < numChildPRs + 2; ++i) {
       String childName = "region" + i;
@@ -986,9 +1006,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
     createData(vm0, 0, NUM_BUCKETS, "c", "region2");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertFalse(vm0Buckets.isEmpty());
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertFalse(vm1Buckets.isEmpty());
     for (int i = 2; i < numChildPRs + 2; ++i) {
       String childName = "region" + i;
@@ -1041,9 +1061,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
     createData(vm0, 0, NUM_BUCKETS, "c", "region3");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertFalse(vm0Buckets.isEmpty());
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertFalse(vm1Buckets.isEmpty());
     for (int i = 2; i < numChildGenerations + 2; ++i) {
       String childName = "region" + i;
@@ -1096,9 +1116,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
     createData(vm0, 0, NUM_BUCKETS, "c", "region3");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertFalse(vm0Buckets.isEmpty());
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertFalse(vm1Buckets.isEmpty());
     for (int i = 2; i < numChildGenerations + 2; ++i) {
       String childName = "region" + i;
@@ -1332,7 +1352,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
       }
     };
     SerializableRunnable createChildPR = getCreateChildPRRunnable();
@@ -1346,18 +1366,18 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "a");
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertEquals(vm0Buckets, getBucketList(vm0, "region2"));
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1Buckets, getBucketList(vm1, "region2"));
-    Set<Integer> vm2Buckets = getBucketList(vm2, PR_REGION_NAME);
+    Set<Integer> vm2Buckets = getBucketList(vm2, getPartitionedRegionName());
     assertEquals(vm2Buckets, getBucketList(vm2, "region2"));
 
-    Set<Integer> vm0PrimaryBuckets = getPrimaryBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0PrimaryBuckets = getPrimaryBucketList(vm0, getPartitionedRegionName());
     assertEquals(vm0PrimaryBuckets, getPrimaryBucketList(vm0, "region2"));
-    Set<Integer> vm1PrimaryBuckets = getPrimaryBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1PrimaryBuckets = getPrimaryBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1PrimaryBuckets, getPrimaryBucketList(vm1, "region2"));
-    Set<Integer> vm2PrimaryBuckets = getPrimaryBucketList(vm2, PR_REGION_NAME);
+    Set<Integer> vm2PrimaryBuckets = getPrimaryBucketList(vm2, getPartitionedRegionName());
     assertEquals(vm2PrimaryBuckets, getPrimaryBucketList(vm2, "region2"));
 
     closeCache(vm0);
@@ -1377,19 +1397,19 @@ public class PersistentColocatedPartitionedRegionDUnitTest
 
     Wait.pause(4000);
 
-    assertEquals(vm0Buckets, getBucketList(vm0, PR_REGION_NAME));
+    assertEquals(vm0Buckets, getBucketList(vm0, getPartitionedRegionName()));
     assertEquals(vm0Buckets, getBucketList(vm0, "region2"));
-    assertEquals(vm1Buckets, getBucketList(vm1, PR_REGION_NAME));
+    assertEquals(vm1Buckets, getBucketList(vm1, getPartitionedRegionName()));
     assertEquals(vm1Buckets, getBucketList(vm1, "region2"));
-    assertEquals(vm2Buckets, getBucketList(vm2, PR_REGION_NAME));
+    assertEquals(vm2Buckets, getBucketList(vm2, getPartitionedRegionName()));
     assertEquals(vm2Buckets, getBucketList(vm2, "region2"));
 
     // primary can differ
-    vm0PrimaryBuckets = getPrimaryBucketList(vm0, PR_REGION_NAME);
+    vm0PrimaryBuckets = getPrimaryBucketList(vm0, getPartitionedRegionName());
     assertEquals(vm0PrimaryBuckets, getPrimaryBucketList(vm0, "region2"));
-    vm1PrimaryBuckets = getPrimaryBucketList(vm1, PR_REGION_NAME);
+    vm1PrimaryBuckets = getPrimaryBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1PrimaryBuckets, getPrimaryBucketList(vm1, "region2"));
-    vm2PrimaryBuckets = getPrimaryBucketList(vm2, PR_REGION_NAME);
+    vm2PrimaryBuckets = getPrimaryBucketList(vm2, getPartitionedRegionName());
     assertEquals(vm2PrimaryBuckets, getPrimaryBucketList(vm2, "region2"));
 
 
@@ -1429,7 +1449,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         AttributesFactory af = new AttributesFactory();
         PartitionAttributesFactory paf = new PartitionAttributesFactory();
         paf.setRedundantCopies(1);
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setPartitionAttributes(paf.create());
         cache.createRegion("region2", af.create());
 
@@ -1463,7 +1483,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
       }
     };
     SerializableRunnable createChildPR = getCreateChildPRRunnable();
@@ -1478,18 +1498,18 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     createData(vm0, 0, NUM_BUCKETS, "a");
     createData(vm0, 0, NUM_BUCKETS, "b", "region2");
 
-    Set<Integer> vm0Buckets = getBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0Buckets = getBucketList(vm0, getPartitionedRegionName());
     assertEquals(vm0Buckets, getBucketList(vm0, "region2"));
-    Set<Integer> vm1Buckets = getBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1Buckets = getBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1Buckets, getBucketList(vm1, "region2"));
-    Set<Integer> vm2Buckets = getBucketList(vm2, PR_REGION_NAME);
+    Set<Integer> vm2Buckets = getBucketList(vm2, getPartitionedRegionName());
     assertEquals(vm2Buckets, getBucketList(vm2, "region2"));
 
-    Set<Integer> vm0PrimaryBuckets = getPrimaryBucketList(vm0, PR_REGION_NAME);
+    Set<Integer> vm0PrimaryBuckets = getPrimaryBucketList(vm0, getPartitionedRegionName());
     assertEquals(vm0PrimaryBuckets, getPrimaryBucketList(vm0, "region2"));
-    Set<Integer> vm1PrimaryBuckets = getPrimaryBucketList(vm1, PR_REGION_NAME);
+    Set<Integer> vm1PrimaryBuckets = getPrimaryBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1PrimaryBuckets, getPrimaryBucketList(vm1, "region2"));
-    Set<Integer> vm2PrimaryBuckets = getPrimaryBucketList(vm2, PR_REGION_NAME);
+    Set<Integer> vm2PrimaryBuckets = getPrimaryBucketList(vm2, getPartitionedRegionName());
     assertEquals(vm2PrimaryBuckets, getPrimaryBucketList(vm2, "region2"));
 
     closeCache(vm2);
@@ -1527,18 +1547,18 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     vm2.invoke(createParentPR);
     // Make sure vm2 hasn't created any buckets in the parent PR yet
     // We don't want any buckets until the child PR is created
-    assertEquals(Collections.emptySet(), getBucketList(vm2, PR_REGION_NAME));
+    assertEquals(Collections.emptySet(), getBucketList(vm2, getPartitionedRegionName()));
     vm2.invoke(createChildPR);
 
     // Now vm2 should have created all of the appropriate buckets.
-    assertEquals(vm2Buckets, getBucketList(vm2, PR_REGION_NAME));
+    assertEquals(vm2Buckets, getBucketList(vm2, getPartitionedRegionName()));
     assertEquals(vm2Buckets, getBucketList(vm2, "region2"));
 
-    vm0PrimaryBuckets = getPrimaryBucketList(vm0, PR_REGION_NAME);
+    vm0PrimaryBuckets = getPrimaryBucketList(vm0, getPartitionedRegionName());
     assertEquals(vm0PrimaryBuckets, getPrimaryBucketList(vm0, "region2"));
-    vm1PrimaryBuckets = getPrimaryBucketList(vm1, PR_REGION_NAME);
+    vm1PrimaryBuckets = getPrimaryBucketList(vm1, getPartitionedRegionName());
     assertEquals(vm1PrimaryBuckets, getPrimaryBucketList(vm1, "region2"));
-    vm2PrimaryBuckets = getPrimaryBucketList(vm2, PR_REGION_NAME);
+    vm2PrimaryBuckets = getPrimaryBucketList(vm2, getPartitionedRegionName());
     assertEquals(vm2PrimaryBuckets, getPrimaryBucketList(vm2, "region2"));
   }
 
@@ -1569,9 +1589,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
 
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setPartitionAttributes(paf.create());
         cache.createRegion("region2", af.create());
 
@@ -1621,14 +1641,14 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
 
         DiskStore ds2 = cache.findDiskStore("disk2");
         if (ds2 == null) {
           ds2 = cache.createDiskStoreFactory().setDiskDirs(getDiskDirs()).create("disk2");
         }
 
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setPartitionAttributes(paf.create());
         af.setDiskStoreName("disk2");
         cache.createRegion("region2", af.create());
@@ -1676,7 +1696,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     closeCache(vm2);
 
     // Wait until redundancy is recovered.
-    waitForRedundancyRecovery(vm0, 1, PR_REGION_NAME);
+    waitForRedundancyRecovery(vm0, 1, getPartitionedRegionName());
     waitForRedundancyRecovery(vm0, 1, "region2");
 
     createData(vm0, 0, NUM_BUCKETS, "b");
@@ -1719,11 +1739,11 @@ public class PersistentColocatedPartitionedRegionDUnitTest
       checkData(vm0, 0, NUM_BUCKETS, "b");
       checkData(vm0, 0, NUM_BUCKETS, "b", "region2");
 
-      waitForRedundancyRecovery(vm0, 1, PR_REGION_NAME);
+      waitForRedundancyRecovery(vm0, 1, getPartitionedRegionName());
       waitForRedundancyRecovery(vm0, 1, "region2");
-      waitForRedundancyRecovery(vm1, 1, PR_REGION_NAME);
+      waitForRedundancyRecovery(vm1, 1, getPartitionedRegionName());
       waitForRedundancyRecovery(vm1, 1, "region2");
-      waitForRedundancyRecovery(vm2, 1, PR_REGION_NAME);
+      waitForRedundancyRecovery(vm2, 1, getPartitionedRegionName());
       waitForRedundancyRecovery(vm2, 1, "region2");
 
       // Make sure we don't have any extra buckets after the restart
@@ -1760,7 +1780,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
       }
     };
 
@@ -1783,7 +1803,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         PartitionAttributesFactory paf = new PartitionAttributesFactory();
         paf.setRedundantCopies(1);
         paf.setRecoveryDelay(0);
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
         af.setPartitionAttributes(paf.create());
@@ -1820,7 +1840,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
       }
     };
 
@@ -1848,7 +1868,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         PartitionAttributesFactory paf = new PartitionAttributesFactory();
         paf.setRedundantCopies(1);
         paf.setRecoveryDelay(0);
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk2");
         af.setPartitionAttributes(paf.create());
@@ -1907,7 +1927,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     closeCache(vm2);
 
     // Wait until redundancy is recovered.
-    waitForRedundancyRecovery(vm0, 1, PR_REGION_NAME);
+    waitForRedundancyRecovery(vm0, 1, getPartitionedRegionName());
     waitForRedundancyRecovery(vm0, 1, "region2");
 
     createData(vm0, 0, NUM_BUCKETS, "b");
@@ -1963,7 +1983,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     // Make sure we can actually use the buckets in the child region.
     createData(vm0, 0, NUM_BUCKETS, "c", "region2");
 
-    waitForRedundancyRecovery(vm0, 1, PR_REGION_NAME);
+    waitForRedundancyRecovery(vm0, 1, getPartitionedRegionName());
     waitForRedundancyRecovery(vm0, 1, "region2");
 
     // Make sure we don't have any extra buckets after the restart
@@ -2012,9 +2032,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
 
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setPartitionAttributes(paf.create());
         cache.createRegion("region2", af.create());
       }
@@ -2143,9 +2163,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         paf.setLocalMaxMemory(0);
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PARTITION);
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
 
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setPartitionAttributes(paf.create());
         cache.createRegion("region2", af.create());
       }
@@ -2166,9 +2186,9 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
 
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setPartitionAttributes(paf.create());
         cache.createRegion("region2", af.create());
       }
@@ -2186,7 +2206,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
     SerializableRunnable doABunchOfPuts = new SerializableRunnable("doABunchOfPuts") {
       public void run() {
         Cache cache = getCache();
-        Region region = cache.getRegion(PR_REGION_NAME);
+        Region region = cache.getRegion(getPartitionedRegionName());
         try {
           for (int i = 0;; i++) {
             try {
@@ -2278,7 +2298,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
       }
     };
 
@@ -2290,7 +2310,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         PartitionAttributesFactory paf = new PartitionAttributesFactory();
         paf.setRedundantCopies(0);
         paf.setRecoveryDelay(0);
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
         af.setPartitionAttributes(paf.create());
@@ -2331,11 +2351,11 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
 
         paf.setRedundantCopies(1);
         paf.setRecoveryDelay(-1);
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
         af.setPartitionAttributes(paf.create());
@@ -2449,7 +2469,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
       }
     };
 
@@ -2466,7 +2486,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
         PartitionAttributesFactory paf = new PartitionAttributesFactory();
         paf.setRedundantCopies(0);
         paf.setRecoveryDelay(0);
-        paf.setColocatedWith(PR_REGION_NAME);
+        paf.setColocatedWith(getPartitionedRegionName());
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk2");
         af.setPartitionAttributes(paf.create());
@@ -2549,7 +2569,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           af.setPartitionAttributes(paf.create());
           af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
           af.setDiskStoreName("disk");
-          cache.createRegion(PR_REGION_NAME, af.create());
+          cache.createRegion(getPartitionedRegionName(), af.create());
         } finally {
           System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "partitionedRegionRetryTimeout",
               String.valueOf(PartitionedRegionHelper.DEFAULT_TOTAL_WAIT_RETRY_ITERATION));
@@ -2567,7 +2587,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           PartitionAttributesFactory paf = new PartitionAttributesFactory();
           paf.setRedundantCopies(0);
           paf.setRecoveryDelay(0);
-          paf.setColocatedWith(PR_REGION_NAME);
+          paf.setColocatedWith(getPartitionedRegionName());
           af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
           af.setDiskStoreName("disk");
           af.setPartitionAttributes(paf.create());
@@ -2614,7 +2634,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           af.setPartitionAttributes(paf.create());
           af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
           af.setDiskStoreName("disk");
-          cache.createRegion(PR_REGION_NAME, af.create());
+          cache.createRegion(getPartitionedRegionName(), af.create());
         } finally {
           System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "partitionedRegionRetryTimeout",
               String.valueOf(PartitionedRegionHelper.DEFAULT_TOTAL_WAIT_RETRY_ITERATION));
@@ -2633,7 +2653,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           PartitionAttributesFactory paf = new PartitionAttributesFactory();
           paf.setRedundantCopies(0);
           paf.setRecoveryDelay(0);
-          paf.setColocatedWith(PR_REGION_NAME);
+          paf.setColocatedWith(getPartitionedRegionName());
           af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
           af.setDiskStoreName("disk");
           af.setPartitionAttributes(paf.create());
@@ -2679,7 +2699,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           af.setPartitionAttributes(paf.create());
           af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
           af.setDiskStoreName("disk");
-          cache.createRegion(PR_REGION_NAME, af.create());
+          cache.createRegion(getPartitionedRegionName(), af.create());
         } finally {
           System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "partitionedRegionRetryTimeout",
               String.valueOf(PartitionedRegionHelper.DEFAULT_TOTAL_WAIT_RETRY_ITERATION));
@@ -2697,7 +2717,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
           PartitionAttributesFactory paf = new PartitionAttributesFactory();
           paf.setRedundantCopies(0);
           paf.setRecoveryDelay(0);
-          paf.setColocatedWith(PR_REGION_NAME);
+          paf.setColocatedWith(getPartitionedRegionName());
           af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
           af.setDiskStoreName("disk");
           af.setPartitionAttributes(paf.create());
@@ -2856,7 +2876,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
       @Override
       public void run() {
         Cache cache = getCache();
-        Region region = cache.getRegion(PR_REGION_NAME);
+        Region region = cache.getRegion(getPartitionedRegionName());
 
         for (int i = 0; i < NUM_BUCKETS; i++) {
           assertEquals("For key " + i, "a", region.get(i));
@@ -2937,7 +2957,7 @@ public class PersistentColocatedPartitionedRegionDUnitTest
       @Override
       public void run() {
         Cache cache = getCache();
-        Region region = cache.getRegion(PR_REGION_NAME);
+        Region region = cache.getRegion(getPartitionedRegionName());
 
         for (int i = 0; i < NUM_BUCKETS; i++) {
           assertEquals("For key " + i, "a", region.get(i));
@@ -2949,8 +2969,8 @@ public class PersistentColocatedPartitionedRegionDUnitTest
 
       public void run() {
         Cache cache = getCache();
-        LogWriterUtils.getLogWriter().info("creating data in " + PR_REGION_NAME);
-        Region region = cache.getRegion(PR_REGION_NAME);
+        LogWriterUtils.getLogWriter().info("creating data in " + getPartitionedRegionName());
+        Region region = cache.getRegion(getPartitionedRegionName());
 
         for (int i = 0; i < NUM_BUCKETS; i++) {
           region.put(i, "c");
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
index 014eea8..dad51c4 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionDUnitTest.java
@@ -32,7 +32,6 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.awaitility.Awaitility;
 import org.junit.Ignore;
@@ -77,6 +76,7 @@ import org.apache.geode.distributed.internal.DistributionMessage;
 import org.apache.geode.distributed.internal.DistributionMessageObserver;
 import org.apache.geode.distributed.internal.ReplyException;
 import org.apache.geode.internal.AvailablePort;
+import org.apache.geode.internal.cache.DiskRegion;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InitialImageOperation.RequestImageMessage;
 import org.apache.geode.internal.cache.PartitionedRegion;
@@ -169,7 +169,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     } catch (RMIException exp) {
       assertTrue(exp.getCause() instanceof IllegalStateException);
       IllegalStateException ise = (IllegalStateException) exp.getCause();
-      Object[] prms = new Object[] {"/" + PR_REGION_NAME, 2, 5};
+      Object[] prms = new Object[] {"/" + getPartitionedRegionName(), 2, 5};
       assertTrue(ise.getMessage().contains(
           LocalizedStrings.PartitionedRegion_FOR_REGION_0_TotalBucketNum_1_SHOULD_NOT_BE_CHANGED_Previous_Configured_2
               .toString(prms)));
@@ -181,7 +181,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     } catch (RMIException exp) {
       assertTrue(exp.getCause() instanceof IllegalStateException);
       IllegalStateException ise = (IllegalStateException) exp.getCause();
-      Object[] prms = new Object[] {"/" + PR_REGION_NAME, 10, 5};
+      Object[] prms = new Object[] {"/" + getPartitionedRegionName(), 10, 5};
       assertTrue(ise.getMessage().contains(
           LocalizedStrings.PartitionedRegion_FOR_REGION_0_TotalBucketNum_1_SHOULD_NOT_BE_CHANGED_Previous_Configured_2
               .toString(prms)));
@@ -213,7 +213,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
         af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
         af.setDiskStoreName("disk");
         RegionAttributes attr = af.create();
-        cache.createRegion(PR_REGION_NAME, attr);
+        cache.createRegion(getPartitionedRegionName(), attr);
       }
     };
 
@@ -292,7 +292,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
       vm0.invoke(new SerializableRunnable() {
         public void run() {
           Cache cache = getCache();
-          Region region = cache.getRegion(PR_REGION_NAME);
+          Region region = cache.getRegion(getPartitionedRegionName());
           try {
             for (int i = 0; i < numBuckets; i++) {
               region.put(i, new BadSerializer());
@@ -328,6 +328,24 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     }
   }
 
+  void checkRecoveredFromDisk(VM vm, final int bucketId, final boolean recoveredLocally) {
+    vm.invoke(new SerializableRunnable("check recovered from disk") {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        PartitionedRegion region = (PartitionedRegion) cache.getRegion(getPartitionedRegionName());
+        DiskRegion disk = region.getRegionAdvisor().getBucket(bucketId).getDiskRegion();
+        if (recoveredLocally) {
+          assertEquals(0, disk.getStats().getRemoteInitializations());
+          assertEquals(1, disk.getStats().getLocalInitializations());
+        } else {
+          assertEquals(1, disk.getStats().getRemoteInitializations());
+          assertEquals(0, disk.getStats().getLocalInitializations());
+        }
+      }
+    });
+  }
+
   public static class BadSerializer implements DataSerializable {
 
     public BadSerializer() {
@@ -610,7 +628,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     vm0.invoke(new SerializableRunnable("Test ways to read") {
       public void run() {
         Cache cache = getCache();
-        Region region = cache.getRegion(PR_REGION_NAME);
+        Region region = cache.getRegion(getPartitionedRegionName());
 
         try {
           FunctionService.onRegion(region).execute(new TestFunction());
@@ -652,7 +670,8 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
         }
 
         try {
-          cache.getQueryService().newQuery("select * from /" + PR_REGION_NAME).execute();
+          cache.getQueryService().newQuery("select * from /" + getPartitionedRegionName())
+              .execute();
           fail("Should not have been able to read from missing buckets!");
         } catch (PartitionOfflineException e) {
           // expected
@@ -1045,6 +1064,9 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
   /**
    * This test this case we replace buckets where are offline on A by creating them on C We then
    * shutdown C and restart A, which recovers those buckets
+   * <p>
+   *
+   * TRAC 41340: data inconsistency after disk recovery from persistent PR
    */
   @Test
   public void testBug41340() throws Throwable {
@@ -1160,6 +1182,9 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
   /**
    * Test that we don't record our old member ID as offline, preventing redundancy recovery in the
    * future.
+   * <p>
+   *
+   * TRAC 41341: Redundancy not restored after reinitializing after locally destroying persistent PR
    */
   @Test
   public void testBug41341() {
@@ -1405,7 +1430,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
         paf.setLocalMaxMemory(0);
         af.setPartitionAttributes(paf.create());
         af.setDataPolicy(DataPolicy.PARTITION);
-        cache.createRegion(PR_REGION_NAME, af.create());
+        cache.createRegion(getPartitionedRegionName(), af.create());
 
         CacheServer server = cache.addCacheServer();
         server.setPort(AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET));
@@ -1438,7 +1463,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
           af.setDataPolicy(DataPolicy.NORMAL);
           af.setScope(Scope.LOCAL);
           af.setPoolName("pool");
-          Region region = cache.createRegion(PR_REGION_NAME, af.create());
+          Region region = cache.createRegion(getPartitionedRegionName(), af.create());
           try {
             region.registerInterestRegex(".*");
           } catch (ServerOperationException e) {
@@ -1468,7 +1493,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
         EvictionAttributes.createLRUEntryAttributes(50, EvictionAction.OVERFLOW_TO_DISK));
     rf.setDiskDirs(getDiskDirs());
 
-    Region region = rf.create(PR_REGION_NAME);
+    Region region = rf.create(getPartitionedRegionName());
     region.get(0);
     cache.getDistributedSystem().disconnect();
     // cache.close();
@@ -1490,8 +1515,8 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
         DistributionMessageObserver.setInstance(new DistributionMessageObserver() {
 
           @Override
-          public void beforeSendMessage(DistributionManager dm, DistributionMessage msg) {
-            if (msg instanceof ManageBucketReplyMessage) {
+          public void beforeSendMessage(DistributionManager dm, DistributionMessage message) {
+            if (message instanceof ManageBucketReplyMessage) {
               Cache cache = getCache();
               disconnectFromDS();
 
@@ -1554,18 +1579,18 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     createNestedPR(vm1);
     createNestedPR(vm2);
 
-    createData(vm0, 0, numBuckets, "a", "parent1/" + PR_REGION_NAME);
-    createData(vm0, 0, numBuckets, "b", "parent2/" + PR_REGION_NAME);
-    checkData(vm2, 0, numBuckets, "a", "parent1/" + PR_REGION_NAME);
-    checkData(vm2, 0, numBuckets, "b", "parent2/" + PR_REGION_NAME);
+    createData(vm0, 0, numBuckets, "a", "parent1/" + getPartitionedRegionName());
+    createData(vm0, 0, numBuckets, "b", "parent2/" + getPartitionedRegionName());
+    checkData(vm2, 0, numBuckets, "a", "parent1/" + getPartitionedRegionName());
+    checkData(vm2, 0, numBuckets, "b", "parent2/" + getPartitionedRegionName());
 
-    Set<Integer> vm1_0Buckets = getBucketList(vm0, "parent1/" + PR_REGION_NAME);
-    Set<Integer> vm1_1Buckets = getBucketList(vm1, "parent1/" + PR_REGION_NAME);
-    Set<Integer> vm1_2Buckets = getBucketList(vm2, "parent1/" + PR_REGION_NAME);
+    Set<Integer> vm1_0Buckets = getBucketList(vm0, "parent1/" + getPartitionedRegionName());
+    Set<Integer> vm1_1Buckets = getBucketList(vm1, "parent1/" + getPartitionedRegionName());
+    Set<Integer> vm1_2Buckets = getBucketList(vm2, "parent1/" + getPartitionedRegionName());
 
-    Set<Integer> vm2_0Buckets = getBucketList(vm0, "parent2/" + PR_REGION_NAME);
-    Set<Integer> vm2_1Buckets = getBucketList(vm1, "parent2/" + PR_REGION_NAME);
-    Set<Integer> vm2_2Buckets = getBucketList(vm2, "parent2/" + PR_REGION_NAME);
+    Set<Integer> vm2_0Buckets = getBucketList(vm0, "parent2/" + getPartitionedRegionName());
+    Set<Integer> vm2_1Buckets = getBucketList(vm1, "parent2/" + getPartitionedRegionName());
+    Set<Integer> vm2_2Buckets = getBucketList(vm2, "parent2/" + getPartitionedRegionName());
 
     closeCache(vm0);
     closeCache(vm1);
@@ -1583,20 +1608,20 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     async1.getResult();
     async2.getResult();
 
-    assertEquals(vm1_0Buckets, getBucketList(vm0, "parent1/" + PR_REGION_NAME));
-    assertEquals(vm1_1Buckets, getBucketList(vm1, "parent1/" + PR_REGION_NAME));
-    assertEquals(vm1_2Buckets, getBucketList(vm2, "parent1/" + PR_REGION_NAME));
+    assertEquals(vm1_0Buckets, getBucketList(vm0, "parent1/" + getPartitionedRegionName()));
+    assertEquals(vm1_1Buckets, getBucketList(vm1, "parent1/" + getPartitionedRegionName()));
+    assertEquals(vm1_2Buckets, getBucketList(vm2, "parent1/" + getPartitionedRegionName()));
 
-    assertEquals(vm2_0Buckets, getBucketList(vm0, "parent2/" + PR_REGION_NAME));
-    assertEquals(vm2_1Buckets, getBucketList(vm1, "parent2/" + PR_REGION_NAME));
-    assertEquals(vm2_2Buckets, getBucketList(vm2, "parent2/" + PR_REGION_NAME));
+    assertEquals(vm2_0Buckets, getBucketList(vm0, "parent2/" + getPartitionedRegionName()));
+    assertEquals(vm2_1Buckets, getBucketList(vm1, "parent2/" + getPartitionedRegionName()));
+    assertEquals(vm2_2Buckets, getBucketList(vm2, "parent2/" + getPartitionedRegionName()));
 
-    checkData(vm0, 0, numBuckets, "a", "parent1/" + PR_REGION_NAME);
-    checkData(vm0, 0, numBuckets, "b", "parent2/" + PR_REGION_NAME);
-    createData(vm1, numBuckets, 113, "c", "parent1/" + PR_REGION_NAME);
-    createData(vm1, numBuckets, 113, "d", "parent2/" + PR_REGION_NAME);
-    checkData(vm2, numBuckets, 113, "c", "parent1/" + PR_REGION_NAME);
-    checkData(vm2, numBuckets, 113, "d", "parent2/" + PR_REGION_NAME);
+    checkData(vm0, 0, numBuckets, "a", "parent1/" + getPartitionedRegionName());
+    checkData(vm0, 0, numBuckets, "b", "parent2/" + getPartitionedRegionName());
+    createData(vm1, numBuckets, 113, "c", "parent1/" + getPartitionedRegionName());
+    createData(vm1, numBuckets, 113, "d", "parent2/" + getPartitionedRegionName());
+    checkData(vm2, numBuckets, 113, "c", "parent1/" + getPartitionedRegionName());
+    checkData(vm2, numBuckets, 113, "d", "parent2/" + getPartitionedRegionName());
   }
 
   @Test
@@ -1616,7 +1641,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
 
       public Object call() {
         Cache cache = getCache();
-        Region region = cache.getRegion(PR_REGION_NAME);
+        Region region = cache.getRegion(getPartitionedRegionName());
 
         int i = 0;
         while (true) {
@@ -1636,7 +1661,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
     SerializableCallable waitForIntValue = new SerializableCallable() {
       public Object call() {
         Cache cache = getCache();
-        Region region = cache.getRegion(PR_REGION_NAME);
+        Region region = cache.getRegion(getPartitionedRegionName());
         // The value is initialized as a String so wait
         // for it to be changed to an Integer.
         await().atMost(60, SECONDS).until(() -> {
@@ -1669,7 +1694,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
 
       public Object call() {
         Cache cache = getCache();
-        Region region = cache.getRegion(PR_REGION_NAME);
+        Region region = cache.getRegion(getPartitionedRegionName());
         int value = (Integer) region.get(0);
         return value;
       }
@@ -1687,8 +1712,10 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
    * member A that it hosts the bucket 3. Member A crashes 4. Member B destroys the bucket and
    * throws a partition offline exception, because it wasn't able to complete initialization. 5.
    * Member A recovers, and gets stuck waiting for member B.
+   * <p>
    *
-   * @throws Throwable
+   * TRAC 42226: recycled VM hangs during re-start while waiting for Partition to come online (after
+   * Controller VM sees unexpected PartitionOffLineException while doing ops)
    */
   @Category(FlakyTest.class) // GEODE-1208: time sensitive, multiple non-thread-safe test hooks,
                              // async actions
@@ -1834,7 +1861,7 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
 
           Cache cache = getCache();
           RegionAttributes attr = getPersistentPRAttributes(redundancy, -1, cache, 113, true);
-          cache.createRegion(PR_REGION_NAME, attr);
+          cache.createRegion(getPartitionedRegionName(), attr);
         }
       };
 
@@ -2129,15 +2156,15 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
 
     // create some buckets
     createData(vm0, 0, 2, "a");
-    createData(vm0, 0, 2, "a", PR_CHILD_REGION_NAME);
-    closePR(vm0, PR_CHILD_REGION_NAME);
+    createData(vm0, 0, 2, "a", getChildRegionName());
+    closePR(vm0, getChildRegionName());
     closePR(vm0);
 
     // createPR(vm1, 1);
     createCoLocatedPR(vm1, 1, false);
     // create an overlapping bucket
     createData(vm1, 2, 4, "a");
-    createData(vm1, 2, 4, "a", PR_CHILD_REGION_NAME);
+    createData(vm1, 2, 4, "a", getChildRegionName());
 
     IgnoredException[] expectVm0 =
         {IgnoredException.addIgnoredException("ConflictingPersistentDataException", vm0),
@@ -2350,19 +2377,18 @@ public class PersistentPartitionedRegionDUnitTest extends PersistentPartitionedR
   private void createChildPR(VM vm) {
     vm.invoke(() -> {
       PartitionAttributes PRatts =
-          new PartitionAttributesFactory().setColocatedWith(PR_REGION_NAME).create();
+          new PartitionAttributesFactory().setColocatedWith(getPartitionedRegionName()).create();
       PartitionedRegion child =
           (PartitionedRegion) PartitionedRegionTestHelper.createPartionedRegion("CHILD", PRatts);
     });
   }
 
-  private static final class RecoveryObserver
-      extends InternalResourceManager.ResourceObserverAdapter {
+  private final class RecoveryObserver extends InternalResourceManager.ResourceObserverAdapter {
     final CountDownLatch recoveryDone = new CountDownLatch(1);
 
     @Override
     public void rebalancingOrRecoveryFinished(Region region) {
-      if (region.getName().equals(PR_REGION_NAME)) {
+      if (region.getName().equals(getPartitionedRegionName())) {
         recoveryDone.countDown();
       }
     }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
index d401b7f..1b2732e 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionTestBase.java
@@ -14,12 +14,16 @@
  */
 package org.apache.geode.internal.cache.partitioned;
 
-import static org.apache.geode.test.dunit.Assert.assertEquals;
+import static org.apache.commons.io.FileUtils.listFiles;
+import static org.apache.commons.io.filefilter.DirectoryFileFilter.DIRECTORY;
+import static org.apache.geode.admin.AdminDistributedSystemFactory.defineDistributedSystem;
+import static org.apache.geode.admin.AdminDistributedSystemFactory.getDistributedSystem;
+import static org.apache.geode.test.dunit.Invoke.invokeInEveryVM;
+import static org.assertj.core.api.Assertions.assertThat;
 
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.IOException;
-import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.net.InetAddress;
 import java.util.Collection;
@@ -28,12 +32,11 @@ import java.util.TreeSet;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.io.filefilter.DirectoryFileFilter;
 import org.apache.commons.io.filefilter.RegexFileFilter;
+import org.apache.logging.log4j.Logger;
+import org.junit.Before;
 
 import org.apache.geode.admin.AdminDistributedSystem;
-import org.apache.geode.admin.AdminDistributedSystemFactory;
 import org.apache.geode.admin.AdminException;
 import org.apache.geode.admin.DistributedSystemConfig;
 import org.apache.geode.cache.AttributesFactory;
@@ -43,361 +46,224 @@ import org.apache.geode.cache.DiskStore;
 import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionAttributes;
-import org.apache.geode.cache.control.RebalanceFactory;
 import org.apache.geode.cache.partition.PartitionRegionHelper;
 import org.apache.geode.cache.partition.PartitionRegionInfo;
 import org.apache.geode.cache.persistence.ConflictingPersistentDataException;
 import org.apache.geode.cache.persistence.PersistentID;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.internal.cache.BackupUtil;
 import org.apache.geode.internal.cache.DiskRegion;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PartitionedRegionDataStore;
+import org.apache.geode.internal.cache.backup.BackupUtil;
 import org.apache.geode.internal.cache.control.InternalResourceManager;
 import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceObserver;
-import org.apache.geode.internal.cache.persistence.PersistenceAdvisor;
+import org.apache.geode.internal.cache.control.InternalResourceManager.ResourceObserverAdapter;
 import org.apache.geode.internal.cache.persistence.PersistenceAdvisorImpl;
+import org.apache.geode.internal.cache.persistence.PersistenceAdvisorImpl.PersistenceAdvisorObserver;
 import org.apache.geode.internal.cache.persistence.PersistentMemberID;
+import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.management.BackupStatus;
 import org.apache.geode.management.ManagementException;
-import org.apache.geode.test.dunit.Assert;
 import org.apache.geode.test.dunit.AsyncInvocation;
-import org.apache.geode.test.dunit.Invoke;
-import org.apache.geode.test.dunit.LogWriterUtils;
-import org.apache.geode.test.dunit.SerializableCallable;
 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;
 
+@SuppressWarnings("serial")
 public abstract class PersistentPartitionedRegionTestBase extends JUnit4CacheTestCase {
+  private static final Logger logger = LogService.getLogger();
 
-  public static String PR_REGION_NAME = "region";
-  public static String PR_CHILD_REGION_NAME = "childRegion";
   // This must be bigger than the dunit ack-wait-threshold for the revoke
   // tests. The command line is setting the ack-wait-threshold to be
   // 60 seconds.
   private static final int MAX_WAIT = 70 * 1000;
 
-  /*
-   * (non-Javadoc) Set the region name for this test so that multiple subclasses of this test base
-   * do not conflict with one another during parallel dunit runs
-   *
-   * @see dunit.DistributedTestCase#setUp()
-   */
-  @Override
-  public final void postSetUp() throws Exception {
-    disconnectAllFromDS();
-    Invoke.invokeInEveryVM(PersistentPartitionedRegionTestBase.class, "setRegionName",
-        new Object[] {getUniqueName()});
-    setRegionName(getUniqueName());
-    postSetUpPersistentPartitionedRegionTestBase();
-  }
+  private static final int NUM_BUCKETS = 113;
+  private static final String CHILD_REGION_NAME = "childRegion";
 
-  protected void postSetUpPersistentPartitionedRegionTestBase() throws Exception {}
+  private String partitionedRegionName;
 
-  public static void setRegionName(String testName) {
-    PR_REGION_NAME = testName + "Region";
-  }
+  @Before
+  public void setUpPersistentPartitionedRegionTestBase() throws Exception {
+    disconnectAllFromDS();
 
-  protected void checkRecoveredFromDisk(VM vm, final int bucketId, final boolean recoveredLocally) {
-    vm.invoke(new SerializableRunnable("check recovered from disk") {
-      public void run() {
-        Cache cache = getCache();
-        PartitionedRegion region = (PartitionedRegion) cache.getRegion(PR_REGION_NAME);
-        DiskRegion disk = region.getRegionAdvisor().getBucket(bucketId).getDiskRegion();
-        if (recoveredLocally) {
-          assertEquals(0, disk.getStats().getRemoteInitializations());
-          assertEquals(1, disk.getStats().getLocalInitializations());
-        } else {
-          assertEquals(1, disk.getStats().getRemoteInitializations());
-          assertEquals(0, disk.getStats().getLocalInitializations());
-        }
-      }
-    });
-  }
+    partitionedRegionName = getUniqueName() + "Region";
+    invokeInEveryVM(() -> partitionedRegionName = getUniqueName() + "Region");
 
-  protected void fakeCleanShutdown(VM vm, final int bucketId) {
-    vm.invoke(new SerializableRunnable("mark clean") {
-      public void run() {
-        Cache cache = getCache();
-        PartitionedRegion region = (PartitionedRegion) cache.getRegion(PR_REGION_NAME);
-        DiskRegion disk = region.getRegionAdvisor().getBucket(bucketId).getDiskRegion();
-        for (PersistentMemberID id : disk.getOnlineMembers()) {
-          disk.memberOfflineAndEqual(id);
-        }
-        for (PersistentMemberID id : disk.getOfflineMembers()) {
-          disk.memberOfflineAndEqual(id);
-        }
-        cache.close();
-      }
-    });
+    postSetUpPersistentPartitionedRegionTestBase();
   }
 
-  private PersistentMemberID getPersistentID(VM vm, final int bucketId) {
-    Object id = vm.invoke(new SerializableCallable("get bucket persistent id") {
-      public Object call() {
-        Cache cache = getCache();
-        PartitionedRegion region = (PartitionedRegion) cache.getRegion(PR_REGION_NAME);
-        PersistenceAdvisor advisor =
-            region.getRegionAdvisor().getBucket(bucketId).getPersistenceAdvisor();
-        return advisor.getPersistentID();
-      }
-    });
-
-    return (PersistentMemberID) id;
+  protected void postSetUpPersistentPartitionedRegionTestBase() throws Exception {
+    // override as needed
   }
 
-  private void forceRecovery(VM vm) {
-    vm.invoke(new SerializableRunnable("force recovery") {
-      public void run() {
-        Cache cache = getCache();
-        RebalanceFactory rf = cache.getResourceManager().createRebalanceFactory();
-        try {
-          rf.start().getResults();
-        } catch (Exception e) {
-          Assert.fail("interupted", e);
-        }
+  void fakeCleanShutdown(final VM vm, final int bucketId) {
+    vm.invoke("fakeCleanShutdown", () -> {
+      Cache cache = getCache();
+      PartitionedRegion region = (PartitionedRegion) cache.getRegion(getPartitionedRegionName());
+      DiskRegion disk = region.getRegionAdvisor().getBucket(bucketId).getDiskRegion();
+      for (PersistentMemberID id : disk.getOnlineMembers()) {
+        disk.memberOfflineAndEqual(id);
+      }
+      for (PersistentMemberID id : disk.getOfflineMembers()) {
+        disk.memberOfflineAndEqual(id);
       }
+      cache.close();
     });
   }
 
-  protected void checkData(VM vm0, final int startKey, final int endKey, final String value) {
-    checkData(vm0, startKey, endKey, value, PR_REGION_NAME);
+  protected void checkData(VM vm, final int startKey, final int endKey, final String value) {
+    checkData(vm, startKey, endKey, value, getPartitionedRegionName());
   }
 
-  protected void checkData(VM vm0, final int startKey, final int endKey, final String value,
+  protected void checkData(final VM vm, final int startKey, final int endKey, final String value,
       final String regionName) {
-    SerializableRunnable checkData = new SerializableRunnable("CheckData") {
-
-      public void run() {
-        Cache cache = getCache();
-        Region region = cache.getRegion(regionName);
-
-        for (int i = startKey; i < endKey; i++) {
-          assertEquals("For key " + i, value, region.get(i));
-        }
+    vm.invoke("checkData", () -> {
+      Region region = getCache().getRegion(regionName);
+      for (int i = startKey; i < endKey; i++) {
+        assertThat(region.get(i)).isEqualTo(value);
       }
-    };
-
-    vm0.invoke(checkData);
+    });
   }
 
-  protected void removeData(VM vm, final int startKey, final int endKey) {
-    SerializableRunnable createData = new SerializableRunnable() {
-
-      public void run() {
-        Cache cache = getCache();
-        Region region = cache.getRegion(PR_REGION_NAME);
-
-        for (int i = startKey; i < endKey; i++) {
-          region.destroy(i);
-        }
+  void removeData(final VM vm, final int startKey, final int endKey) {
+    vm.invoke("removeData", () -> {
+      Region region = getCache().getRegion(getPartitionedRegionName());
+      for (int i = startKey; i < endKey; i++) {
+        region.destroy(i);
       }
-    };
-    vm.invoke(createData);
+    });
   }
 
-  protected void createData(VM vm, final int startKey, final int endKey, final String value) {
-    LogWriterUtils.getLogWriter().info("createData invoked.  PR_REGION_NAME is " + PR_REGION_NAME);
-    createData(vm, startKey, endKey, value, PR_REGION_NAME);
+  protected void createData(final VM vm, final int startKey, final int endKey, final String value) {
+    createData(vm, startKey, endKey, value, getPartitionedRegionName());
   }
 
-  protected void createData(VM vm, final int startKey, final int endKey, final String value,
+  protected void createData(final VM vm, final int startKey, final int endKey, final String value,
       final String regionName) {
-    SerializableRunnable createData = new SerializableRunnable("createData") {
-
-      public void run() {
-        Cache cache = getCache();
-        cache.getLogger().info("creating data in " + regionName);
-        Region region = cache.getRegion(regionName);
-
-        for (int i = startKey; i < endKey; i++) {
-          region.put(i, value);
-        }
+    vm.invoke("createData", () -> {
+      Region region = getCache().getRegion(regionName);
+      for (int i = startKey; i < endKey; i++) {
+        region.put(i, value);
       }
-    };
-    vm.invoke(createData);
+    });
   }
 
-  protected void closeCache(VM vm0) {
-    SerializableRunnable close = new SerializableRunnable("Close Cache") {
-      public void run() {
-        Cache cache = getCache();
-        cache.close();
-      }
-    };
-
-    vm0.invoke(close);
+  protected void closeCache(final VM vm) {
+    vm.invoke("closeCache", () -> getCache().close());
   }
 
-  protected AsyncInvocation closeCacheAsync(VM vm0) {
-    SerializableRunnable close = new SerializableRunnable() {
-      public void run() {
-        Cache cache = getCache();
-        cache.close();
-      }
-    };
-
-    return vm0.invokeAsync(close);
+  AsyncInvocation closeCacheAsync(final VM vm) {
+    return vm.invokeAsync("closeCacheAsync", () -> getCache().close());
   }
 
-  protected void closePR(VM vm0) {
-    closePR(vm0, PR_REGION_NAME);
+  void closePR(final VM vm) {
+    closePR(vm, getPartitionedRegionName());
   }
 
-  protected void closePR(VM vm0, String regionName) {
-    SerializableRunnable close = new SerializableRunnable("Close PR") {
-      public void run() {
-        Cache cache = getCache();
-        Region region = cache.getRegion(regionName);
-        region.close();
-      }
-    };
-
-    vm0.invoke(close);
+  void closePR(final VM vm, final String regionName) {
+    vm.invoke("closePR", () -> getCache().getRegion(regionName).close());
   }
 
-  protected void destroyPR(VM vm0) {
-    destroyPR(vm0, PR_REGION_NAME);
+  void destroyPR(final VM vm) {
+    destroyPR(vm, getPartitionedRegionName());
   }
 
-  protected void destroyPR(VM vm0, String regionName) {
-    SerializableRunnable destroy = new SerializableRunnable("Destroy PR") {
-      public void run() {
-        Cache cache = getCache();
-        Region region = cache.getRegion(regionName);
-        region.localDestroyRegion();
-      }
-    };
-
-    vm0.invoke(destroy);
+  private void destroyPR(final VM vm, String regionName) {
+    vm.invoke("destroyPR", () -> getCache().getRegion(regionName).localDestroyRegion());
   }
 
-  protected void localDestroyPR(VM vm0) {
-    SerializableRunnable destroyPR = new SerializableRunnable("destroy pr") {
-
-      public void run() {
-        Cache cache = getCache();
-        Region region = cache.getRegion(PR_REGION_NAME);
-        region.localDestroyRegion();
-      }
-    };
-    vm0.invoke(destroyPR);
+  void localDestroyPR(final VM vm) {
+    vm.invoke("localDestroyPR",
+        () -> getCache().getRegion(getPartitionedRegionName()).localDestroyRegion());
   }
 
-  protected void createPR(VM vm0, final int redundancy, final int recoveryDelay, int numBuckets) {
-    SerializableRunnable createPR = getCreatePRRunnable(redundancy, recoveryDelay, numBuckets);
-
-    vm0.invoke(createPR);
+  protected void createPR(final VM vm, final int redundancy, final int recoveryDelay,
+      final int numBuckets) {
+    vm.invoke(getCreatePRRunnable(redundancy, recoveryDelay, numBuckets));
   }
 
-  protected void createPR(VM vm0, final int redundancy, final int recoveryDelay, int numBuckets,
-      boolean synchronous) {
-    SerializableRunnable createPR =
-        getCreatePRRunnable(redundancy, recoveryDelay, numBuckets, synchronous);
-
-    vm0.invoke(createPR);
+  protected void createPR(final VM vm, final int redundancy, final int recoveryDelay,
+      final int numBuckets, final boolean synchronous) {
+    vm.invoke(getCreatePRRunnable(redundancy, recoveryDelay, numBuckets, synchronous));
   }
 
-  protected void createPR(VM vm0, final int redundancy, final int recoveryDelay) {
-    SerializableRunnable createPR = getCreatePRRunnable(redundancy, recoveryDelay);
-
-    vm0.invoke(createPR);
+  protected void createPR(final VM vm, final int redundancy, final int recoveryDelay) {
+    vm.invoke(getCreatePRRunnable(redundancy, recoveryDelay));
   }
 
-  protected void createPR(VM vm0, final int redundancy) {
-    SerializableRunnable createPR = getCreatePRRunnable(redundancy, -1);
-
-    vm0.invoke(createPR);
+  protected void createPR(final VM vm, final int redundancy) {
+    vm.invoke(getCreatePRRunnable(redundancy, -1));
   }
 
-  protected void createNestedPR(VM vm) {
-    SerializableRunnable createPR = getNestedPRRunnable();
-    vm.invoke(createPR);
+  void createNestedPR(final VM vm) {
+    vm.invoke(getNestedPRRunnable());
   }
 
-  protected AsyncInvocation createNestedPRAsync(VM vm) {
-    SerializableRunnable createPR = getNestedPRRunnable();
-    return vm.invokeAsync(createPR);
+  AsyncInvocation createNestedPRAsync(final VM vm) {
+    return vm.invokeAsync(getNestedPRRunnable());
   }
 
   private SerializableRunnable getNestedPRRunnable() {
-    SerializableRunnable createPR = new SerializableRunnable("create pr") {
-
+    return new SerializableRunnable("getNestedPRRunnable") {
+      @Override
       public void run() {
-        Cache cache = getCache();
 
         // Wait for both nested PRs to be created
         final CountDownLatch recoveryDone = new CountDownLatch(2);
 
-        ResourceObserver observer = new InternalResourceManager.ResourceObserverAdapter() {
+        ResourceObserver observer = new ResourceObserverAdapter() {
           @Override
-          public void recoveryFinished(Region region) {
+          public void recoveryFinished(final Region region) {
             recoveryDone.countDown();
           }
         };
         InternalResourceManager.setResourceObserver(observer);
 
-        DiskStore ds = cache.findDiskStore("disk");
-        if (ds == null) {
-          ds = cache.createDiskStoreFactory().setDiskDirs(getDiskDirs()).create("disk");
-        }
-        Region parent1;
-        {
-          AttributesFactory af = new AttributesFactory();
-          af.setDataPolicy(DataPolicy.REPLICATE);
-          parent1 = cache.createRegion("parent1", af.create());
-        }
-        Region parent2;
-        {
-          AttributesFactory af = new AttributesFactory();
-          af.setDataPolicy(DataPolicy.REPLICATE);
-          parent2 = cache.createRegion("parent2", af.create());
-        }
-        {
-          AttributesFactory af = new AttributesFactory();
-          PartitionAttributesFactory paf = new PartitionAttributesFactory();
-          paf.setRedundantCopies(1);
-          af.setPartitionAttributes(paf.create());
-          af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
-          af.setDiskStoreName("disk");
-          parent1.createSubregion(PR_REGION_NAME, af.create());
-        }
-        {
-          AttributesFactory af = new AttributesFactory();
-          PartitionAttributesFactory paf = new PartitionAttributesFactory();
-          paf.setRedundantCopies(1);
-          af.setPartitionAttributes(paf.create());
-          af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
-          af.setDiskStoreName("disk");
-          parent2.createSubregion(PR_REGION_NAME, af.create());
+        Cache cache = getCache();
+        DiskStore diskStore = cache.findDiskStore("disk");
+        if (diskStore == null) {
+          diskStore = cache.createDiskStoreFactory().setDiskDirs(getDiskDirs()).create("disk");
         }
 
+        AttributesFactory attributesFactory = new AttributesFactory();
+        attributesFactory.setDataPolicy(DataPolicy.REPLICATE);
+
+        Region parent1 = cache.createRegion("parent1", attributesFactory.create());
+        Region parent2 = cache.createRegion("parent2", attributesFactory.create());
+
+        attributesFactory.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
+        attributesFactory.setDiskStoreName("disk");
+
+        PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory();
+        partitionAttributesFactory.setRedundantCopies(1);
+        attributesFactory.setPartitionAttributes(partitionAttributesFactory.create());
+
+        parent1.createSubregion(getPartitionedRegionName(), attributesFactory.create());
+        parent2.createSubregion(getPartitionedRegionName(), attributesFactory.create());
+
         try {
           recoveryDone.await(MAX_WAIT, TimeUnit.MILLISECONDS);
         } catch (InterruptedException e) {
-          Assert.fail("interrupted", e);
+          throw new RuntimeException(e);
         }
       }
     };
-    return createPR;
   }
 
-  protected void createCoLocatedPR(VM vm, int setRedundantCopies,
-      boolean setPersistenceAdvisorObserver) {
+  void createCoLocatedPR(final VM vm, final int setRedundantCopies,
+      final boolean setPersistenceAdvisorObserver) {
     vm.invoke(() -> {
       String dsName = "colacatedpr";
 
-      Cache cache = getCache();
-
       // Wait for both nested PRs to be created
       final CountDownLatch recoveryDone = new CountDownLatch(2);
-      ResourceObserver observer = new InternalResourceManager.ResourceObserverAdapter() {
+      ResourceObserver observer = new ResourceObserverAdapter() {
         @Override
-        public void recoveryFinished(Region region) {
+        public void recoveryFinished(final Region region) {
           recoveryDone.countDown();
         }
       };
@@ -407,47 +273,53 @@ public abstract class PersistentPartitionedRegionTestBase extends JUnit4CacheTes
       // And throw exception while region is getting initialized.
       final CountDownLatch childRegionCreated = new CountDownLatch(1);
       if (setPersistenceAdvisorObserver) {
-        PersistenceAdvisorImpl
-            .setPersistenceAdvisorObserver(new PersistenceAdvisorImpl.PersistenceAdvisorObserver() {
-              public void observe(String regionPath) {
-                if (regionPath.contains(PR_CHILD_REGION_NAME)) {
-                  try {
-                    childRegionCreated.await(MAX_WAIT, TimeUnit.MILLISECONDS);
-                  } catch (Exception e) {
-                    Assert.fail("Exception", e);
-                  }
-                  throw new ConflictingPersistentDataException(
-                      "Testing Cache Close with ConflictingPersistentDataException for region."
-                          + regionPath);
-                }
+        PersistenceAdvisorImpl.setPersistenceAdvisorObserver(new PersistenceAdvisorObserver() {
+          @Override
+          public void observe(String regionPath) {
+            if (regionPath.contains(getChildRegionName())) {
+              try {
+                childRegionCreated.await(MAX_WAIT, TimeUnit.MILLISECONDS);
+              } catch (InterruptedException e) {
+                throw new RuntimeException(e);
               }
-            });
+              throw new ConflictingPersistentDataException(
+                  "Testing Cache Close with ConflictingPersistentDataException for region "
+                      + regionPath);
+            }
+          }
+        });
       }
 
       // Create region.
       try {
-        DiskStore ds = cache.findDiskStore(dsName);
-        if (ds == null) {
-          ds = cache.createDiskStoreFactory().setDiskDirs(getDiskDirs()).create(dsName);
+        Cache cache = getCache();
+
+        DiskStore diskStore = cache.findDiskStore(dsName);
+        if (diskStore == null) {
+          diskStore = cache.createDiskStoreFactory().setDiskDirs(getDiskDirs()).create(dsName);
         }
 
         // Parent Region
-        PartitionAttributesFactory paf =
+        PartitionAttributesFactory partitionAttributesFactory =
             new PartitionAttributesFactory().setRedundantCopies(setRedundantCopies);
-        AttributesFactory af = new AttributesFactory();
-        af.setPartitionAttributes(paf.create());
-        af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
-        af.setDiskStoreName(dsName);
-        cache.createRegion(PR_REGION_NAME, af.create());
+
+        AttributesFactory attributesFactory = new AttributesFactory();
+        attributesFactory.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
+        attributesFactory.setDiskStoreName(dsName);
+        attributesFactory.setPartitionAttributes(partitionAttributesFactory.create());
+
+        cache.createRegion(getPartitionedRegionName(), attributesFactory.create());
 
         // Colocated region
-        paf = (new PartitionAttributesFactory()).setRedundantCopies(setRedundantCopies)
-            .setColocatedWith(PR_REGION_NAME);
-        af = new AttributesFactory();
-        af.setPartitionAttributes(paf.create());
-        af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
-        af.setDiskStoreName(dsName);
-        cache.createRegion(PR_CHILD_REGION_NAME, af.create());
+        partitionAttributesFactory = (new PartitionAttributesFactory())
+            .setRedundantCopies(setRedundantCopies).setColocatedWith(getPartitionedRegionName());
+
+        attributesFactory = new AttributesFactory();
+        attributesFactory.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
+        attributesFactory.setDiskStoreName(dsName);
+        attributesFactory.setPartitionAttributes(partitionAttributesFactory.create());
+
+        cache.createRegion(getChildRegionName(), attributesFactory.create());
 
         // Count down on region create.
         childRegionCreated.countDown();
@@ -455,7 +327,7 @@ public abstract class PersistentPartitionedRegionTestBase extends JUnit4CacheTes
         try {
           recoveryDone.await(MAX_WAIT, TimeUnit.MILLISECONDS);
         } catch (InterruptedException e) {
-          Assert.fail("interrupted", e);
+          throw new RuntimeException(e);
         }
 
       } finally {
@@ -465,7 +337,7 @@ public abstract class PersistentPartitionedRegionTestBase extends JUnit4CacheTes
   }
 
   private SerializableRunnable getCreatePRRunnable(final int redundancy, final int recoveryDelay) {
-    return getCreatePRRunnable(redundancy, recoveryDelay, 113);
+    return getCreatePRRunnable(redundancy, recoveryDelay, NUM_BUCKETS);
   }
 
   private SerializableRunnable getCreatePRRunnable(final int redundancy, final int recoveryDelay,
@@ -475,14 +347,12 @@ public abstract class PersistentPartitionedRegionTestBase extends JUnit4CacheTes
 
   private SerializableRunnable getCreatePRRunnable(final int redundancy, final int recoveryDelay,
       final int numBuckets, final boolean synchronous) {
-    SerializableRunnable createPR = new SerializableRunnable("create pr") {
-
+    return new SerializableRunnable("getCreatePRRunnable") {
+      @Override
       public void run() {
-        final CountDownLatch recoveryDone;
+        final CountDownLatch recoveryDone = new CountDownLatch(1);
         if (redundancy > 0) {
-          recoveryDone = new CountDownLatch(1);
-
-          ResourceObserver observer = new InternalResourceManager.ResourceObserverAdapter() {
+          ResourceObserver observer = new ResourceObserverAdapter() {
             @Override
             public void recoveryFinished(Region region) {
               recoveryDone.countDown();
@@ -490,391 +360,302 @@ public abstract class PersistentPartitionedRegionTestBase extends JUnit4CacheTes
           };
           InternalResourceManager.setResourceObserver(observer);
         } else {
-          recoveryDone = null;
+          recoveryDone.countDown();
         }
 
         Cache cache = getCache();
 
-        RegionAttributes attr =
+        RegionAttributes regionAttributes =
             getPersistentPRAttributes(redundancy, recoveryDelay, cache, numBuckets, synchronous);
-        cache.createRegion(PR_REGION_NAME, attr);
-        if (recoveryDone != null) {
-          try {
-            recoveryDone.await();
-          } catch (InterruptedException e) {
-            Assert.fail("Interrupted", e);
-          }
+        cache.createRegion(getPartitionedRegionName(), regionAttributes);
+
+        try {
+          recoveryDone.await();
+        } catch (InterruptedException e) {
+          throw new RuntimeException(e);
         }
       }
     };
-    return createPR;
   }
 
   protected RegionAttributes getPersistentPRAttributes(final int redundancy,
-      final int recoveryDelay, Cache cache, int numBuckets, boolean synchronous) {
-    DiskStore ds = cache.findDiskStore("disk");
-    if (ds == null) {
-      ds = cache.createDiskStoreFactory().setDiskDirs(getDiskDirs()).create("disk");
+      final int recoveryDelay, final Cache cache, final int numBuckets, final boolean synchronous) {
+    DiskStore diskStore = cache.findDiskStore("disk");
+    if (diskStore == null) {
+      diskStore = cache.createDiskStoreFactory().setDiskDirs(getDiskDirs()).create("disk");
     }
-    AttributesFactory af = new AttributesFactory();
-    PartitionAttributesFactory paf = new PartitionAttributesFactory();
-    paf.setRedundantCopies(redundancy);
-    paf.setRecoveryDelay(recoveryDelay);
-    paf.setTotalNumBuckets(numBuckets);
+
+    PartitionAttributesFactory partitionAttributesFactory = new PartitionAttributesFactory();
+    partitionAttributesFactory.setRedundantCopies(redundancy);
+    partitionAttributesFactory.setRecoveryDelay(recoveryDelay);
+    partitionAttributesFactory.setTotalNumBuckets(numBuckets);
     // Make sure all vms end up with the same local max memory
-    paf.setLocalMaxMemory(500);
-    af.setPartitionAttributes(paf.create());
-    af.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
-    af.setDiskStoreName("disk");
-    af.setDiskSynchronous(synchronous);
-    RegionAttributes attr = af.create();
-    return attr;
+    partitionAttributesFactory.setLocalMaxMemory(500);
+
+    AttributesFactory attributesFactory = new AttributesFactory();
+    attributesFactory.setDataPolicy(DataPolicy.PERSISTENT_PARTITION);
+    attributesFactory.setDiskStoreName("disk");
+    attributesFactory.setDiskSynchronous(synchronous);
+    attributesFactory.setPartitionAttributes(partitionAttributesFactory.create());
+
+    return attributesFactory.create();
   }
 
-  protected AsyncInvocation createPRAsync(VM vm0, final int redundancy, int recoveryDelay,
+  AsyncInvocation createPRAsync(final VM vm, final int redundancy, int recoveryDelay,
       int numBuckets) {
-    SerializableRunnable createPR = getCreatePRRunnable(redundancy, recoveryDelay, numBuckets);
-    return vm0.invokeAsync(createPR);
+    return vm.invokeAsync(getCreatePRRunnable(redundancy, recoveryDelay, numBuckets));
   }
 
-  protected AsyncInvocation createPRAsync(VM vm0, final int redundancy) {
-    SerializableRunnable createPR = getCreatePRRunnable(redundancy, -1);
-    return vm0.invokeAsync(createPR);
+  AsyncInvocation createPRAsync(final VM vm, final int redundancy) {
+    return vm.invokeAsync(getCreatePRRunnable(redundancy, -1));
   }
 
-  protected Set<Integer> getBucketList(VM vm0) {
-    return getBucketList(vm0, PR_REGION_NAME);
+  protected Set<Integer> getBucketList(final VM vm) {
+    return getBucketList(vm, getPartitionedRegionName());
   }
 
-  protected Set<Integer> getBucketList(VM vm0, final String regionName) {
-    SerializableCallable getBuckets = new SerializableCallable("get buckets") {
-
-      public Object call() throws Exception {
-        Cache cache = getCache();
-        PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionName);
-        return new TreeSet<Integer>(region.getDataStore().getAllLocalBucketIds());
-      }
-    };
-
-    return (Set<Integer>) vm0.invoke(getBuckets);
+  protected Set<Integer> getBucketList(final VM vm, final String regionName) {
+    return vm.invoke("getBucketList", () -> {
+      PartitionedRegion region = (PartitionedRegion) getCache().getRegion(regionName);
+      return new TreeSet<>(region.getDataStore().getAllLocalBucketIds());
+    });
   }
 
-  protected void waitForBuckets(VM vm, final Set<Integer> expectedBuckets,
-      final String regionName) {
-    SerializableCallable getBuckets = new SerializableCallable("get buckets") {
-
-      public Object call() throws Exception {
-        Cache cache = getCache();
-        final PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionName);
-        Wait.waitForCriterion(new WaitCriterion() {
-
-          public boolean done() {
-            return expectedBuckets.equals(getActualBuckets());
-          }
-
-          public String description() {
-            return "Buckets on vm " + getActualBuckets() + " never became equal to expected "
-                + expectedBuckets;
-          }
+  void waitForBuckets(final VM vm, final Set<Integer> expectedBuckets, final String regionName) {
+    vm.invoke("waitForBuckets", () -> {
+      Cache cache = getCache();
+      final PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionName);
 
-          public TreeSet<Integer> getActualBuckets() {
-            return new TreeSet<Integer>(region.getDataStore().getAllLocalBucketIds());
-          }
-        }, 30 * 1000, 100, true);
+      Wait.waitForCriterion(new WaitCriterion() {
+        @Override
+        public boolean done() {
+          return expectedBuckets.equals(getActualBuckets());
+        }
 
-        return null;
-      }
-    };
+        @Override
+        public String description() {
+          return "Buckets on vm " + getActualBuckets() + " never became equal to expected "
+              + expectedBuckets;
+        }
 
-    vm.invoke(getBuckets);
+        Set<Integer> getActualBuckets() {
+          return new TreeSet<>(region.getDataStore().getAllLocalBucketIds());
+        }
+      }, 30 * 1000, 100, true);
+    });
   }
 
-  protected Set<Integer> getPrimaryBucketList(VM vm0) {
-    return getPrimaryBucketList(vm0, PR_REGION_NAME);
+  Set<Integer> getPrimaryBucketList(final VM vm) {
+    return getPrimaryBucketList(vm, getPartitionedRegionName());
   }
 
-  protected Set<Integer> getPrimaryBucketList(VM vm0, final String regionName) {
-    SerializableCallable getPrimaryBuckets = new SerializableCallable("get primary buckets") {
-
-      public Object call() throws Exception {
-        Cache cache = getCache();
-        PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionName);
-        return new TreeSet<Integer>(region.getDataStore().getAllLocalPrimaryBucketIds());
-      }
-    };
-
-    return (Set<Integer>) vm0.invoke(getPrimaryBuckets);
+  Set<Integer> getPrimaryBucketList(final VM vm, final String regionName) {
+    return vm.invoke("getPrimaryBucketList", () -> {
+      Cache cache = getCache();
+      PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionName);
+      return new TreeSet<>(region.getDataStore().getAllLocalPrimaryBucketIds());
+    });
   }
 
+  void revokeKnownMissingMembers(final VM vm, final int numExpectedMissing) {
+    vm.invoke("revokeKnownMissingMembers", () -> {
+      DistributedSystemConfig config = defineDistributedSystem(getSystem(), "");
+      AdminDistributedSystem adminDS = getDistributedSystem(config);
+      adminDS.connect();
+      try {
+        adminDS.waitToBeConnected(MAX_WAIT);
 
-  protected void revokeKnownMissingMembers(VM vm2, final int numExpectedMissing) {
-    vm2.invoke(new SerializableRunnable("Revoke the member") {
-
-      public void run() {
-        final DistributedSystemConfig config;
-        final AdminDistributedSystem adminDS;
-        try {
-          config = AdminDistributedSystemFactory.defineDistributedSystem(getSystem(), "");
-          adminDS = AdminDistributedSystemFactory.getDistributedSystem(config);
-          adminDS.connect();
-          adminDS.waitToBeConnected(MAX_WAIT);
-          try {
-            final WaitCriterion wc = new WaitCriterion() {
-
-              public boolean done() {
-                try {
-                  final Set<PersistentID> missingIds = adminDS.getMissingPersistentMembers();
-                  if (missingIds.size() != numExpectedMissing) {
-                    return false;
-                  }
-                  for (PersistentID missingId : missingIds) {
-                    adminDS.revokePersistentMember(missingId.getUUID());
-                  }
-                  return true;
-                } catch (AdminException ae) {
-                  throw new RuntimeException(ae);
-                }
+        final WaitCriterion wc = new WaitCriterion() {
+          @Override
+          public boolean done() {
+            try {
+              Set<PersistentID> missingIds = adminDS.getMissingPersistentMembers();
+              if (missingIds.size() != numExpectedMissing) {
+                return false;
               }
-
-              public String description() {
-                try {
-                  return "expected " + numExpectedMissing
-                      + " missing members for revocation, current: "
-                      + adminDS.getMissingPersistentMembers();
-                } catch (AdminException ae) {
-                  throw new RuntimeException(ae);
-                }
+              for (PersistentID missingId : missingIds) {
+                adminDS.revokePersistentMember(missingId.getUUID());
               }
-            };
-            Wait.waitForCriterion(wc, MAX_WAIT, 500, true);
-          } finally {
-            adminDS.disconnect();
+              return true;
+            } catch (AdminException e) {
+              throw new RuntimeException(e);
+            }
           }
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
-  }
 
-  protected void revokeAllMembers(VM vm) {
-    vm.invoke(new SerializableRunnable("Revoke the member") {
-
-      public void run() {
-        GemFireCacheImpl cache = (GemFireCacheImpl) getCache();
-        DistributedSystemConfig config;
-        AdminDistributedSystem adminDS = null;
-        try {
-          config = AdminDistributedSystemFactory.defineDistributedSystem(getSystem(), "");
-          adminDS = AdminDistributedSystemFactory.getDistributedSystem(config);
-          adminDS.connect();
-          adminDS.waitToBeConnected(MAX_WAIT);
-          adminDS.revokePersistentMember(InetAddress.getLocalHost(), null);
-        } catch (RuntimeException e) {
-          throw e;
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        } finally {
-          if (adminDS != null) {
-            adminDS.disconnect();
+          @Override
+          public String description() {
+            try {
+              return "expected " + numExpectedMissing + " missing members for revocation, current: "
+                  + adminDS.getMissingPersistentMembers();
+            } catch (AdminException e) {
+              throw new RuntimeException(e);
+            }
           }
-        }
+        };
+        Wait.waitForCriterion(wc, MAX_WAIT, 500, true);
+
+      } finally {
+        adminDS.disconnect();
       }
     });
   }
 
-  protected void revokeMember(VM vm, final File directory) {
-    vm.invoke(new SerializableRunnable("Revoke the member") {
+  void revokeAllMembers(final VM vm) {
+    vm.invoke("revokeAllMembers", () -> {
+      InternalCache cache = getCache(); // TODO:KIRK: delete this line
+      DistributedSystemConfig config = defineDistributedSystem(getSystem(), "");
+      AdminDistributedSystem adminDS = getDistributedSystem(config);
+      adminDS.connect();
 
-      public void run() {
-        GemFireCacheImpl cache = (GemFireCacheImpl) getCache();
-        DistributedSystemConfig config;
-        AdminDistributedSystem adminDS = null;
-        try {
-          config = AdminDistributedSystemFactory.defineDistributedSystem(getSystem(), "");
-          adminDS = AdminDistributedSystemFactory.getDistributedSystem(config);
-          adminDS.connect();
-          adminDS.waitToBeConnected(MAX_WAIT);
-          adminDS.revokePersistentMember(InetAddress.getLocalHost(), directory.getCanonicalPath());
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        } finally {
-          if (adminDS != null) {
-            adminDS.disconnect();
-          }
-        }
+      try {
+        adminDS.waitToBeConnected(MAX_WAIT);
+        adminDS.revokePersistentMember(InetAddress.getLocalHost(), null);
+      } finally {
+        adminDS.disconnect();
       }
     });
   }
 
-  protected boolean moveBucket(final int bucketId, VM source, VM target) {
-
-    SerializableCallable getId = new SerializableCallable("Get Id") {
-
-      public Object call() throws Exception {
-        Cache cache = getCache();
-        return cache.getDistributedSystem().getDistributedMember();
-      }
-    };
-
-    final InternalDistributedMember sourceId = (InternalDistributedMember) source.invoke(getId);
-
-    SerializableCallable move = new SerializableCallable("move bucket") {
-
-      public Object call() {
-        Cache cache = getCache();
-        PartitionedRegion region = (PartitionedRegion) cache.getRegion(PR_REGION_NAME);
-        return region.getDataStore().moveBucket(bucketId, sourceId, false);
+  void revokeMember(final VM vm, final File directory) {
+    vm.invoke("revokeMember", () -> {
+      InternalCache cache = getCache(); // TODO:KIRK: delete this line
+      DistributedSystemConfig config = defineDistributedSystem(getSystem(), "");
+      AdminDistributedSystem adminDS = getDistributedSystem(config);
+      adminDS.connect();
+      try {
+        adminDS.waitToBeConnected(MAX_WAIT);
+        adminDS.revokePersistentMember(InetAddress.getLocalHost(), directory.getCanonicalPath());
+      } finally {
+        adminDS.disconnect();
       }
-    };
-
-    return (Boolean) target.invoke(move);
-
+    });
   }
 
-  protected Set<PersistentMemberID> getOfflineMembers(final int bucketId, VM vm) {
-
-    SerializableCallable getId = new SerializableCallable("Get Id") {
-
-      public Object call() throws Exception {
-        Cache cache = getCache();
-        PartitionedRegion region = (PartitionedRegion) cache.getRegion(PR_REGION_NAME);
-        return region.getRegionAdvisor().getProxyBucketArray()[bucketId].getPersistenceAdvisor()
-            .getMembershipView().getOfflineMembers();
-      }
-    };
-
-
-    return (Set<PersistentMemberID>) vm.invoke(getId);
-
+  protected boolean moveBucket(final int bucketId, final VM source, final VM target) {
+    InternalDistributedMember sourceId = getInternalDistributedMember(source);
 
+    return target.invoke("moveBucket", () -> {
+      PartitionedRegion region =
+          (PartitionedRegion) getCache().getRegion(getPartitionedRegionName());
+      return region.getDataStore().moveBucket(bucketId, sourceId, false);
+    });
   }
 
-  protected Set<PersistentMemberID> getOnlineMembers(final int bucketId, VM vm) {
-
-    SerializableCallable getId = new SerializableCallable("Get Id") {
-
-      public Object call() throws Exception {
-        Cache cache = getCache();
-        PartitionedRegion region = (PartitionedRegion) cache.getRegion(PR_REGION_NAME);
-        return region.getRegionAdvisor().getProxyBucketArray()[bucketId].getPersistenceAdvisor()
-            .getPersistedOnlineOrEqualMembers();
-      }
-    };
-
-
-    return (Set<PersistentMemberID>) vm.invoke(getId);
+  private InternalDistributedMember getInternalDistributedMember(final VM vm) {
+    return (InternalDistributedMember) vm.invoke("getDistributedMember",
+        () -> getCache().getDistributedSystem().getDistributedMember());
   }
 
-  protected void waitForBucketRecovery(final VM vm2, final Set<Integer> lostBuckets) {
-    waitForBucketRecovery(vm2, lostBuckets, PR_REGION_NAME);
+  Set<PersistentMemberID> getOfflineMembers(final int bucketId, final VM vm) {
+    return vm.invoke("getOfflineMembers", () -> {
+      PartitionedRegion region =
+          (PartitionedRegion) getCache().getRegion(getPartitionedRegionName());
+      return region.getRegionAdvisor().getProxyBucketArray()[bucketId].getPersistenceAdvisor()
+          .getMembershipView().getOfflineMembers();
+    });
   }
 
-  protected void waitForBucketRecovery(final VM vm2, final Set<Integer> lostBuckets,
-      final String regionName) {
-    vm2.invoke(new SerializableRunnable() {
-      public void run() {
-        Cache cache = getCache();
-        PartitionedRegion region = (PartitionedRegion) cache.getRegion(regionName);
-        final PartitionedRegionDataStore dataStore = region.getDataStore();
-        Wait.waitForCriterion(new WaitCriterion() {
-
-          public boolean done() {
-            Set<Integer> vm2Buckets = dataStore.getAllLocalBucketIds();
-            return lostBuckets.equals(vm2Buckets);
-          }
-
-          public String description() {
-            return "expected to recover " + lostBuckets + " buckets, now have "
-                + dataStore.getAllLocalBucketIds();
-          }
-        }, MAX_WAIT, 100, true);
-      }
+  Set<PersistentMemberID> getOnlineMembers(final int bucketId, final VM vm) {
+    return vm.invoke("getOnlineMembers", () -> {
+      PartitionedRegion region =
+          (PartitionedRegion) getCache().getRegion(getPartitionedRegionName());
+      return region.getRegionAdvisor().getProxyBucketArray()[bucketId].getPersistenceAdvisor()
+          .getPersistedOnlineOrEqualMembers();
     });
   }
 
-  protected void waitForRedundancyRecovery(VM vm, final int expectedRedundancy,
-      final String regionName) {
-    vm.invoke(new SerializableRunnable() {
+  void waitForBucketRecovery(final VM vm, final Set<Integer> lostBuckets) {
+    waitForBucketRecovery(vm, lostBuckets, getPartitionedRegionName());
+  }
 
-      public void run() {
-        Cache cache = getCache();
-        final Region region = cache.getRegion(regionName);
-        Wait.waitForCriterion(new WaitCriterion() {
+  private void waitForBucketRecovery(final VM vm, final Set<Integer> lostBuckets,
+      final String regionName) {
+    vm.invoke("waitForBucketRecovery", () -> {
+      PartitionedRegion region = (PartitionedRegion) getCache().getRegion(regionName);
+      PartitionedRegionDataStore dataStore = region.getDataStore();
 
-          public boolean done() {
-            PartitionRegionInfo info = PartitionRegionHelper.getPartitionRegionInfo(region);
-            return info.getActualRedundantCopies() == expectedRedundancy;
-          }
+      Wait.waitForCriterion(new WaitCriterion() {
+        @Override
+        public boolean done() {
+          Set<Integer> vm2Buckets = dataStore.getAllLocalBucketIds();
+          return lostBuckets.equals(vm2Buckets);
+        }
 
-          public String description() {
-            PartitionRegionInfo info = PartitionRegionHelper.getPartitionRegionInfo(region);
-            return "Did not reach expected redundancy " + expectedRedundancy + " redundancy info = "
-                + info.getActualRedundantCopies();
-          }
-        }, 30 * 1000, 100, true);
-      }
+        @Override
+        public String description() {
+          return "expected to recover " + lostBuckets + " buckets, now have "
+              + dataStore.getAllLocalBucketIds();
+        }
+      }, MAX_WAIT, 100, true);
     });
   }
 
-  protected void invalidateData(VM vm, final int startKey, final int endKey) {
-    SerializableRunnable createData = new SerializableRunnable() {
+  void waitForRedundancyRecovery(final VM vm, final int expectedRedundancy,
+      final String regionName) {
+    vm.invoke("waitForRedundancyRecovery", () -> {
+      Region region = getCache().getRegion(regionName);
 
-      public void run() {
-        Cache cache = getCache();
-        Region region = cache.getRegion(PR_REGION_NAME);
+      Wait.waitForCriterion(new WaitCriterion() {
+        @Override
+        public boolean done() {
+          PartitionRegionInfo info = PartitionRegionHelper.getPartitionRegionInfo(region);
+          return info.getActualRedundantCopies() == expectedRedundancy;
+        }
 
-        for (int i = startKey; i < endKey; i++) {
-          region.destroy(i);
-          region.create(i, null);
-          region.invalidate(i);
+        @Override
+        public String description() {
+          PartitionRegionInfo info = PartitionRegionHelper.getPartitionRegionInfo(region);
+          return "Did not reach expected redundancy " + expectedRedundancy + " redundancy info = "
+              + info.getActualRedundantCopies();
         }
-      }
-    };
-    vm.invoke(createData);
+      }, 30 * 1000, 100, true);
+    });
   }
 
-  // used for above test
-  protected BackupStatus backup(VM vm) {
-    return (BackupStatus) vm.invoke(new SerializableCallable("Backup all members") {
-
-      public Object call() {
-        try {
-          return BackupUtil.backupAllMembers(getSystem().getDistributionManager(), getBackupDir(),
-              null);
-        } catch (ManagementException e) {
-          throw new RuntimeException(e);
-        }
+  protected BackupStatus backup(final VM vm) {
+    return vm.invoke("backup", () -> {
+      try {
+        return BackupUtil.backupAllMembers(getSystem().getDistributionManager(), getBackupDir(),
+            null);
+      } catch (ManagementException e) {
+        throw new RuntimeException(e);
       }
     });
   }
 
-  protected void restoreBackup(int expectedNumScripts) throws IOException, InterruptedException {
-    Collection<File> restoreScripts = FileUtils.listFiles(getBackupDir(),
-        new RegexFileFilter(".*restore.*"), DirectoryFileFilter.DIRECTORY);
-    assertEquals("Restore scripts " + restoreScripts, expectedNumScripts, restoreScripts.size());
+  protected void restoreBackup(final int expectedNumScripts)
+      throws IOException, InterruptedException {
+    Collection<File> restoreScripts =
+        listFiles(getBackupDir(), new RegexFileFilter(".*restore.*"), DIRECTORY);
+    assertThat(restoreScripts).hasSize(expectedNumScripts);
     for (File script : restoreScripts) {
       execute(script);
     }
-
   }
 
-  private void execute(File script) throws IOException, InterruptedException {
-    ProcessBuilder pb = new ProcessBuilder(script.getAbsolutePath());
-    pb.redirectErrorStream(true);
-    Process process = pb.start();
+  private void execute(final File script) throws IOException, InterruptedException {
+    ProcessBuilder processBuilder = new ProcessBuilder(script.getAbsolutePath());
+    processBuilder.redirectErrorStream(true);
+    Process process = processBuilder.start();
+
+    try (BufferedReader reader =
+        new BufferedReader(new InputStreamReader(process.getInputStream()))) {
+      String line;
+      while ((line = reader.readLine()) != null) {
+        logger.info("OUTPUT:" + line);
+        // TODO validate output
+      }
+    }
 
-    InputStream is = process.getInputStream();
-    byte[] buffer = new byte[1024];
-    BufferedReader br = new BufferedReader(new InputStreamReader(is));
-    String line;
-    while ((line = br.readLine()) != null) {
-      LogWriterUtils.getLogWriter().fine("OUTPUT:" + line);
-      // TODO validate output
-    } ;
+    assertThat(process.waitFor()).isEqualTo(0);
+  }
 
-    assertEquals(0, process.waitFor());
+  public String getPartitionedRegionName() {
+    return partitionedRegionName;
+  }
 
+  String getChildRegionName() {
+    return CHILD_REGION_NAME;
   }
 
   protected static File getBackupDir() {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionWithTransactionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionWithTransactionDUnitTest.java
index 5da461b..135bbca 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionWithTransactionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/partitioned/PersistentPartitionedRegionWithTransactionDUnitTest.java
@@ -22,7 +22,9 @@ import org.junit.experimental.categories.Category;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.CacheTransactionManager;
 import org.apache.geode.cache.Region;
+import org.apache.geode.internal.cache.DiskRegion;
 import org.apache.geode.internal.cache.DiskStoreImpl;
+import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.TXManagerImpl;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.Host;
@@ -30,8 +32,6 @@ import org.apache.geode.test.dunit.Invoke;
 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.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
 import org.apache.geode.test.junit.categories.DistributedTest;
 
 /**
@@ -120,7 +120,7 @@ public class PersistentPartitionedRegionWithTransactionDUnitTest
         Cache cache = getCache();
 
         CacheTransactionManager tx = cache.getCacheTransactionManager();
-        Region region = cache.getRegion(PR_REGION_NAME);
+        Region region = cache.getRegion(getPartitionedRegionName());
 
         for (int i = startKey; i < endKey; i++) {
           tx.begin();
@@ -169,7 +169,7 @@ public class PersistentPartitionedRegionWithTransactionDUnitTest
   }
 
   @Override
-  protected void checkData(VM vm0, final int startKey, final int endKey, final String value,
+  protected void checkData(VM vm, final int startKey, final int endKey, final String value,
       final String regionName) {
     SerializableRunnable checkData = new SerializableRunnable() {
 
@@ -187,6 +187,24 @@ public class PersistentPartitionedRegionWithTransactionDUnitTest
       }
     };
 
-    vm0.invoke(checkData);
+    vm.invoke(checkData);
+  }
+
+  void checkRecoveredFromDisk(VM vm, final int bucketId, final boolean recoveredLocally) {
+    vm.invoke(new SerializableRunnable("check recovered from disk") {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        PartitionedRegion region = (PartitionedRegion) cache.getRegion(getPartitionedRegionName());
+        DiskRegion disk = region.getRegionAdvisor().getBucket(bucketId).getDiskRegion();
+        if (recoveredLocally) {
+          assertEquals(0, disk.getStats().getRemoteInitializations());
+          assertEquals(1, disk.getStats().getLocalInitializations());
+        } else {
+          assertEquals(1, disk.getStats().getRemoteInitializations());
+          assertEquals(0, disk.getStats().getLocalInitializations());
+        }
+      }
+    });
   }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
index df17039..9dcf90b 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/persistence/PersistentRecoveryOrderDUnitTest.java
@@ -1213,7 +1213,7 @@ public class PersistentRecoveryOrderDUnitTest extends PersistentReplicatedTestBa
           }
 
           @Override
-          public void beforeSendMessage(DistributionManager dm, DistributionMessage msg) {}
+          public void beforeSendMessage(DistributionManager dm, DistributionMessage message) {}
         });
       }
     });
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/beans/DistributedSystemBridgeJUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/beans/DistributedSystemBridgeJUnitTest.java
index 2c34ddd..c742c5f 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/beans/DistributedSystemBridgeJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/beans/DistributedSystemBridgeJUnitTest.java
@@ -26,13 +26,13 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.InOrder;
 
-import org.apache.geode.admin.internal.BackupDataStoreHelper;
-import org.apache.geode.admin.internal.FinishBackupRequest;
-import org.apache.geode.admin.internal.PrepareBackupRequest;
 import org.apache.geode.distributed.internal.DM;
 import org.apache.geode.distributed.internal.locks.DLockService;
-import org.apache.geode.internal.cache.BackupManager;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.backup.BackupDataStoreHelper;
+import org.apache.geode.internal.cache.backup.BackupManager;
+import org.apache.geode.internal.cache.backup.FinishBackupRequest;
+import org.apache.geode.internal.cache.backup.PrepareBackupRequest;
 import org.apache.geode.internal.cache.persistence.PersistentMemberManager;
 import org.apache.geode.test.fake.Fakes;
 import org.apache.geode.test.junit.categories.UnitTest;
diff --git a/geode-core/src/test/java/org/apache/geode/pdx/ClientsWithVersioningRetryDUnitTest.java b/geode-core/src/test/java/org/apache/geode/pdx/ClientsWithVersioningRetryDUnitTest.java
index b67cc48..37c7960 100644
--- a/geode-core/src/test/java/org/apache/geode/pdx/ClientsWithVersioningRetryDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/pdx/ClientsWithVersioningRetryDUnitTest.java
@@ -215,8 +215,8 @@ public class ClientsWithVersioningRetryDUnitTest extends JUnit4CacheTestCase {
         DistributionMessageObserver.setInstance(new DistributionMessageObserver() {
 
           @Override
-          public void beforeSendMessage(DistributionManager dm, DistributionMessage msg) {
-            if (msg instanceof DistributedPutAllOperation.PutAllMessage) {
+          public void beforeSendMessage(DistributionManager dm, DistributionMessage message) {
+            if (message instanceof DistributedPutAllOperation.PutAllMessage) {
               DistributionMessageObserver.setInstance(null);
               disconnectFromDS(vm1);
             }
diff --git a/geode-core/src/test/resources/org/apache/geode/internal/cache/BackupJUnitTest.cache.xml b/geode-core/src/test/resources/org/apache/geode/internal/cache/backup/BackupIntegrationTest.cache.xml
similarity index 100%
rename from geode-core/src/test/resources/org/apache/geode/internal/cache/BackupJUnitTest.cache.xml
rename to geode-core/src/test/resources/org/apache/geode/internal/cache/backup/BackupIntegrationTest.cache.xml

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