You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2017/11/06 22:09:58 UTC

[geode] branch GEODE-3801-backup-hardlinks-recommit created (now 81377bb)

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

klund pushed a change to branch GEODE-3801-backup-hardlinks-recommit
in repository https://gitbox.apache.org/repos/asf/geode.git.


      at 81377bb  GEODE-3801: Use hardlinks for backup oplog files (#963)

This branch includes the following new commits:

     new 81377bb  GEODE-3801: Use hardlinks for backup oplog files (#963)

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.


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

[geode] 01/01: GEODE-3801: Use hardlinks for backup oplog files (#963)

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

klund pushed a commit to branch GEODE-3801-backup-hardlinks-recommit
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 81377bb9eef3f46fb282a52d557dd1d9b12ccc6e
Author: Nick Reich <nr...@pivotal.io>
AuthorDate: Tue Oct 24 08:24:42 2017 -0700

    GEODE-3801: Use hardlinks for backup oplog files (#963)
    
    * Oplog files that are backed up are read-only. For this reason,
      they can be transfered to the backup location through hard links,
      instead of copying the file. This change improves speed of backups.
      If the creation of a hard link fails, we revert to the existing copy
      behavior.
    * During backups, the copying of the oplog's krf file was being done while
      that file could still be in the process of writing. This change ensures
      that if a krf is to be written, that it is finished and included in the
      backup
    * cleanup existing oplog tests
---
 .../apache/geode/internal/cache/BackupManager.java |   25 +-
 .../geode/internal/cache/InternalRegion.java       |    4 +
 .../apache/geode/internal/cache/LocalRegion.java   |    1 +
 .../org/apache/geode/internal/cache/Oplog.java     |   30 +-
 .../cache/Bug34179TooManyFilesOpenJUnitTest.java   |    2 +-
 .../internal/cache/DiskRegRecoveryJUnitTest.java   |    2 +-
 .../cache/DiskRegionAsyncRecoveryJUnitTest.java    |    2 +-
 .../internal/cache/DiskRegionHelperFactory.java    |  109 +-
 .../geode/internal/cache/DiskRegionJUnitTest.java  |    6 +-
 .../internal/cache/DiskRegionTestingBase.java      |  118 +-
 .../geode/internal/cache/OplogJUnitTest.java       | 2235 ++------------------
 .../internal/cache/SimpleDiskRegionJUnitTest.java  |    2 +-
 ...skRegionOverflowAsyncRollingOpLogJUnitTest.java |    4 +-
 ...iskRegionOverflowSyncRollingOpLogJUnitTest.java |    4 +-
 .../DiskRegionPerfJUnitPerformanceTest.java        |    2 +-
 .../DiskRegionRollOpLogJUnitPerformanceTest.java   |    2 +-
 16 files changed, 329 insertions(+), 2219 deletions(-)

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/BackupManager.java
index 72bbd48..70a4345 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/BackupManager.java
@@ -18,6 +18,7 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.net.URL;
+import java.nio.file.Files;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -266,7 +267,7 @@ public class BackupManager implements MembershipListener {
         File backupDir = getBackupDir(backup.getTargetDir(), index);
         // TODO prpersist - We could probably optimize this to *move* the files
         // that we know are supposed to be deleted.
-        oplog.copyTo(backupDir);
+        backupOplog(backupDir, oplog);
 
         // Allow the oplog to be deleted, and process any pending delete
         backup.backupFinished(oplog);
@@ -569,6 +570,28 @@ public class BackupManager implements MembershipListener {
     }
   }
 
+  private void backupOplog(File targetDir, Oplog oplog) throws IOException {
+    File crfFile = oplog.getCrfFile();
+    backupFile(targetDir, crfFile);
+
+    File drfFile = oplog.getDrfFile();
+    backupFile(targetDir, drfFile);
+
+    oplog.finishKrf();
+    File krfFile = oplog.getKrfFile();
+    backupFile(targetDir, krfFile);
+  }
+
+  private void backupFile(File targetDir, File file) throws IOException {
+    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) {
     return string.replaceAll("[^\\w]+", "_");
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
index 1b7c1b0..3937666 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalRegion.java
@@ -45,6 +45,10 @@ import org.apache.geode.internal.cache.versions.VersionTag;
 public interface InternalRegion<K, V>
     extends Region<K, V>, HasCachePerfStats, RegionEntryContext, RegionAttributes, HasDiskRegion {
 
+  CachePerfStats getCachePerfStats();
+
+  DiskRegion getDiskRegion();
+
   RegionEntry getRegionEntry(K key);
 
   RegionVersionVector getVersionVector();
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
index 16e4cef..78dddd1 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
@@ -2049,6 +2049,7 @@ public class LocalRegion extends AbstractRegion implements InternalRegion, Loade
    *
    * @since GemFire 3.2
    */
+  @Override
   public DiskRegion getDiskRegion() {
     return this.diskRegion;
   }
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 edc6906..9c3e52b 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
@@ -30,6 +30,7 @@ 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;
@@ -1160,7 +1161,7 @@ public class Oplog implements CompactableOplog, Flushable {
    * Otherwise, for windows the actual file length does not match with the File size obtained from
    * the File object
    */
-  File getOplogFile() throws SyncFailedException, IOException {
+  File getOplogFileForTest() throws IOException {
     // @todo check callers for drf
     // No need to get the backup lock prior to synchronizing (correct lock order) since the
     // synchronized block does not attempt to get the backup lock (incorrect lock order)
@@ -1172,6 +1173,14 @@ public class Oplog implements CompactableOplog, Flushable {
     }
   }
 
+  File getCrfFile() {
+    return this.crf.f;
+  }
+
+  File getDrfFile() {
+    return this.drf.f;
+  }
+
   /**
    * Given a set of Oplog file names return a Set of the oplog files that match those names that are
    * managed by this Oplog.
@@ -4224,7 +4233,7 @@ public class Oplog implements CompactableOplog, Flushable {
     }
   }
 
-  private File getKrfFile() {
+  File getKrfFile() {
     return new File(this.diskFile.getPath() + KRF_FILE_EXT);
   }
 
@@ -5751,23 +5760,6 @@ public class Oplog implements CompactableOplog, Flushable {
     deleteFile(this.drf);
   }
 
-  public void copyTo(File targetDir) throws IOException {
-    if (this.crf.f != null && this.crf.f.exists()) {
-      FileUtils.copyFileToDirectory(this.crf.f, targetDir);
-    }
-    if (this.drf.f.exists()) {
-      FileUtils.copyFileToDirectory(this.drf.f, targetDir);
-    }
-
-    // this krf existence check fixes 45089
-    // TODO: should we wait for the async KRF creation to finish by calling this.finishKrf?
-    if (getParent().getDiskInitFile().hasKrf(this.oplogId)) {
-      if (this.getKrfFile().exists()) {
-        FileUtils.copyFileToDirectory(this.getKrfFile(), targetDir);
-      }
-    }
-  }
-
   /**
    * Returns "crf" or "drf".
    */
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/Bug34179TooManyFilesOpenJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/Bug34179TooManyFilesOpenJUnitTest.java
index 052e6ae..caa5f10 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/Bug34179TooManyFilesOpenJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/Bug34179TooManyFilesOpenJUnitTest.java
@@ -84,7 +84,7 @@ public class Bug34179TooManyFilesOpenJUnitTest extends DiskRegionTestingBase {
   /**
    * cleans all the directory of all the files present in them
    */
-  protected static void deleteFiles() {
+  protected void deleteFiles() {
     for (int i = 0; i < dirs.length; i++) {
       File[] files = dirs[i].listFiles();
       for (int j = 0; j < files.length; j++) {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegRecoveryJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegRecoveryJUnitTest.java
index 1984347..fe7f5f5 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegRecoveryJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegRecoveryJUnitTest.java
@@ -662,7 +662,7 @@ public class DiskRegRecoveryJUnitTest extends DiskRegionTestingBase {
     region.put("3", value);
     File oplogFile = null;
     try {
-      oplogFile = ((LocalRegion) region).getDiskRegion().testHook_getChild().getOplogFile();
+      oplogFile = ((LocalRegion) region).getDiskRegion().testHook_getChild().getOplogFileForTest();
     } catch (Exception e) {
       logWriter.error(
           "Exception in synching data present in the buffers of RandomAccessFile of Oplog, to the disk",
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionAsyncRecoveryJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionAsyncRecoveryJUnitTest.java
index 1bfcaa5..8829740 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionAsyncRecoveryJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionAsyncRecoveryJUnitTest.java
@@ -192,7 +192,7 @@ public class DiskRegionAsyncRecoveryJUnitTest extends DiskRegionTestingBase {
     putEntries(region, 10, 15, "A");
 
     PersistentOplogSet set = store.getPersistentOplogSet(region.getDiskRegion());
-    String currentChild = set.getChild().getOplogFile().getName();
+    String currentChild = set.getChild().getOplogFileForTest().getName();
     // Wait for the krfs to be created
     Set<String> crfs;
     Set<String> krfs;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionHelperFactory.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionHelperFactory.java
index eef1c74..326aca7 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionHelperFactory.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionHelperFactory.java
@@ -27,8 +27,8 @@ import org.apache.geode.cache.util.ObjectSizer;
  */
 public class DiskRegionHelperFactory {
 
-  public static Region getRegion(Cache cache, DiskRegionProperties diskProps, Scope regionScope) {
-    Region region = null;
+  private static Region<Object, Object> getRegion(Cache cache, DiskRegionProperties diskProps,
+      Scope regionScope) {
     DiskStoreFactory dsf = cache.createDiskStoreFactory();
     AttributesFactory factory = new AttributesFactory();
     if (diskProps.getDiskDirs() == null) {
@@ -46,7 +46,6 @@ public class DiskRegionHelperFactory {
     } else {
       dsf.setDiskDirsAndSizes(diskProps.getDiskDirs(), diskProps.getDiskDirSizes());
     }
-    // Properties props = new Properties();
     ((DiskStoreFactoryImpl) dsf).setMaxOplogSizeInBytes(diskProps.getMaxOplogSize());
     dsf.setAutoCompact(diskProps.isRolling());
     dsf.setAllowForceCompaction(diskProps.getAllowForceCompaction());
@@ -87,6 +86,7 @@ public class DiskRegionHelperFactory {
     factory.setLoadFactor(diskProps.getLoadFactor());
     factory.setStatisticsEnabled(diskProps.getStatisticsEnabled());
 
+    Region<Object, Object> region = null;
     try {
       region = cache.createVMRegion(diskProps.getRegionName(), factory.createRegionAttributes());
     } catch (TimeoutException e) {
@@ -97,7 +97,7 @@ public class DiskRegionHelperFactory {
     return region;
   }
 
-  public static Region getSyncPersistOnlyRegion(Cache cache,
+  public static Region<Object, Object> getSyncPersistOnlyRegion(Cache cache,
       DiskRegionProperties diskRegionProperties, Scope regionScope) {
     if (diskRegionProperties == null) {
       diskRegionProperties = new DiskRegionProperties();
@@ -108,7 +108,7 @@ public class DiskRegionHelperFactory {
 
   }
 
-  public static Region getAsyncPersistOnlyRegion(Cache cache,
+  public static Region<Object, Object> getAsyncPersistOnlyRegion(Cache cache,
       DiskRegionProperties diskRegionProperties) {
     if (diskRegionProperties == null) {
       diskRegionProperties = new DiskRegionProperties();
@@ -118,7 +118,7 @@ public class DiskRegionHelperFactory {
     return getRegion(cache, diskRegionProperties, Scope.LOCAL);
   }
 
-  public static Region getSyncOverFlowOnlyRegion(Cache cache,
+  public static Region<Object, Object> getSyncOverFlowOnlyRegion(Cache cache,
       DiskRegionProperties diskRegionProperties) {
     if (diskRegionProperties == null) {
       diskRegionProperties = new DiskRegionProperties();
@@ -129,7 +129,7 @@ public class DiskRegionHelperFactory {
     return getRegion(cache, diskRegionProperties, Scope.LOCAL);
   }
 
-  public static Region getAsyncOverFlowOnlyRegion(Cache cache,
+  public static Region<Object, Object> getAsyncOverFlowOnlyRegion(Cache cache,
       DiskRegionProperties diskRegionProperties) {
     if (diskRegionProperties == null) {
       diskRegionProperties = new DiskRegionProperties();
@@ -140,7 +140,7 @@ public class DiskRegionHelperFactory {
     return getRegion(cache, diskRegionProperties, Scope.LOCAL);
   }
 
-  public static Region getSyncOverFlowAndPersistRegion(Cache cache,
+  public static Region<Object, Object> getSyncOverFlowAndPersistRegion(Cache cache,
       DiskRegionProperties diskRegionProperties) {
     if (diskRegionProperties == null) {
       diskRegionProperties = new DiskRegionProperties();
@@ -151,7 +151,7 @@ public class DiskRegionHelperFactory {
     return getRegion(cache, diskRegionProperties, Scope.LOCAL);
   }
 
-  public static Region getAsyncOverFlowAndPersistRegion(Cache cache,
+  public static Region<Object, Object> getAsyncOverFlowAndPersistRegion(Cache cache,
       DiskRegionProperties diskRegionProperties) {
     if (diskRegionProperties == null) {
       diskRegionProperties = new DiskRegionProperties();
@@ -162,7 +162,7 @@ public class DiskRegionHelperFactory {
     return getRegion(cache, diskRegionProperties, Scope.LOCAL);
   }
 
-  public static Region getSyncHeapLruAndPersistRegion(Cache cache,
+  public static Region<Object, Object> getSyncHeapLruAndPersistRegion(Cache cache,
       DiskRegionProperties diskRegionProperties) {
     if (diskRegionProperties == null) {
       diskRegionProperties = new DiskRegionProperties();
@@ -172,93 +172,4 @@ public class DiskRegionHelperFactory {
     diskRegionProperties.setHeapEviction(true);
     return getRegion(cache, diskRegionProperties, Scope.LOCAL);
   }
-
-  public static Region getAsyncHeapLruAndPersistRegion(Cache cache,
-      DiskRegionProperties diskRegionProperties) {
-    if (diskRegionProperties == null) {
-      diskRegionProperties = new DiskRegionProperties();
-    }
-    diskRegionProperties.setPersistBackup(true);
-    diskRegionProperties.setSynchronous(false);
-    diskRegionProperties.setHeapEviction(true);
-    return getRegion(cache, diskRegionProperties, Scope.LOCAL);
-  }
-
-  public static Region getSyncPersistOnlyRegionInfiniteOplog(Cache cache,
-      DiskRegionProperties diskRegionProperties, String regionName) {
-    if (diskRegionProperties == null) {
-      diskRegionProperties = new DiskRegionProperties();
-    }
-    diskRegionProperties.setMaxOplogSize(0);
-    diskRegionProperties.setRolling(false);
-    diskRegionProperties.setPersistBackup(true);
-    diskRegionProperties.setSynchronous(true);
-    return getRegion(cache, diskRegionProperties, Scope.LOCAL);
-
-  }
-
-  public static Region getAsyncPersistOnlyRegionInfiniteOplog(Cache cache,
-      DiskRegionProperties diskRegionProperties) {
-    if (diskRegionProperties == null) {
-      diskRegionProperties = new DiskRegionProperties();
-    }
-    diskRegionProperties.setMaxOplogSize(0);
-    diskRegionProperties.setRolling(false);
-    diskRegionProperties.setPersistBackup(true);
-    diskRegionProperties.setSynchronous(false);
-    return getRegion(cache, diskRegionProperties, Scope.LOCAL);
-  }
-
-  public static Region getSyncOverFlowOnlyRegionInfiniteOplog(Cache cache,
-      DiskRegionProperties diskRegionProperties) {
-    if (diskRegionProperties == null) {
-      diskRegionProperties = new DiskRegionProperties();
-    }
-    diskRegionProperties.setMaxOplogSize(0);
-    diskRegionProperties.setRolling(false);
-    diskRegionProperties.setPersistBackup(false);
-    diskRegionProperties.setSynchronous(true);
-    diskRegionProperties.setOverflow(true);
-    return getRegion(cache, diskRegionProperties, Scope.LOCAL);
-  }
-
-  public static Region getAsyncOverFlowOnlyRegionInfiniteOplog(Cache cache,
-      DiskRegionProperties diskRegionProperties) {
-    if (diskRegionProperties == null) {
-      diskRegionProperties = new DiskRegionProperties();
-    }
-    diskRegionProperties.setMaxOplogSize(0);
-    diskRegionProperties.setRolling(false);
-    diskRegionProperties.setPersistBackup(false);
-    diskRegionProperties.setSynchronous(false);
-    diskRegionProperties.setOverflow(true);
-    return getRegion(cache, diskRegionProperties, Scope.LOCAL);
-  }
-
-  public static Region getSyncOverFlowAndPersistRegionInfiniteOplog(Cache cache,
-      DiskRegionProperties diskRegionProperties) {
-    if (diskRegionProperties == null) {
-      diskRegionProperties = new DiskRegionProperties();
-    }
-    diskRegionProperties.setMaxOplogSize(0);
-    diskRegionProperties.setRolling(false);
-    diskRegionProperties.setPersistBackup(true);
-    diskRegionProperties.setSynchronous(true);
-    diskRegionProperties.setOverflow(true);
-    return getRegion(cache, diskRegionProperties, Scope.LOCAL);
-  }
-
-  public static Region getAsyncOverFlowAndPersistRegionInfiniteOplog(Cache cache,
-      DiskRegionProperties diskRegionProperties) {
-    if (diskRegionProperties == null) {
-      diskRegionProperties = new DiskRegionProperties();
-    }
-    diskRegionProperties.setMaxOplogSize(0);
-    diskRegionProperties.setRolling(false);
-    diskRegionProperties.setPersistBackup(true);
-    diskRegionProperties.setSynchronous(false);
-    diskRegionProperties.setOverflow(true);
-    return getRegion(cache, diskRegionProperties, Scope.LOCAL);
-  }
-
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java
index 6abda96..7f225c0 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionJUnitTest.java
@@ -1415,7 +1415,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase {
 
     for (int i = 0; i < dirs.length; i++) {
       File[] files = dirs[i].listFiles();
-      assertTrue("Files already exists", files.length == 0);
+      assertTrue("Files already exists", files == null || files.length == 0);
     }
     region = DiskRegionHelperFactory.getAsyncOverFlowOnlyRegion(cache, diskProps);
 
@@ -1460,7 +1460,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase {
 
     for (int i = 0; i < dirs.length; i++) {
       File[] files = dirs[i].listFiles();
-      assertTrue("Files already exists", files.length == 0);
+      assertTrue("Files already exists", files == null || files.length == 0);
     }
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
 
@@ -2354,7 +2354,7 @@ public class DiskRegionJUnitTest extends DiskRegionTestingBase {
         }
       }
       assertTrue(i > 1);
-      assertTrue(switchedOplog[0].getOplogFile().delete());
+      assertTrue(switchedOplog[0].getOplogFileForTest().delete());
       region.close();
       // We don't validate the oplogs until we recreate the disk store.
       DiskStoreImpl store = ((LocalRegion) region).getDiskStore();
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionTestingBase.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionTestingBase.java
index 5a9c7b2..33c2fed 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionTestingBase.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskRegionTestingBase.java
@@ -34,6 +34,7 @@ import java.util.Properties;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
 import org.junit.rules.TestName;
 
 import org.apache.geode.LogWriter;
@@ -58,23 +59,28 @@ import org.apache.geode.internal.cache.versions.VersionTag;
  */
 public abstract class DiskRegionTestingBase {
 
-  protected static final boolean debug = false;
+  protected final boolean debug = false;
 
-  protected static Cache cache = null;
-  protected static DistributedSystem ds = null;
-  protected static Properties props = new Properties();
-  protected static File[] dirs = null;
-  protected static int[] diskDirSize = null;
+  protected Cache cache = null;
+  protected DistributedSystem ds = null;
+  protected Properties props = new Properties();
+  protected File[] dirs = null;
+  protected int[] diskDirSize = null;
 
-  protected Region region;
+  protected Region<Object, Object> region;
   protected LogWriter logWriter;
 
   boolean testFailed;
   String failureCause = "";
+  private File statsDir;
+  private File testingDirectory;
 
   @Rule
   public TestName name = new TestName();
 
+  @Rule
+  public TemporaryFolder tempDir = new TemporaryFolder();
+
   @Before
   public final void setUp() throws Exception {
     preSetUp();
@@ -84,27 +90,23 @@ public abstract class DiskRegionTestingBase {
     props.setProperty(LOG_LEVEL, "config"); // to keep diskPerf logs smaller
     props.setProperty(STATISTIC_SAMPLING_ENABLED, "true");
     props.setProperty(ENABLE_TIME_STATISTICS, "true");
-    props.setProperty(STATISTIC_ARCHIVE_FILE, "stats.gfs");
+    props.setProperty(STATISTIC_ARCHIVE_FILE, getStatsDir().getAbsolutePath() + "stats.gfs");
 
-    File testingDirectory = new File("testingDirectory");
-    testingDirectory.mkdir();
-    testingDirectory.deleteOnExit();
+    if (testingDirectory == null) {
+      testingDirectory = tempDir.newFolder("testingDirectory");
+    }
     failureCause = "";
     testFailed = false;
     cache = createCache();
 
-    File file1 = new File("testingDirectory/" + name.getMethodName() + "1");
+    File file1 = new File(testingDirectory, name.getMethodName() + "1");
     file1.mkdir();
-    file1.deleteOnExit();
-    File file2 = new File("testingDirectory/" + name.getMethodName() + "2");
+    File file2 = new File(testingDirectory, name.getMethodName() + "2");
     file2.mkdir();
-    file2.deleteOnExit();
-    File file3 = new File("testingDirectory/" + name.getMethodName() + "3");
+    File file3 = new File(testingDirectory, name.getMethodName() + "3");
     file3.mkdir();
-    file3.deleteOnExit();
-    File file4 = new File("testingDirectory/" + name.getMethodName() + "4");
+    File file4 = new File(testingDirectory, name.getMethodName() + "4");
     file4.mkdir();
-    file4.deleteOnExit();
     dirs = new File[4];
     dirs[0] = file1;
     dirs[1] = file2;
@@ -116,7 +118,6 @@ public abstract class DiskRegionTestingBase {
     diskDirSize[1] = Integer.MAX_VALUE;
     diskDirSize[2] = Integer.MAX_VALUE;
     diskDirSize[3] = Integer.MAX_VALUE;
-    deleteFiles();
 
     DiskStoreImpl.SET_IGNORE_PREALLOCATE = true;
 
@@ -133,8 +134,7 @@ public abstract class DiskRegionTestingBase {
 
     try {
       if (cache != null && !cache.isClosed()) {
-        for (Iterator itr = cache.rootRegions().iterator(); itr.hasNext();) {
-          Region root = (Region) itr.next();
+        for (Region root : cache.rootRegions()) {
           if (root.isDestroyed() || root instanceof HARegion) {
             continue;
           }
@@ -156,7 +156,6 @@ public abstract class DiskRegionTestingBase {
     } finally {
       closeCache();
     }
-    ds.disconnect();
     // Asif : below is not needed but leave it
     deleteFiles();
     DiskStoreImpl.SET_IGNORE_PREALLOCATE = false;
@@ -176,7 +175,7 @@ public abstract class DiskRegionTestingBase {
   }
 
   /** Close the cache */
-  private static synchronized void closeCache() {
+  private synchronized void closeCache() {
     if (cache != null) {
       try {
         if (!cache.isClosed()) {
@@ -198,46 +197,19 @@ public abstract class DiskRegionTestingBase {
   /**
    * cleans all the directory of all the files present in them
    */
-  protected static void deleteFiles() {
+  protected void deleteFiles() {
     closeDiskStores();
-    for (int i = 0; i < dirs.length; i++) {
-      System.out.println("trying to delete files in " + dirs[i].getAbsolutePath());
-      File[] files = dirs[i].listFiles();
-      for (int j = 0; j < files.length; j++) {
-        System.out.println("deleting " + files[j]);
-        int cnt = 0;
-        IOException ioe = null;
-        while (cnt < 3) {
-          try {
-            cnt++;
-            Files.delete(files[j].toPath());
-            break;
-          } catch (IOException e) {
-            ioe = e;
-            try {
-              Thread.sleep(1000);
-            } catch (Exception ignore) {
-            }
-          }
-        }
-        if (cnt >= 3) {
-          throw new RuntimeException("Error deleting file " + files[j], ioe);
-        }
-      }
-    }
+    tempDir.delete();
   }
 
-  protected static void closeDiskStores() {
+  protected void closeDiskStores() {
     if (cache != null) {
       ((GemFireCacheImpl) cache).closeDiskStores();
     }
   }
 
-  /**
-   * clears and closes the region
-   *
-   */
-  protected void closeDown() {
+
+  protected void closeDown(Region region) {
     try {
       if (!region.isDestroyed()) {
         region.destroyRegion();
@@ -249,6 +221,14 @@ public abstract class DiskRegionTestingBase {
   }
 
   /**
+   * clears and closes the region
+   *
+   */
+  protected void closeDown() {
+    closeDown(region);
+  }
+
+  /**
    * puts a 100 integers into the region
    */
   protected void put100Int() {
@@ -257,10 +237,6 @@ public abstract class DiskRegionTestingBase {
     }
   }
 
-  protected void verify100Int() {
-    verify100Int(true);
-  }
-
   protected void verify100Int(boolean verifySize) {
     if (verifySize) {
       assertEquals(100, region.size());
@@ -275,9 +251,8 @@ public abstract class DiskRegionTestingBase {
   /**
    * will keep on putting till region overflows
    */
-  protected void putTillOverFlow(Region region) {
-    int i = 0;
-    for (i = 0; i < 1010; i++) {
+  protected void putTillOverFlow(Region<Object, Object> region) {
+    for (int i = 0; i < 1010; i++) {
       region.put(i + 200, i + 200);
     }
   }
@@ -285,7 +260,7 @@ public abstract class DiskRegionTestingBase {
   /**
    * put an entry
    */
-  protected void putForValidation(Region region) {
+  protected void putForValidation(Region<Object, Object> region) {
     final byte[] value = new byte[1024];
     region.put("testKey", value);
   }
@@ -333,11 +308,11 @@ public abstract class DiskRegionTestingBase {
    * Since these are not visible to cache.diskPerf we add wrapper methods to make the following
    * parameters/visible
    */
-  public static void setCacheObserverCallBack() {
+  protected void setCacheObserverCallBack() {
     LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
   }
 
-  public static void unSetCacheObserverCallBack() {
+  protected void unSetCacheObserverCallBack() {
     LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
   }
 
@@ -376,4 +351,15 @@ public abstract class DiskRegionTestingBase {
     return name.getMethodName();
   }
 
+  protected File getTestingDirectory() {
+    return testingDirectory;
+  }
+
+  private File getStatsDir() throws IOException {
+    if (statsDir == null) {
+      statsDir = tempDir.newFolder("stats");
+    }
+    return statsDir;
+  }
+
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/OplogJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/OplogJUnitTest.java
index cb5832a..3008cda 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/OplogJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/OplogJUnitTest.java
@@ -24,7 +24,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.IntStream;
@@ -37,23 +36,20 @@ import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import org.apache.geode.StatisticsFactory;
-import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.CacheWriterException;
-import org.apache.geode.cache.CommitConflictException;
-import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.DiskAccessException;
 import org.apache.geode.cache.DiskStore;
 import org.apache.geode.cache.DiskStoreFactory;
 import org.apache.geode.cache.EntryEvent;
 import org.apache.geode.cache.EntryNotFoundException;
+import org.apache.geode.cache.RegionFactory;
+import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.Scope;
 import org.apache.geode.cache.util.CacheWriterAdapter;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.cache.Oplog.OPLOG_TYPE;
 import org.apache.geode.internal.cache.entries.DiskEntry;
 import org.apache.geode.test.dunit.ThreadUtils;
-import org.apache.geode.test.dunit.Wait;
-import org.apache.geode.test.dunit.WaitCriterion;
 import org.apache.geode.test.junit.categories.FlakyTest;
 import org.apache.geode.test.junit.categories.IntegrationTest;
 
@@ -63,52 +59,15 @@ import org.apache.geode.test.junit.categories.IntegrationTest;
 @Category(IntegrationTest.class)
 public class OplogJUnitTest extends DiskRegionTestingBase {
 
-  boolean proceed = false;
+  private boolean proceed = false;
 
   private final DiskRegionProperties diskProps = new DiskRegionProperties();
 
-  static final int OP_CREATE = 1;
+  private long delta;
 
-  static final int OP_MODIFY = 2;
+  private volatile boolean assertDone = false;
 
-  static final int OP_DEL = 3;
-
-  protected static volatile Random random = new Random();
-
-  protected long expectedOplogSize = Oplog.OPLOG_NEW_ENTRY_BASE_REC_SIZE;
-
-  volatile int totalSuccessfulOperations = 0;
-
-  protected int numCreate = 0;
-
-  protected int numModify = 0;
-
-  protected int numDel = 0;
-
-  protected long delta;
-
-  protected boolean flushOccurredAtleastOnce = false;
-
-  protected volatile boolean assertDone = false;
-
-  boolean failure = false;
-
-  /** The key for entry */
-  static final String KEY = "KEY1";
-
-  /** The initial value for key */
-  static final String OLD_VALUE = "VAL1";
-
-  /** The updated value for key */
-  static final String NEW_VALUE = "VAL2";
-
-  /** The value read from cache using LocalRegion.getValueOnDiskOrBuffer API */
-  static volatile String valueRead = null;
-
-  /** Boolean to indicate test to proceed for validation */
-  static volatile boolean proceedForValidation = false;
-
-  protected volatile Thread rollerThread = null;
+  private boolean failure = false;
 
   @Override
   protected final void postSetUp() throws Exception {
@@ -127,170 +86,66 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
   @Test
   public void testIsBackup() {
 
-    region = DiskRegionHelperFactory.getSyncOverFlowAndPersistRegion(cache, diskProps);
-    if (!((LocalRegion) region).getDiskRegion().isBackup()) {
-      fail("Test persist backup not being correctly set for overflow and persist");
-    }
-    closeDown();
-
-    region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
-    if (((LocalRegion) region).getDiskRegion().isBackup()) {
-      fail("Test persist backup not being correctly set for overflow only mode");
-    }
-    closeDown();
-
-    region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
-    if (!((LocalRegion) region).getDiskRegion().isBackup()) {
-      fail("Test persist backup not being correctly set for persist only");
-    }
-    closeDown();
+    InternalRegion overFlowAndPersistRegionRegion =
+        (InternalRegion) DiskRegionHelperFactory.getSyncOverFlowAndPersistRegion(cache, diskProps);
+    assertTrue("Not correctly setup for overflow and persist",
+        overFlowAndPersistRegionRegion.getDiskRegion().isBackup());
+    closeDown(overFlowAndPersistRegionRegion);
+
+    InternalRegion overFlowOnlyRegion =
+        (InternalRegion) DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
+    assertFalse("Not correctly setup for overflow only mode",
+        overFlowOnlyRegion.getDiskRegion().isBackup());
+    closeDown(overFlowOnlyRegion);
+
+    InternalRegion persistOnlyRegion = (InternalRegion) DiskRegionHelperFactory
+        .getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
+    assertTrue("Not correctly setup for  persist only mode",
+        persistOnlyRegion.getDiskRegion().isBackup());
+    closeDown(persistOnlyRegion);
   }
 
   /*
    * Test method for 'org.apache.geode.internal.cache.Oplog.useSyncWrites()'
    */
   @Test
-  public void testUseSyncWrites() {
-    boolean result;
+  public void testUseSyncWritesWhenSet() {
     diskProps.setSynchronous(true);
-    region = DiskRegionHelperFactory.getSyncOverFlowAndPersistRegion(cache, diskProps);
-    result = ((LocalRegion) region).getAttributes().isDiskSynchronous();
-    if (!result) {
-      fail("Synchronous is false when it is supposed to be true");
-    }
-    closeDown();
-
-    region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
-
-    result = ((LocalRegion) region).getAttributes().isDiskSynchronous();
-    if (!result) {
-      fail("Synchronous is false when it is supposed to be true");
-    }
-    closeDown();
-
-    region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
-
-    result = ((LocalRegion) region).getAttributes().isDiskSynchronous();
-    if (!result) {
-      fail("Synchronous is false when it is supposed to be true");
-    }
-    closeDown();
+    InternalRegion syncOverFlowAndPersistRegion =
+        (InternalRegion) DiskRegionHelperFactory.getSyncOverFlowAndPersistRegion(cache, diskProps);
+    assertTrue(syncOverFlowAndPersistRegion.getAttributes().isDiskSynchronous());
+    closeDown(syncOverFlowAndPersistRegion);
+
+    InternalRegion syncOverFlowOnlyRegion =
+        (InternalRegion) DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
+    assertTrue(syncOverFlowOnlyRegion.getAttributes().isDiskSynchronous());
+    closeDown(syncOverFlowOnlyRegion);
+
+    InternalRegion syncPersistOnlyRegion = (InternalRegion) DiskRegionHelperFactory
+        .getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
+    assertTrue(syncPersistOnlyRegion.getAttributes().isDiskSynchronous());
+    closeDown(syncPersistOnlyRegion);
+  }
 
+  @Test
+  public void testNotUseSyncWritesWhenNotSet() {
     diskProps.setSynchronous(false);
-    region = DiskRegionHelperFactory.getAsyncOverFlowAndPersistRegion(cache, diskProps);
-
-    result = ((LocalRegion) region).getAttributes().isDiskSynchronous();
-    if (result) {
-      fail("Synchronous is true when it is supposed to be false");
-    }
-
-    closeDown();
-    region = DiskRegionHelperFactory.getAsyncOverFlowOnlyRegion(cache, diskProps);
-
-    result = ((LocalRegion) region).getAttributes().isDiskSynchronous();
-    if (result) {
-      fail("Synchronous is true when it is supposed to be false");
-    }
-    closeDown();
-
-    region = DiskRegionHelperFactory.getAsyncPersistOnlyRegion(cache, diskProps);
-
-    result = ((LocalRegion) region).getAttributes().isDiskSynchronous();
-    if (result) {
-      fail("Synchronous is true when it is supposed to be false");
-    }
-    closeDown();
+    InternalRegion asyncOverFlowAndPersistRegion =
+        (InternalRegion) DiskRegionHelperFactory.getAsyncOverFlowAndPersistRegion(cache, diskProps);
+    assertFalse(asyncOverFlowAndPersistRegion.getAttributes().isDiskSynchronous());
+    closeDown(asyncOverFlowAndPersistRegion);
+
+    InternalRegion asyncOverFlowOnlyRegion =
+        (InternalRegion) DiskRegionHelperFactory.getAsyncOverFlowOnlyRegion(cache, diskProps);
+    assertFalse(asyncOverFlowOnlyRegion.getAttributes().isDiskSynchronous());
+    closeDown(asyncOverFlowOnlyRegion);
+
+    InternalRegion asyncPersistOnlyRegion =
+        (InternalRegion) DiskRegionHelperFactory.getAsyncPersistOnlyRegion(cache, diskProps);
+    assertFalse(asyncPersistOnlyRegion.getAttributes().isDiskSynchronous());
+    closeDown(asyncPersistOnlyRegion);
   }
 
-  // @todo port testBufferOperations
-  /**
-   * Asif: Tests the correct behaviour of attributes like byte-threshhold, asynch thread wait
-   * time,etc. 'org.apache.geode.internal.cache.Oplog.bufferOperations()'
-   */
-  // @Test
-  // public void testBufferOperations()
-  // {
-  // boolean result;
-
-  // diskProps.setBytesThreshold(0);
-  // region = DiskRegionHelperFactory.getAsyncOverFlowAndPersistRegion(cache,
-  // diskProps);
-  // Oplog.WriterThread writer = ((LocalRegion)region).getDiskRegion()
-  // .getChild().getAsynchWriter();
-  // long waitTime = writer.getAsynchThreadWaitTime();
-  // long buffSize = writer.getBufferSize();
-  // result = waitTime == writer.getDefaultAsynchThreadWaitTime()
-  // && buffSize == 0;
-
-  // assertTrue("buffer operations is true when it is supposed to be false",
-  // result);
-
-  // closeDown();
-
-  // region = DiskRegionHelperFactory.getAsyncOverFlowOnlyRegion(cache,
-  // diskProps);
-  // writer = ((LocalRegion)region).getDiskRegion().getChild().getAsynchWriter();
-  // waitTime = writer.getAsynchThreadWaitTime();
-  // buffSize = writer.getBufferSize();
-  // result = waitTime == writer.getDefaultAsynchThreadWaitTime()
-  // && buffSize == 0;
-
-  // assertTrue("buffer operations is true when it is supposed to be false",
-  // result);
-  // closeDown();
-
-  // region = DiskRegionHelperFactory
-  // .getAsyncPersistOnlyRegion(cache, diskProps);
-  // writer = ((LocalRegion)region).getDiskRegion().getChild().getAsynchWriter();
-  // waitTime = writer.getAsynchThreadWaitTime();
-  // buffSize = writer.getBufferSize();
-  // result = waitTime == writer.getDefaultAsynchThreadWaitTime()
-  // && buffSize == 0;
-
-  // assertTrue("buffer operations is true when it is supposed to be false",
-  // result);
-
-  // closeDown();
-
-  // diskProps.setBytesThreshold(100);
-
-  // region = DiskRegionHelperFactory.getAsyncOverFlowAndPersistRegion(cache,
-  // diskProps);
-
-  // writer = ((LocalRegion)region).getDiskRegion().getChild().getAsynchWriter();
-  // waitTime = writer.getAsynchThreadWaitTime();
-  // buffSize = writer.getBufferSize();
-  // result = waitTime <= 0 && buffSize > 0;
-  // assertTrue("bufferoperations is false when it is supposed to be true",
-  // result);
-
-  // closeDown();
-
-  // region = DiskRegionHelperFactory.getAsyncOverFlowOnlyRegion(cache,
-  // diskProps);
-
-  // writer = ((LocalRegion)region).getDiskRegion().getChild().getAsynchWriter();
-  // waitTime = writer.getAsynchThreadWaitTime();
-  // buffSize = writer.getBufferSize();
-  // result = waitTime <= 0 && buffSize > 0;
-  // assertTrue("baufferoperations is false when it is supposed to be true",
-  // result);
-
-  // closeDown();
-
-  // region = DiskRegionHelperFactory
-  // .getAsyncPersistOnlyRegion(cache, diskProps);
-
-  // writer = ((LocalRegion)region).getDiskRegion().getChild().getAsynchWriter();
-  // waitTime = writer.getAsynchThreadWaitTime();
-  // buffSize = writer.getBufferSize();
-  // result = waitTime <= 0 && buffSize > 0;
-  // assertTrue("baufferoperations is false when it is supposed to be true",
-  // result);
-
-  // closeDown();
-  // }
-
   /**
    * Test method for 'org.apache.geode.internal.cache.Oplog.clear(File)'
    */
@@ -301,7 +156,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     region.clear();
     region.close();
     region = DiskRegionHelperFactory.getSyncOverFlowAndPersistRegion(cache, diskProps);
-    assertTrue(" failed in get OverflowAndPersist ", region.get(new Integer(0)) == null);
+    assertTrue(" failed in get OverflowAndPersist ", region.get(0) == null);
     closeDown();
 
     region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
@@ -309,7 +164,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     region.clear();
     region.close();
     region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
-    assertTrue(" failed in get OverflowOnly ", region.get(new Integer(0)) == null);
+    assertTrue(" failed in get OverflowOnly ", region.get(0) == null);
     closeDown();
 
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
@@ -317,7 +172,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     region.clear();
     region.close();
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
-    assertTrue(" failed in get PersistOnly ", region.get(new Integer(0)) == null);
+    assertTrue(" failed in get PersistOnly ", region.get(0) == null);
     closeDown();
   }
 
@@ -333,10 +188,8 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       Oplog oplog = dr.testHook_getChild();
       long id = oplog.getOplogId();
       oplog.close();
-      // lk should still exist since it locks DiskStore not just one oplog
-      // checkIfContainsFile(".lk");
 
-      StatisticsFactory factory = region.getCache().getDistributedSystem();
+      StatisticsFactory factory = cache.getDistributedSystem();
       Oplog newOplog =
           new Oplog(id, dr.getOplogSet(), new DirectoryHolder(factory, dirs[0], 1000, 0));
       dr.getOplogSet().setChild(newOplog);
@@ -347,8 +200,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
       DiskRegion dr = ((LocalRegion) region).getDiskRegion();
       dr.testHookCloseAllOverflowOplogs();
-      // lk should still exist since it locks DiskStore not just one oplog
-      // checkIfContainsFile(".lk");
       checkIfContainsFile("OVERFLOW");
       closeDown();
     }
@@ -359,9 +210,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       Oplog oplog = dr.testHook_getChild();
       long id = oplog.getOplogId();
       oplog.close();
-      // lk should still exist since it locks DiskStore not just one oplog
-      // checkIfContainsFile(".lk");
-      StatisticsFactory factory = region.getCache().getDistributedSystem();
+      StatisticsFactory factory = cache.getDistributedSystem();
       Oplog newOplog =
           new Oplog(id, dr.getOplogSet(), new DirectoryHolder(factory, dirs[0], 1000, 2));
       dr.setChild(newOplog);
@@ -370,6 +219,15 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
   }
 
+  private void closeDown(InternalRegion region) {
+    super.closeDown(region);
+    DiskRegion diskRegion = region != null ? region.getDiskRegion() : null;
+    if (diskRegion != null) {
+      diskRegion.getDiskStore().close();
+      ((InternalCache) cache).removeDiskStore(diskRegion.getDiskStore());
+    }
+  }
+
   @Override
   protected void closeDown() {
     DiskRegion dr = null;
@@ -383,14 +241,12 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     }
   }
 
-
-
-  void checkIfContainsFile(String fileExtension) {
-    for (int i = 0; i < 4; i++) {
-      File[] files = dirs[i].listFiles();
-      for (int j = 0; j < files.length; j++) {
-        if (files[j].getAbsolutePath().endsWith(fileExtension)) {
-          fail("file " + files[j] + " still exists after oplog.close()");
+  private void checkIfContainsFile(String fileExtension) {
+    for (File dir : dirs) {
+      File[] files = dir.listFiles();
+      for (File file : files) {
+        if (file.getAbsolutePath().endsWith(fileExtension)) {
+          fail("file " + file + " still exists after oplog.close()");
         }
       }
     }
@@ -405,42 +261,42 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     put100Int();
     putTillOverFlow(region);
     try {
-      region.destroy(new Integer(0));
+      region.destroy(0);
     } catch (EntryNotFoundException e1) {
       logWriter.error("Exception occurred", e1);
       fail(" Entry not found when it was expected to be there");
     }
     region.close();
     region = DiskRegionHelperFactory.getSyncOverFlowAndPersistRegion(cache, diskProps);
-    assertTrue(" failed in get OverflowAndPersist ", region.get(new Integer(0)) == null);
+    assertTrue(" failed in get OverflowAndPersist ", region.get(0) == null);
     closeDown();
 
     region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
     put100Int();
     putTillOverFlow(region);
     try {
-      region.destroy(new Integer(0));
+      region.destroy(0);
     } catch (EntryNotFoundException e1) {
       logWriter.error("Exception occurred", e1);
       fail(" Entry not found when it was expected to be there");
     }
     region.close();
     region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
-    assertTrue(" failed in get OverflowOnly ", region.get(new Integer(0)) == null);
+    assertTrue(" failed in get OverflowOnly ", region.get(0) == null);
 
     closeDown();
 
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
     put100Int();
     try {
-      region.destroy(new Integer(0));
+      region.destroy(0);
     } catch (EntryNotFoundException e1) {
       logWriter.error("Exception occurred", e1);
       fail(" Entry not found when it was expected to be there");
     }
     region.close();
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
-    assertTrue(" failed in get PersistOnly ", region.get(new Integer(0)) == null);
+    assertTrue(" failed in get PersistOnly ", region.get(0) == null);
     closeDown();
 
   }
@@ -452,119 +308,30 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
   public void testRemove() {
     region = DiskRegionHelperFactory.getSyncOverFlowAndPersistRegion(cache, diskProps);
     putTillOverFlow(region);
-    region.remove(new Integer(0));
+    region.remove(0);
     region.close();
     region = DiskRegionHelperFactory.getSyncOverFlowAndPersistRegion(cache, diskProps);
-    assertTrue(" failed in get OverflowAndPersist ", region.get(new Integer(0)) == null);
+    assertTrue(" failed in get OverflowAndPersist ", region.get(0) == null);
     closeDown();
 
     region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
     putTillOverFlow(region);
-    region.remove(new Integer(0));
-    assertTrue(" failed in get OverflowOnly ", region.get(new Integer(0)) == null);
+    region.remove(0);
+    assertTrue(" failed in get OverflowOnly ", region.get(0) == null);
     region.close();
     region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
     closeDown();
 
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
     put100Int();
-    region.remove(new Integer(0));
-    assertTrue(" failed in get PersistOnly ", region.get(new Integer(0)) == null);
+    region.remove(0);
+    assertTrue(" failed in get PersistOnly ", region.get(0) == null);
     region.close();
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
     closeDown();
 
   }
 
-  // @todo: port testByteBufferCreationForCreateModifyAndDeleteOperation
-  /**
-   * This tests the final ByteBuffer object that gets created for synch/Asynch operation for a
-   * create / modify & Delete operation
-   *
-   */
-  // @Test
-  // public void testByteBufferCreationForCreateModifyAndDeleteOperation()
-  // {
-  // // Asif First create a persist only disk region which is of aysnch
-  // // & switch of OplOg type
-  // diskProps.setMaxOplogSize(1000);
-  // diskProps.setBytesThreshold(500);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(false);
-  // diskProps.setTimeInterval(-1);
-  // diskProps.setOverflow(false);
-
-  // region = DiskRegionHelperFactory
-  // .getAsyncPersistOnlyRegion(cache, diskProps);
-  // byte[] val = new byte[10];
-  // for (int i = 0; i < 10; ++i) {
-  // val[i] = (byte)i;
-  // }
-  // region.put(new Integer(1), val);
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(new Integer(1)));
-  // long opKey = entry.getDiskId().getKeyId();
-  // // The final position in the Byte Buffer created in Asynch Op should be
-  // int createPos = 2 + 4 + val.length;
-  // if (opKey > Integer.MAX_VALUE) {
-  // createPos += 8;
-  // }
-  // else if (opKey > Short.MAX_VALUE) {
-  // createPos += 4;
-  // }
-  // else {
-  // createPos += 2;
-  // }
-  // createPos += 4;
-  // createPos += EntryEventImpl.serialize(new Integer(1)).length;
-  // DiskRegion dr = ((LocalRegion)region).getDiskRegion();
-  // Oplog.WriterThread writer = dr.getChild().getAsynchWriter();
-  // Oplog.AsyncOp asynchOp = writer
-  // .getAsynchOpForEntryFromPendingFlushMap(entry.getDiskId());
-  // ByteBuffer bb = asynchOp.getByteBuffer();
-  // assertTrue(createPos == bb.position());
-  // assertTrue(bb.limit() == bb.capacity());
-  // byte val1[] = new byte[20];
-  // for (int i = 0; i < 20; ++i) {
-  // val1[i] = (byte)i;
-  // }
-  // region.put(new Integer(1), val1);
-  // bb = writer.getAsynchOpForEntryFromPendingFlushMap(entry.getDiskId())
-  // .getByteBuffer();
-  // createPos += 10;
-  // assertTrue(createPos == bb.position());
-  // assertTrue(bb.limit() == bb.capacity());
-  // byte val2[] = new byte[30];
-  // for (int i = 0; i < 30; ++i) {
-  // val2[i] = (byte)i;
-  // }
-  // region.put(new Integer(1), val2);
-  // bb = writer.getAsynchOpForEntryFromPendingFlushMap(entry.getDiskId())
-  // .getByteBuffer();
-  // createPos += 10;
-  // assertTrue(createPos == bb.position());
-  // assertTrue(bb.limit() == bb.capacity());
-  // long opSizeBeforeCreateRemove = dr.getChild().getOplogSize();
-  // long pendingFlushSize = dr.getChild().getAsynchWriter()
-  // .getCurrentBufferedBytesSize();
-  // region.put(new Integer(2), val2);
-  // DiskEntry entry2 = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(new Integer(2)));
-  // bb = writer.getAsynchOpForEntryFromPendingFlushMap(entry2.getDiskId())
-  // .getByteBuffer();
-  // assertNotNull(bb);
-  // region.remove(new Integer(2));
-  // assertNull(writer
-  // .getAsynchOpForEntryFromPendingFlushMap(entry2.getDiskId()));
-  // assertIndexDetailsEquals(opSizeBeforeCreateRemove, dr.getChild().getOplogSize());
-  // assertIndexDetailsEquals(pendingFlushSize, dr.getChild().getAsynchWriter()
-  // .getCurrentBufferedBytesSize());
-
-  // closeDown();
-
-  // }
-
   /**
    * Tests whether the data is written in the right format on the disk
    *
@@ -584,28 +351,28 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
       region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
       byte[] val = new byte[10];
-      for (int i = 0; i < 10; ++i) {
+      for (int i = 0; i < val.length; ++i) {
         val[i] = (byte) i;
       }
-      region.put(new Integer(1), val);
+      region.put(1, val);
 
-      DiskEntry entry = ((DiskEntry) ((LocalRegion) region).basicGetEntry(new Integer(1)));
+      DiskEntry entry = ((DiskEntry) ((LocalRegion) region).basicGetEntry(1));
       DiskRegion dr = ((LocalRegion) region).getDiskRegion();
 
       val = (byte[]) dr.getNoBuffer(entry.getDiskId());
-      for (int i = 0; i < 10; ++i) {
+      for (int i = 0; i < val.length; ++i) {
         if (val[i] != (byte) i) {
           fail("Test for fault in from disk failed");
         }
       }
       val = (byte[]) DiskStoreImpl.convertBytesAndBitsIntoObject(
           dr.getBytesAndBitsWithoutLock(entry.getDiskId(), true, false));
-      for (int i = 0; i < 10; ++i) {
+      for (int i = 0; i < val.length; ++i) {
         if (val[i] != (byte) i) {
           fail("Test for fault in from disk failed");
         }
       }
-      region.invalidate(new Integer(1));
+      region.invalidate(1);
       assertTrue(dr.getNoBuffer(entry.getDiskId()) == Token.INVALID);
 
     } catch (Exception e) {
@@ -615,60 +382,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     closeDown();
   }
 
-  // @todo port testAsynchWriterTerminationOnSwitch
-  /**
-   * Tests the termination of asynch writer for an Oplog after the switch has been made
-   *
-   */
-  // @Test
-  // public void testAsynchWriterTerminationOnSwitch()
-  // {
-  // // & switch of OplOg type
-  // diskProps.setMaxOplogSize(23);
-  // diskProps.setBytesThreshold(0);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(false);
-  // diskProps.setTimeInterval(10000);
-  // diskProps.setOverflow(false);
-  // // diskProps.setDiskDirs(new File[]{new File("test1"), new
-  // // File("test2"),
-  // // new File("test3")});
-
-  // region = DiskRegionHelperFactory
-  // .getAsyncPersistOnlyRegion(cache, diskProps);
-  // DiskRegion dr = ((LocalRegion)region).getDiskRegion();
-  // Oplog.WriterThread writer = dr.getChild().getAsynchWriter();
-  // // Populate data just below the switch over threshhold
-  // byte[] val = new byte[5];
-  // for (int i = 0; i < 5; ++i) {
-  // val[i] = (byte)i;
-  // }
-
-  // region.put(new Integer(1), val);
-
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(new Integer(1)));
-  // long opKey = entry.getDiskId().getKeyId();
-  // // The final position in the Byte Buffer created in Asynch Op should be
-  // int createPos = 2 + 4 + val.length;
-  // if (opKey > Integer.MAX_VALUE) {
-  // createPos += 8;
-  // }
-  // else if (opKey > Short.MAX_VALUE) {
-  // createPos += 4;
-  // }
-  // else {
-  // createPos += 2;
-  // }
-  // createPos += 4;
-  // createPos += EntryEventImpl.serialize(new Integer(1)).length;
-  // assertTrue(createPos == 22);
-  // region.put(new Integer(2), val);
-  // DistributedTestCase.join(writer.getThread(), 10 * 1000, null);
-  // closeDown();
-  // }
-
   /**
    * Tests the original ByteBufferPool gets transferred to the new Oplog for synch mode
    *
@@ -682,26 +395,22 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     diskProps.setSynchronous(true);
     diskProps.setTimeInterval(10000);
     diskProps.setOverflow(false);
-    // diskProps.setDiskDirs(new File[]{new File("test1"), new
-    // File("test2"),
-    // new File("test3")});
 
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
     DiskRegion dr = ((LocalRegion) region).getDiskRegion();
-    // assertNull(dr.getChild().getAsynchWriter());
     // Populate data just below the switch over threshhold
     byte[] val = new byte[5];
-    for (int i = 0; i < 5; ++i) {
+    for (int i = 0; i < val.length; ++i) {
       val[i] = (byte) i;
     }
 
-    region.put(new Integer(1), val);
+    region.put(1, val);
 
-    ((LocalRegion) region).basicGetEntry(new Integer(1));
+    ((LocalRegion) region).basicGetEntry(1);
     Oplog old = dr.testHook_getChild();
     ByteBuffer oldWriteBuf = old.getWriteBuf();
-    region.forceRolling(); // start a new oplog
-    region.put(new Integer(2), val);
+    dr.forceRolling();
+    region.put(2, val);
     Oplog switched = dr.testHook_getChild();
     assertTrue(old != switched);
     assertEquals(dr.getDiskStore().persistentOplogs.getChild(2), switched);
@@ -711,872 +420,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
   }
 
-  // @todo port this test if needed. ByteBufferPool code is going to change
-  /**
-   * Tests the ByteBufferPool usage during asynch mode operation & ensuring that GetOperation does
-   * not get corrupted data due to returing of ByetBuffer to the pool. There are 5 pre created pools
-   * in Oplog . Each pool has size of 1. Out of 5 pools , only one pool is used by the test. Thus
-   * there are 4 bytebuffers which will always be free. Thus if the asynch writer had initially 8
-   * byte buffers only 4 will be released
-   *
-   */
-  // @Test
-  // public void testByteBufferPoolUsageForAsynchMode()
-  // {
-  // final int PRCREATED_POOL_NUM = 5;
-  // try {
-  // // Asif First create a persist only disk region which is of aysnch
-  // // & switch of OplOg type
-  // diskProps.setMaxOplogSize(1000);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(false);
-  // diskProps.setTimeInterval(-1);
-  // diskProps.setOverflow(false);
-  // final int byte_threshold = 500;
-  // diskProps.setBytesThreshold(byte_threshold);
-  // byte[] val = new byte[50];
-  // region = DiskRegionHelperFactory.getAsyncPersistOnlyRegion(cache,
-  // diskProps);
-  // for (int i = 0; i < 50; ++i) {
-  // val[i] = (byte)i;
-  // }
-  // region.put(new Integer(1), val);
-  // final int singleOpSize = evaluateSizeOfOperationForPersist(
-  // new Integer(1), val, ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(new Integer(1))).getDiskId(), OP_CREATE);
-
-  // final int loopCount = byte_threshold / singleOpSize + 1;
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
-
-  // final Thread th = new Thread(new Runnable() {
-  // public void run()
-  // {
-  // takeRecursiveLockOnAllEntries(1);
-  // DiskRegion dr = ((LocalRegion)region).getDiskRegion();
-  // // Asif : Sleep for somemore time
-  // try {
-  // Thread.yield();
-  // Thread.sleep(4000);
-  // }
-  // catch (InterruptedException ie) {
-  // logWriter.error("Exception occurred", ie);
-  // failureCause = "No guarantee of vaildity of result hence failing. Exception = "
-  // + ie;
-  // testFailed = true;
-  // fail("No guarantee of vaildity of result hence failing. Exception = "
-  // + ie);
-  // }
-
-  // // There shoudl beatleast one Pool which has active counts
-  // // as two
-  // Oplog.ByteBufferPool bbp = null;
-  // List pools = dr.getChild().getByteBufferPoolList();
-  // Iterator itr = pools.iterator();
-  // boolean found = false;
-  // while (itr.hasNext()) {
-  // bbp = (Oplog.ByteBufferPool)itr.next();
-  // int len = bbp.getByteBufferHolderList().size();
-  // if (len == (loopCount - (PRCREATED_POOL_NUM - 1))) {
-  // found = true;
-  // break;
-  // }
-  // }
-
-  // if (!found) {
-  // testFailed = true;
-  // failureCause = "Test failed as the Asynch writer did not release ByetBuffer after get
-  // operation";
-  // fail("Test failed as the Asynch writer did not release ByetBuffer after get operation");
-
-  // }
-
-  // }
-
-  // private void takeRecursiveLockOnAllEntries(int key)
-  // {
-  // // Get the DisKID
-  // DiskRegion dr = ((LocalRegion)region).getDiskRegion();
-  // if (key > loopCount) {
-  // // Interrupt the writer thread so as to start releasing
-  // // bytebuffer to pool
-  // //dr.getChild().getAsynchWriter().interrupt();
-  // // Sleep for a while & check the active ByteBuffer
-  // // count.
-  // // It should be two
-  // try {
-  // Thread.yield();
-  // Thread.sleep(5000);
-  // }
-  // catch (InterruptedException ie) {
-  // logWriter.error("Exception occurred", ie);
-  // failureCause = "No guarantee of vaildity of result hence failing. Exception = "
-  // + ie;
-  // testFailed = true;
-  // fail("No guarantee of vaildity of result hence failing. Exception = "
-  // + ie);
-  // }
-  // // Check the number of ByteBuffers in the pool.
-  // List pools = dr.getChild().getByteBufferPoolList();
-  // // There shoudl beatleast one Pool which has active
-  // // counts as two
-  // Oplog.ByteBufferPool bbp = null;
-  // Iterator itr = pools.iterator();
-  // boolean found = true;
-  // int len = -1;
-  // while (itr.hasNext()) {
-  // bbp = (Oplog.ByteBufferPool)itr.next();
-  // len = bbp.getByteBufferHolderList().size();
-  // if (len > 1) {
-  // found = false;
-  // break;
-  // }
-  // }
-  // if (!found) {
-  // failureCause = "Test failed as the Asynch writer released ByteBuffer before get operation. The
-  // length of byte buffer pool is found to be greater than 0. the length is"
-  // + len;
-  // testFailed = true;
-  // fail("Test failed as the Asynch writer released ByteBuffer before get operation");
-  // }
-  // }
-  // else {
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(new Integer(key)));
-  // DiskId id = entry.getDiskId();
-
-  // synchronized (id) {
-  // takeRecursiveLockOnAllEntries(++key);
-
-  // }
-  // }
-  // }
-
-  // });
-
-  // CacheObserver old = CacheObserverHolder
-  // .setInstance(new CacheObserverAdapter() {
-  // public void afterWritingBytes()
-  // {
-  // // Asif Start a Thread & do a get in the thread without
-  // // releasing the
-  // // lock on dik ID
-  // th.start();
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.proceed = true;
-  // OplogJUnitTest.this.notify();
-  // }
-  // try {
-  // th.join(30 * 1000); // Yes, really use Thread#join here
-  // fail("never interrupted");
-  // }
-  // catch (InterruptedException ie) {
-  // // OK. Expected the interrupted Exception
-  // if (debug)
-  // System.out.println("Got the right exception");
-  // }
-
-  // }
-  // });
-
-  // int totalOpSize = singleOpSize;
-  // for (int j = 1; j < loopCount; ++j) {
-  // region.put(new Integer(j + 1), val);
-  // totalOpSize += evaluateSizeOfOperationForPersist(new Integer(j + 1),
-  // val, ((DiskEntry)((LocalRegion)region).basicGetEntry(new Integer(
-  // j + 1))).getDiskId(), OP_CREATE);
-  // }
-  // assertTrue(totalOpSize - byte_threshold <= singleOpSize);
-
-  // if (!proceed) {
-  // synchronized (this) {
-  // if (!proceed) {
-  // this.wait(25000);
-  // if (!proceed) {
-  // fail("Test failed as no callback recieved from asynch writer");
-  // }
-  // }
-  // }
-  // }
-  // DistributedTestCase.join(th, 30 * 1000, null);
-  // CacheObserverHolder.setInstance(old);
-  // }
-  // catch (Exception e) {
-  // logWriter.error("Exception occurred", e);
-  // fail(e.toString());
-  // }
-  // assertFalse(failureCause, testFailed);
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
-  // closeDown();
-  // }
-
-  // give the new oplog record format it is too hard for the test to calculate
-  // the expected size
-  // /**
-  // */
-  // @Test
-  // public void testSynchModeConcurrentOperations()
-  // {
-  // final Map map = new HashMap();
-  // diskProps.setMaxOplogSize(1024 * 1024 * 20);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(true);
-  // diskProps.setOverflow(false);
-  // final int THREAD_COUNT = 90;
-
-  // final byte[] val = new byte[50];
-  // region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps);
-  // for (int i = 1; i < 101; ++i) {
-  // map.put(new Integer(i), new Integer(i));
-  // }
-  // Thread[] threads = new Thread[THREAD_COUNT];
-  // for (int i = 0; i < THREAD_COUNT; ++i) {
-  // threads[i] = new Thread(new Runnable() {
-
-  // public void run()
-  // {
-  // int sizeOfOp = 0;
-  // DiskId id = null;
-  // for (int j = 0; j < 50; ++j) {
-  // int keyNum = random.nextInt(10) + 1;
-  // Integer key = new Integer(keyNum);
-  // Integer intgr = (Integer)map.get(key);
-  // try {
-  // synchronized (intgr) {
-
-  // region.create(key, val);
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(key));
-  // id = entry.getDiskId();
-
-  // }
-  // sizeOfOp = OplogJUnitTest.evaluateSizeOfOperationForPersist(key,
-  // val, id, OP_CREATE);
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.expectedOplogSize += sizeOfOp;
-  // ++OplogJUnitTest.this.totalSuccessfulOperations;
-  // ++OplogJUnitTest.this.numCreate;
-  // }
-  // }
-  // catch (EntryExistsException eee) {
-  // if (OplogJUnitTest.this.logWriter.finerEnabled()) {
-  // OplogJUnitTest.this.logWriter
-  // .finer("The entry already exists so this operation will not increase the size of oplog");
-  // }
-  // }
-  // try {
-  // boolean isUpdate = false;
-  // synchronized (intgr) {
-  // isUpdate = region.containsKey(key);
-  // region.put(key, val);
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(key));
-  // id = entry.getDiskId();
-  // }
-  // sizeOfOp = OplogJUnitTest.evaluateSizeOfOperationForPersist(key,
-  // val, id, (isUpdate ? OP_MODIFY : OP_CREATE));
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.expectedOplogSize += sizeOfOp;
-  // ++OplogJUnitTest.this.totalSuccessfulOperations;
-  // if (!isUpdate) {
-  // ++OplogJUnitTest.this.numCreate;
-  // }
-  // else {
-  // ++OplogJUnitTest.this.numModify;
-  // }
-  // }
-  // }
-  // catch (EntryDestroyedException ede) {
-  // if (OplogJUnitTest.this.logWriter.finerEnabled()) {
-  // OplogJUnitTest.this.logWriter
-  // .finer("The entry already exists so this operation will not increase the size of oplog");
-  // }
-  // }
-
-  // boolean deleted = false;
-  // synchronized (intgr) {
-  // if (region.containsKey(key)) {
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(key));
-  // id = entry.getDiskId();
-  // region.remove(key);
-  // deleted = true;
-  // }
-
-  // }
-  // if (deleted) {
-  // sizeOfOp = OplogJUnitTest.evaluateSizeOfOperationForPersist(key,
-  // null, id, OP_DEL);
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.expectedOplogSize += sizeOfOp;
-  // ++OplogJUnitTest.this.totalSuccessfulOperations;
-  // ++OplogJUnitTest.this.numDel;
-
-  // }
-  // }
-
-  // }
-
-  // }
-
-  // });
-  // threads[i].start();
-  // }
-
-  // for (int i = 0; i < THREAD_COUNT; ++i) {
-  // DistributedTestCase.join(threads[i], 30 * 1000, null);
-  // }
-  // long inMemOplogSize = 0;
-  // File opFile = null;
-  // try {
-  // opFile = ((LocalRegion)region).getDiskRegion().getChild().getOplogFile();
-  // }
-  // catch (Exception e) {
-  // logWriter
-  // .error(
-  // "Exception in synching data present in the buffers of RandomAccessFile of Oplog, to the disk",
-  // e);
-  // fail("Test failed because synching of data present in buffer of RandomAccesFile ");
-  // }
-  // synchronized (opFile) {
-  // inMemOplogSize = ((LocalRegion)region).getDiskRegion().getChild().getOplogSize();
-  // }
-
-  // long actFileSize = 0;
-  // try {
-
-  // actFileSize = ((LocalRegion)region).getDiskRegion().getChild().testGetOplogFileLength();
-  // }
-  // catch (IOException e) {
-
-  // fail("exception not expected" + e);
-  // fail("The test failed as the oplog could not eb synched to disk");
-  // }
-  // assertIndexDetailsEquals((this.numCreate + this.numDel + this.numModify),
-  // this.totalSuccessfulOperations);
-  // assertTrue(" The expected oplog size =" + inMemOplogSize
-  // + " Actual Oplog file size =" + actFileSize,
-  // inMemOplogSize == actFileSize);
-  // assertTrue(" The expected oplog size =" + this.expectedOplogSize
-  // + " In memeory Oplog size =" + inMemOplogSize,
-  // this.expectedOplogSize == inMemOplogSize);
-  // closeDown();
-
-  // }
-
-  static int evaluateSizeOfOperationForPersist(Object key, byte[] val, DiskId id,
-      int OperationType) {
-    int size = 1;
-    long opKey = id.getKeyId();
-    switch (OperationType) {
-      case OP_CREATE:
-        size += 4 + EntryEventImpl.serialize(key).length + 1 + 4 + val.length;
-        break;
-
-      case OP_MODIFY:
-        // @todo how do a know if the key needed to be serialized?
-        size += 1 + 4 + val.length + Oplog.bytesNeeded(opKey);
-        break;
-      case OP_DEL:
-        size += Oplog.bytesNeeded(opKey);
-        break;
-    }
-    return size;
-
-  }
-
-  // give the new oplog record format it is too hard for the test to calculate
-  // the expected size
-  // /**
-  // * Tests whether the switching of Oplog happens correctly without size
-  // * violation in case of concurrent region operations for synch mode.
-  // */
-  // @Test
-  // public void testSwitchingForConcurrentSynchedOperations()
-  // {
-  // final Map map = new HashMap();
-  // final int MAX_OPLOG_SIZE = 500;
-  // diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(true);
-  // diskProps.setOverflow(false);
-  // final int THREAD_COUNT = 5;
-  // final byte[] val = new byte[50];
-  // final byte[] uval = new byte[1];
-  // region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps);
-  // for (int i = 1; i < 101; ++i) {
-  // map.put(new Integer(i), new Integer(i));
-  // }
-  // final AI uniqueCtr = CFactory.createAI();
-  // Thread[] threads = new Thread[THREAD_COUNT];
-  // for (int i = 0; i < THREAD_COUNT; ++i) {
-  // threads[i] = new Thread(new Runnable() {
-  // public void run()
-  // {
-  // int sizeOfOp = 0;
-  // DiskId id = null;
-  // for (int j = 0; j < 50; ++j) {
-  // int keyNum = random.nextInt(10) + 1;
-  // Integer key = new Integer(keyNum);
-  // Integer intgr = (Integer)map.get(key);
-  // try {
-  // String uniqueKey = "UK" + uniqueCtr.incrementAndGet();
-  // // since the files for "empty" oplogs now get cleaned up early
-  // // create a unique key to keep this oplog alive.
-  // region.create(uniqueKey, uval);
-  // DiskEntry uentry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(uniqueKey));
-  // sizeOfOp = OplogJUnitTest.evaluateSizeOfOperationForPersist(uniqueKey, uval,
-  // uentry.getDiskId(), OP_CREATE);
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.expectedOplogSize += sizeOfOp;
-  // ++OplogJUnitTest.this.totalSuccessfulOperations;
-  // ++OplogJUnitTest.this.numCreate;
-  // }
-
-  // synchronized (intgr) {
-
-  // region.create(key, val);
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(key));
-  // id = entry.getDiskId();
-
-  // }
-  // sizeOfOp = OplogJUnitTest.evaluateSizeOfOperationForPersist(key,
-  // val, id, OP_CREATE);
-
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.expectedOplogSize += sizeOfOp;
-  // ++OplogJUnitTest.this.totalSuccessfulOperations;
-  // ++OplogJUnitTest.this.numCreate;
-  // }
-  // }
-  // catch (EntryExistsException eee) {
-  // if (logWriter.finerEnabled()) {
-  // logWriter
-  // .finer("The entry already exists so this operation will not increase the size of oplog");
-  // }
-  // }
-  // try {
-  // boolean isUpdate = false;
-  // synchronized (intgr) {
-  // isUpdate = region.containsKey(key) && region.get(key) != null
-  // && region.get(key) != Token.DESTROYED;
-  // region.put(key, val);
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(key));
-  // id = entry.getDiskId();
-  // }
-  // sizeOfOp = OplogJUnitTest.evaluateSizeOfOperationForPersist(key,
-  // val, id, (isUpdate ? OP_MODIFY : OP_CREATE));
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.expectedOplogSize += sizeOfOp;
-  // ++OplogJUnitTest.this.totalSuccessfulOperations;
-  // if (!isUpdate) {
-  // ++OplogJUnitTest.this.numCreate;
-  // }
-  // else {
-  // ++OplogJUnitTest.this.numModify;
-  // }
-  // }
-  // }
-  // catch (EntryDestroyedException ede) {
-  // if (logWriter.finerEnabled()) {
-  // logWriter
-  // .finer("The entry already exists so this operation will not increase the size of oplog");
-  // }
-  // }
-
-  // boolean deleted = false;
-  // synchronized (intgr) {
-
-  // if (region.containsKey(key) && region.get(key) != null
-  // && region.get(key) != Token.DESTROYED) {
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(key));
-  // id = entry.getDiskId();
-  // region.remove(key);
-  // deleted = true;
-  // }
-
-  // }
-  // if (deleted) {
-  // sizeOfOp = OplogJUnitTest.evaluateSizeOfOperationForPersist(key,
-  // null, id, OP_DEL);
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.expectedOplogSize += sizeOfOp;
-  // ++OplogJUnitTest.this.totalSuccessfulOperations;
-  // ++OplogJUnitTest.this.numDel;
-
-  // }
-  // }
-
-  // }
-
-  // }
-
-  // });
-  // threads[i].start();
-  // }
-
-  // for (int i = 0; i < THREAD_COUNT; ++i) {
-  // DistributedTestCase.join(threads[i], 30 * 1000, null);
-  // }
-
-  // long currentOplogID = ((LocalRegion)region).getDiskRegion().getChild()
-  // .getOplogId();
-  // assertTrue(
-  // " Switching did not happen, increase the iterations to insert more data ",
-  // currentOplogID > 1);
-  // long inMemOplogSize = 0;
-
-  // for (int j = 1; j <= currentOplogID; ++j) {
-
-  // Oplog oplog = ((LocalRegion)region).getDiskRegion().getChild(j);
-  // // if (j < currentOplogID) {
-  // // // oplogs are now closed to save memory and file descriptors
-  // // // once they are no longer needed
-  // // assertIndexDetailsEquals(null, oplog);
-  // // } else {
-  // inMemOplogSize += oplog.getOplogSize();
-  // logWriter.info(" Oplog size="+ oplog.getOplogSize() + " Max Oplog size
-  // acceptable="+MAX_OPLOG_SIZE );
-  // assertTrue(
-  // " The max Oplog Size limit is violated when taken the inmemory oplog size",
-  // oplog.getOplogSize() <= MAX_OPLOG_SIZE);
-
-  // // File opFile = null;
-  // try {
-  // oplog.getOplogFile();
-  // }
-  // catch (Exception e) {
-  // logWriter
-  // .error(
-  // "Exception in synching data present in the buffers of RandomAccessFile of Oplog, to the disk",
-  // e);
-  // fail("Test failed because synching of data present in buffer of RandomAccesFile ");
-  // }
-
-  // assertTrue(
-  // " The max Oplog Size limit is violated when taken the actual file size",
-  // oplog.getActualFileLength() <= MAX_OPLOG_SIZE);
-  // assertIndexDetailsEquals(oplog.getOplogSize(), oplog.getActualFileLength());
-  // // }
-  // }
-
-  // inMemOplogSize +=
-  // ((LocalRegion)region).getDiskRegion().getDiskStore().undeletedOplogSize.get();
-
-  // assertTrue(" The sum of all oplogs size as expected ="
-  // + this.expectedOplogSize + " Actual sizes of all oplogs ="
-  // + inMemOplogSize, this.expectedOplogSize == inMemOplogSize);
-
-  // assertIndexDetailsEquals((this.numCreate + this.numDel + this.numModify),
-  // this.totalSuccessfulOperations);
-  // closeDown();
-
-  // }
-
-  // give the new oplog record format it is too hard for the test to calculate
-  // the expected size
-  // /**
-  // * Tests whether the switching of Oplog happens correctly without size
-  // * violation in case of concurrent region operations for asynch mode.
-  // *
-  // */
-  // @Test
-  // public void testSwitchingForConcurrentASynchedOperations()
-  // {
-  // final int MAX_OPLOG_SIZE = 500;
-  // diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(false);
-  // diskProps.setOverflow(false);
-  // diskProps.setBytesThreshold(100);
-  // final int THREAD_COUNT = 40;
-  // final byte[] val = new byte[50];
-  // region = DiskRegionHelperFactory
-  // .getAsyncPersistOnlyRegion(cache, diskProps);
-
-  // Thread[] threads = new Thread[THREAD_COUNT];
-  // for (int i = 0; i < THREAD_COUNT; ++i) {
-  // final int threadNum = (i + 1);
-  // threads[i] = new Thread(new Runnable() {
-  // public void run()
-  // {
-  // int sizeOfOp = 0;
-  // DiskId id = null;
-  // try {
-  // region.create(new Integer(threadNum), val);
-  // }
-
-  // catch (EntryExistsException e) {
-  // e.printStackTrace();
-  // testFailed = true;
-  // failureCause = "Entry existed with key =" + threadNum;
-  // fail("Entry existed with key =" + threadNum);
-  // }
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(new Integer(threadNum)));
-  // id = entry.getDiskId();
-
-  // sizeOfOp = OplogJUnitTest.evaluateSizeOfOperationForPersist(
-  // new Integer(threadNum), val, id, OP_CREATE);
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.expectedOplogSize += sizeOfOp;
-  // ++OplogJUnitTest.this.totalSuccessfulOperations;
-  // ++OplogJUnitTest.this.numCreate;
-  // }
-
-  // }
-
-  // });
-
-  // threads[i].start();
-  // }
-
-  // for (int i = 0; i < THREAD_COUNT; ++i) {
-  // DistributedTestCase.join(threads[i], 30 * 1000, null);
-  // }
-
-  // long currentOplogID = ((LocalRegion)region).getDiskRegion().getChild()
-  // .getOplogId();
-  // assertTrue(
-  // " Switching did not happen, increase the iterations to insert more data ",
-  // currentOplogID > 1);
-  // if (debug)
-  // System.out.print("Total number of oplogs created = " + currentOplogID);
-  // long inMemOplogSize = 0;
-
-  // for (int j = 1; j <= currentOplogID; ++j) {
-  // Oplog oplog = ((LocalRegion)region).getDiskRegion().getChild(j);
-  // // if (j < currentOplogID) {
-  // // // oplogs are now closed to save memory and file descriptors
-  // // // once they are no longer needed
-  // // assertIndexDetailsEquals(null, oplog);
-  // // } else {
-  // inMemOplogSize += oplog.getOplogSize();
-  // //oplog.forceFlush();
-  // assertTrue(
-  // " The max Oplog Size limit is violated when taken the inmemory oplog size",
-  // oplog.getOplogSize() <= MAX_OPLOG_SIZE);
-  // // File opFile = null;
-  // try {
-  // oplog.getOplogFile();
-  // }
-  // catch (Exception e) {
-  // logWriter
-  // .error(
-  // "Exception in synching data present in the buffers of RandomAccessFile of Oplog, to the disk",
-  // e);
-  // fail("Test failed because synching of data present in buffer of RandomAccesFile ");
-  // }
-  // assertTrue(
-  // " The max Oplog Size limit is violated when taken the actual file size",
-  // oplog.getActualFileLength() <= MAX_OPLOG_SIZE);
-  // assertIndexDetailsEquals(oplog.getOplogSize(), oplog.getActualFileLength());
-  // // }
-  // }
-
-  // inMemOplogSize +=
-  // ((LocalRegion)region).getDiskRegion().getDiskStore().undeletedOplogSize.get();
-
-  // assertTrue(" The sum of all oplogs size as expected ="
-  // + this.expectedOplogSize + " Actual sizes of all oplogs ="
-  // + inMemOplogSize, this.expectedOplogSize == inMemOplogSize);
-  // assertIndexDetailsEquals((this.numCreate + this.numDel + this.numModify),
-  // this.totalSuccessfulOperations);
-  // assertFalse(failureCause, testFailed);
-  // closeDown();
-
-  // }
-
-  // /**
-  // */
-  // @Test
-  // public void testAsyncWriterTerminationAfterSwitch()
-  // {
-  // final int MAX_OPLOG_SIZE = 500;
-  // diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(false);
-  // diskProps.setOverflow(false);
-  // diskProps.setBytesThreshold(100);
-  // final int THREAD_COUNT = 40;
-  // final byte[] val = new byte[50];
-  // region = DiskRegionHelperFactory
-  // .getAsyncPersistOnlyRegion(cache, diskProps);
-
-  // Thread[] threads = new Thread[THREAD_COUNT];
-  // for (int i = 0; i < THREAD_COUNT; ++i) {
-  // final int threadNum = (i + 1);
-  // threads[i] = new Thread(new Runnable() {
-  // public void run()
-  // {
-  // int sizeOfOp = 0;
-  // DiskId id = null;
-  // try {
-  // region.create(new Integer(threadNum), val);
-  // }
-
-  // catch (EntryExistsException e) {
-  // testFailed = true;
-  // failureCause = "Entry existed with key =" + threadNum;
-  // fail("Entry existed with key =" + threadNum);
-  // }
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(new Integer(threadNum)));
-  // id = entry.getDiskId();
-
-  // sizeOfOp = OplogJUnitTest.evaluateSizeOfOperationForPersist(
-  // new Integer(threadNum), val, id, OP_CREATE);
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.expectedOplogSize += sizeOfOp;
-  // ++OplogJUnitTest.this.totalSuccessfulOperations;
-  // ++OplogJUnitTest.this.numCreate;
-  // }
-
-  // }
-
-  // });
-
-  // threads[i].start();
-  // }
-
-  // for (int i = 0; i < THREAD_COUNT; ++i) {
-  // DistributedTestCase.join(threads[i], 30 * 1000, null);
-  // }
-
-  // long currentOplogID = ((LocalRegion)region).getDiskRegion().getChild()
-  // .getOplogId();
-  // assertTrue(
-  // " Switching did not happen, increase the iterations to insert more data ",
-  // currentOplogID > 1);
-
-  // for (int j = 1; j < currentOplogID; ++j) {
-  // Oplog oplog = ((LocalRegion)region).getDiskRegion().getChild(j);
-  // // if (oplog != null) {
-  // // DistributedTestCase.join(oplog.getAsynchWriter().getThread(), 10 * 1000, null);
-  // // }
-  // }
-  // assertFalse(failureCause, testFailed);
-  // closeDown();
-
-  // }
-
-  // /**
-  // */
-  // @Test
-  // public void testMultipleByteBuffersASynchOperations()
-  // {
-  // final int MAX_OPLOG_SIZE = 100000;
-  // diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(false);
-  // diskProps.setOverflow(false);
-  // diskProps.setBytesThreshold(1000);
-  // Oplog.testSetMaxByteBufferSize(100);
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
-  // final int OP_COUNT = 40;
-  // final byte[] val = new byte[50];
-  // region = DiskRegionHelperFactory
-  // .getAsyncPersistOnlyRegion(cache, diskProps);
-  // CacheObserver old = CacheObserverHolder
-  // .setInstance(new CacheObserverAdapter() {
-  // public void afterWritingBytes()
-  // {
-
-  // synchronized (OplogJUnitTest.this) {
-  // flushOccurredAtleastOnce = true;
-  // OplogJUnitTest.this.notify();
-  // }
-
-  // }
-  // });
-
-  // int sizeOfOp = 0;
-  // DiskId id = null;
-  // for (int i = 0; i < OP_COUNT; ++i) {
-  // try {
-  // region.create(new Integer(i), val);
-  // DiskEntry entry = ((DiskEntry)((LocalRegion)region)
-  // .basicGetEntry(new Integer(i)));
-  // id = entry.getDiskId();
-  // sizeOfOp += evaluateSizeOfOperationForPersist(new Integer(i), val, id,
-  // OP_CREATE);
-
-  // }
-
-  // catch (EntryExistsException e) {
-  // fail("Entry existed with key =" + i);
-  // }
-  // }
-  // Oplog currOplog = ((LocalRegion)region).getDiskRegion().getChild();
-  // long currentOplogID = currOplog.getOplogId();
-  // long expectedSize = currOplog.getOplogSize();
-  // // Ensure that now switching has happned during the operations
-  // assertIndexDetailsEquals(1, currentOplogID);
-  // assertTrue(
-  // "The number of operations did not cause asynch writer to run atleast once , the expected file
-  // size = "
-  // + expectedSize, expectedSize > 1000);
-  // if (!flushOccurredAtleastOnce) {
-  // synchronized (this) {
-  // if (!flushOccurredAtleastOnce) {
-  // try {
-  // this.wait(20000);
-  // }
-  // catch (InterruptedException e) {
-  // fail("No guarantee as flushed occure deven once.Exception=" + e);
-  // }
-  // }
-  // }
-  // }
-  // if (!flushOccurredAtleastOnce) {
-  // fail("In the wait duration , flush did not occur even once. Try increasing the wait time");
-  // }
-  // long actualFileSize = 0L;
-
-  // try {
-  // actualFileSize = currOplog.getFileChannel().position();
-  // }
-  // catch (IOException e) {
-  // fail(e.toString());
-  // }
-
-  // assertTrue(
-  // "The number of operations did not cause asynch writer to run atleast once as the actual file
-  // size = "
-  // + actualFileSize, actualFileSize >= 1000);
-  // //currOplog.forceFlush();
-  // // File opFile = null;
-  // try {
-  // currOplog.getOplogFile();
-  // }
-  // catch (Exception e) {
-  // logWriter
-  // .error(
-  // "Exception in synching data present in the buffers of RandomAccessFile of Oplog, to the disk",
-  // e);
-  // fail("Test failed because synching of data present in buffer of RandomAccesFile ");
-  // }
-  // actualFileSize = currOplog.getActualFileLength();
-  // assertTrue(
-  // " The expected Oplog Size not equal to the actual file size. Expected size="
-  // + expectedSize + " actual size = " + actualFileSize,
-  // expectedSize == actualFileSize);
-  // Oplog.testSetMaxByteBufferSize(Integer.MAX_VALUE);
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
-  // CacheObserverHolder.setInstance(old);
-  // closeDown();
-
-  // }
-
   /**
    * Tests the bug which arises in case of asynch mode during oplog switching caused by conflation
    * of create/destroy operation.The bug occurs if a create operation is followed by destroy but
@@ -1599,11 +442,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     final CacheObserver old = CacheObserverHolder.setInstance(new CacheObserverAdapter() {
       @Override
       public void afterConflation(ByteBuffer orig, ByteBuffer conflated) {
-        Thread th = new Thread(new Runnable() {
-          public void run() {
-            region.put("2", new byte[75]);
-          }
-        });
+        Thread th = new Thread(() -> region.put("2", new byte[75]));
         assertNull(conflated);
         th.start();
         ThreadUtils.join(th, 30 * 1000);
@@ -1695,10 +534,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       Object val_11 = ((LocalRegion) region).getValueOnDisk("11");
       assertEquals(val_11, Token.LOCAL_INVALID);
 
-    } catch (Exception e) {
-      logWriter.error("Exception occurred", e);
-      // fail("The test failed due to exception = " + e);
-      throw e;
     } finally {
       closeDown();
     }
@@ -1907,7 +742,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       final int MAX_OPLOG_SIZE = 2000;
       diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
       diskProps.setPersistBackup(true);
-      // diskProps.setRolling(true);
       diskProps.setSynchronous(true);
       diskProps.setOverflow(false);
       diskProps.setDiskDirsAndSizes(new File[] {dirs[0]}, new int[] {1400});
@@ -1969,11 +803,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       region.put("key1", val);
       region.put("key2", val);
       region.put("key3", val);
-      final Thread th = new Thread(new Runnable() {
-        public void run() {
-          region.remove("key1");
-        }
-      });
+      final Thread th = new Thread(() -> region.remove("key1"));
       // main thread acquires the write lock
       ((LocalRegion) region).getDiskRegion().acquireWriteLock();
       try {
@@ -1997,146 +827,12 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
   }
 
   /**
-   * Tests the various configurable parameters used by the ByteBufferPool . The behaviour of
-   * parameters is based on the mode of DiskRegion ( synch or asynch) . Pls refer to the class
-   * documentation ( Oplog.ByteBufferPool) for the exact behaviour of the class
-   *
-   */
-  // @Test
-  // public void testByteBufferPoolParameters()
-  // {
-  // // If the mode is asynch , the ByteBuffer obtained should e non direct else
-  // // direct
-  // final int MAX_OPLOG_SIZE = 500;
-  // diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(true);
-  // diskProps.setOverflow(false);
-  // region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps);
-  // List bbPools = ((LocalRegion)region).getDiskRegion().getChild()
-  // .getByteBufferPoolList();
-  // ByteBuffer bb = ((Oplog.ByteBufferPool)bbPools.get(1)).getBufferFromPool();
-  // assertTrue(" ByteBuffer is not of type direct", bb.isDirect());
-  // region.destroyRegion();
-  // diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(false);
-  // diskProps.setOverflow(false);
-  // region = DiskRegionHelperFactory
-  // .getAsyncPersistOnlyRegion(cache, diskProps);
-  // bbPools = ((LocalRegion)region).getDiskRegion().getChild()
-  // .getByteBufferPoolList();
-  // bb = ((Oplog.ByteBufferPool)bbPools.get(1)).getBufferFromPool();
-  // assertTrue(" ByteBuffer is not of type direct", bb.isDirect());
-  // region.close();
-  // // Test max pool limit & wait time ( valid only in synch mode).
-  // diskProps.setSynchronous(true);
-  // diskProps.setRegionName("testRegion");
-  // System.setProperty("/testRegion_MAX_POOL_SIZE", "1");
-
-  // System.setProperty("/testRegion_WAIT_TIME", "4000");
-  // region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps);
-  // bbPools = ((LocalRegion)region).getDiskRegion().getChild()
-  // .getByteBufferPoolList();
-  // bb = ((Oplog.ByteBufferPool)bbPools.get(1)).getBufferFromPool();
-  // assertTrue("Since the Pool has one Entry , it should be direct", bb
-  // .isDirect());
-
-  // long t1 = System.currentTimeMillis();
-  // bb = ((Oplog.ByteBufferPool)bbPools.get(1)).getBufferFromPool();
-  // long t2 = System.currentTimeMillis();
-  // assertTrue(
-  // "Since the Pool should have been exhausted hence non direct byte buffer should have been
-  // returned",
-  // !bb.isDirect());
-  // assertTrue("The wait time for ByteBuffer pool was not respected ",
-  // (t2 - t1) > 3000);
-  // region.close();
-  // // // In case of asynch mode , the upper limit should not have been imposed
-  // // System.setProperty("/testRegion_MAX_POOL_SIZE", "1");
-  // // System.setProperty("/testRegion_WAIT_TIME", "5000");
-  // // diskProps.setSynchronous(false);
-  // // diskProps.setRegionName("testRegion");
-  // // region = DiskRegionHelperFactory
-  // // .getAsyncPersistOnlyRegion(cache, diskProps);
-  // // bbPools = ((LocalRegion)region).getDiskRegion().getChild()
-  // // .getByteBufferPoolList();
-  // // bb = ((Oplog.ByteBufferPool)bbPools.get(1)).getBufferFromPool();
-  // // t1 = System.currentTimeMillis();
-  // // bb = ((Oplog.ByteBufferPool)bbPools.get(1)).getBufferFromPool();
-  // // t2 = System.currentTimeMillis();
-  // // assertTrue(
-  // // "There should not have been any wait time " + (t2-t1) + " for ByteBuffer pool ",
-  // // (t2 - t1) / 1000 < 3);
-  // // region.close();
-  // System.setProperty("/testRegion_MAX_POOL_SIZE", "2");
-  // System.setProperty("/testRegion_WAIT_TIME", "5000");
-  // diskProps.setSynchronous(true);
-  // diskProps.setRegionName("testRegion");
-  // region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps);
-  // bbPools = ((LocalRegion)region).getDiskRegion().getChild()
-  // .getByteBufferPoolList();
-  // Oplog.ByteBufferPool pool = (Oplog.ByteBufferPool)bbPools.get(1);
-  // ByteBuffer bb1 = pool.getBufferFromPool();
-  // ByteBuffer bb2 = pool.getBufferFromPool();
-  // assertIndexDetailsEquals(2, pool.getTotalBuffers());
-  // assertIndexDetailsEquals(2, pool.getBuffersInUse());
-  // ((LocalRegion)region).getDiskRegion().getChild().releaseBuffer(bb1);
-  // ((LocalRegion)region).getDiskRegion().getChild().releaseBuffer(bb2);
-  // assertIndexDetailsEquals(0, pool.getBuffersInUse());
-  // region.close();
-
-  // System.setProperty("/testRegion_MAX_POOL_SIZE", "1");
-  // System.setProperty("/testRegion_WAIT_TIME", "1000");
-  // diskProps.setSynchronous(true);
-  // diskProps.setRegionName("testRegion");
-  // region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps);
-  // bbPools = ((LocalRegion)region).getDiskRegion().getChild()
-  // .getByteBufferPoolList();
-  // pool = (Oplog.ByteBufferPool)bbPools.get(1);
-  // bb1 = pool.getBufferFromPool();
-  // bb2 = pool.getBufferFromPool();
-  // assertIndexDetailsEquals(1, pool.getTotalBuffers());
-  // assertIndexDetailsEquals(1, pool.getBuffersInUse());
-  // ((LocalRegion)region).getDiskRegion().getChild().releaseBuffer(bb1);
-  // ((LocalRegion)region).getDiskRegion().getChild().releaseBuffer(bb2);
-  // assertIndexDetailsEquals(0, pool.getBuffersInUse());
-  // closeDown();
-
-  // }
-
-  /**
-   * Tests the ByteBuffer Pool operations for release of ByteBuffers in case the objects being put
-   * vary in size & hence use ByteBuffer Pools present at different indexes
-   *
-   */
-  // @Test
-  // public void testByteBufferPoolReleaseBugTest()
-  // {
-
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(true);
-  // diskProps.setOverflow(false);
-  // System.setProperty("/testRegion_UNIT_BUFF_SIZE", "100");
-  // System.setProperty("/testRegion_UNIT_BUFF_SIZE", "100");
-  // System.setProperty("gemfire.log-level", getGemFireLogLevel());
-  // region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps);
-  // region.put("key1", new byte[900]);
-  // region.put("key1", new byte[700]);
-  // closeDown();
-
-  // }
-
-  /**
    * Tests if buffer size & time are not set , the asynch writer gets awakened on time basis of
    * default 1 second
    *
    */
   @Test
-  public void testAsynchWriterAttribBehaviour1() {
+  public void testAsynchWriterAttribBehaviour1() throws Exception {
     DiskStoreFactory dsf = cache.createDiskStoreFactory();
     ((DiskStoreFactoryImpl) dsf).setMaxOplogSizeInBytes(10000);
     File dir = new File("testingDirectoryDefault");
@@ -2144,20 +840,14 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     dir.deleteOnExit();
     File[] dirs = {dir};
     dsf.setDiskDirs(dirs);
-    AttributesFactory factory = new AttributesFactory();
+    RegionFactory<Object, Object> factory =
+        cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT);
     final long t1 = System.currentTimeMillis();
     DiskStore ds = dsf.create("test");
     factory.setDiskSynchronous(false);
     factory.setDiskStoreName(ds.getName());
-    factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
     factory.setScope(Scope.LOCAL);
-    try {
-      region = cache.createVMRegion("test", factory.createRegionAttributes());
-    } catch (Exception e1) {
-      logWriter.error("Test failed due to exception", e1);
-      fail("Test failed due to exception " + e1);
-
-    }
+    region = factory.create("test");
 
     LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
     CacheObserver old = CacheObserverHolder.setInstance(
@@ -2179,13 +869,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     region.put("key1", "111111111111");
     synchronized (this) {
       if (LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER) {
-        try {
-          this.wait(10000);
-        } catch (InterruptedException e) {
-          logWriter.error("Test failed due to exception", e);
-          fail("Test failed due to exception " + e);
-
-        }
+        this.wait(10000);
         assertFalse(LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER);
       }
     }
@@ -2201,7 +885,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    */
   @Ignore("TODO:DARREL_DISABLE: test is disabled")
   @Test
-  public void testAsynchWriterAttribBehaviour2() {
+  public void testAsynchWriterAttribBehaviour2() throws Exception {
     DiskStoreFactory dsf = cache.createDiskStoreFactory();
     ((DiskStoreFactoryImpl) dsf).setMaxOplogSizeInBytes(10000);
     dsf.setQueueSize(2);
@@ -2210,19 +894,13 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     dir.deleteOnExit();
     File[] dirs = {dir};
     dsf.setDiskDirs(dirs);
-    AttributesFactory factory = new AttributesFactory();
+    RegionFactory<Object, Object> factory =
+        cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT);
     DiskStore ds = dsf.create("test");
     factory.setDiskSynchronous(false);
     factory.setDiskStoreName(ds.getName());
-    factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
     factory.setScope(Scope.LOCAL);
-    try {
-      region = cache.createVMRegion("test", factory.createRegionAttributes());
-    } catch (Exception e1) {
-      logWriter.error("Test failed due to exception", e1);
-      fail("Test failed due to exception " + e1);
-
-    }
+    region = factory.create("test");
 
     LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
     CacheObserver old = CacheObserverHolder.setInstance(
@@ -2239,23 +917,13 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
         });
 
     region.put("key1", new byte[25]);
-    try {
-      Thread.sleep(1000);
-    } catch (InterruptedException e) {
-      logWriter.error("Test failed due to exception", e);
-      fail("Test failed due to exception " + e);
-    }
+    Thread.sleep(1000);
     assertTrue(LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER);
     region.put("key2", new byte[25]);
     synchronized (this) {
       if (LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER) {
-        try {
-          OplogJUnitTest.this.wait(10000);
-          assertFalse(LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER);
-        } catch (InterruptedException e2) {
-          logWriter.error("Test failed due to exception", e2);
-          fail("Test failed due to exception " + e2);
-        }
+        OplogJUnitTest.this.wait(10000);
+        assertFalse(LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER);
       }
     }
     CacheObserverHolder.setInstance(old);
@@ -2268,7 +936,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    *
    */
   @Test
-  public void testAsynchWriterAttribBehaviour3() {
+  public void testAsynchWriterAttribBehaviour3() throws Exception {
     DiskStoreFactory dsf = cache.createDiskStoreFactory();
     ((DiskStoreFactoryImpl) dsf).setMaxOplogSizeInBytes(500);
     dsf.setQueueSize(0);
@@ -2278,19 +946,13 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     dir.deleteOnExit();
     File[] dirs = {dir};
     dsf.setDiskDirs(dirs);
-    AttributesFactory factory = new AttributesFactory();
+    RegionFactory<Object, Object> factory =
+        cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT);
     DiskStore ds = dsf.create("test");
     factory.setDiskSynchronous(false);
     factory.setDiskStoreName(ds.getName());
-    factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
     factory.setScope(Scope.LOCAL);
-    try {
-      region = cache.createVMRegion("test", factory.createRegionAttributes());
-    } catch (Exception e1) {
-      logWriter.error("Test failed due to exception", e1);
-      fail("Test failed due to exception " + e1);
-
-    }
+    region = factory.create("test");
 
     LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
     CacheObserver old = CacheObserverHolder.setInstance(
@@ -2305,27 +967,19 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
             }
           }
         });
-    try {
-      region.put("key1", new byte[100]);
-      region.put("key2", new byte[100]);
-      region.put("key3", new byte[100]);
-      region.put("key4", new byte[100]);
-      region.put("key5", new byte[100]);
-      Thread.sleep(1000);
-      assertTrue(LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER);
-    } catch (Exception e) {
-      logWriter.error("Test failed due to exception", e);
-      fail("Test failed due to exception " + e);
-    }
-    region.forceRolling();
+    region.put("key1", new byte[100]);
+    region.put("key2", new byte[100]);
+    region.put("key3", new byte[100]);
+    region.put("key4", new byte[100]);
+    region.put("key5", new byte[100]);
+    Thread.sleep(1000);
+    assertTrue(LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER);
+
+
+    ((LocalRegion) region).getDiskRegion().forceRolling();
     synchronized (this) {
       if (LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER) {
-        try {
-          OplogJUnitTest.this.wait(10000);
-        } catch (InterruptedException e2) {
-          logWriter.error("Test failed due to exception", e2);
-          fail("Test failed due to exception " + e2);
-        }
+        OplogJUnitTest.this.wait(10000);
       }
     }
     assertFalse(LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER);
@@ -2334,62 +988,11 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
   }
 
   /**
-   * Tests if the preblowing of a file with size greater than the disk space available so that
-   * preblowing results in IOException , is able to recover without problem
-   *
-   */
-  // Now we preallocate spaces for if files and also crfs and drfs. So the below test is not valid
-  // any more. See revision: r42359 and r42320. So disabling this test.
-  @Ignore("TODO: test is disabled")
-  @Test
-  public void testPreblowErrorCondition() {
-    DiskStoreFactory dsf = cache.createDiskStoreFactory();
-    ((DiskStoreFactoryImpl) dsf).setMaxOplogSizeInBytes(100000000L * 1024L * 1024L * 1024L);
-    dsf.setAutoCompact(false);
-    File dir = new File("testingDirectoryDefault");
-    dir.mkdir();
-    dir.deleteOnExit();
-    File[] dirs = {dir};
-    int size[] = new int[] {Integer.MAX_VALUE};
-    dsf.setDiskDirsAndSizes(dirs, size);
-    AttributesFactory factory = new AttributesFactory();
-    logWriter.info("<ExpectedException action=add>" + "Could not pregrow" + "</ExpectedException>");
-    try {
-      DiskStore ds = dsf.create("test");
-      factory.setDiskStoreName(ds.getName());
-      factory.setDiskSynchronous(true);
-      factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
-      factory.setScope(Scope.LOCAL);
-      try {
-        region = cache.createVMRegion("test", factory.createRegionAttributes());
-      } catch (Exception e1) {
-        logWriter.error("Test failed due to exception", e1);
-        fail("Test failed due to exception " + e1);
-
-      }
-      region.put("key1", new byte[900]);
-      byte[] val = null;
-      try {
-        val = (byte[]) ((LocalRegion) region).getValueOnDisk("key1");
-      } catch (Exception e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
-        fail(e.toString());
-      }
-      assertTrue(val.length == 900);
-    } finally {
-      logWriter
-          .info("<ExpectedException action=remove>" + "Could not pregrow" + "</ExpectedException>");
-    }
-    closeDown();
-  }
-
-  /**
    * Tests if the byte buffer pool in asynch mode tries to contain the pool size
    *
    */
   @Test
-  public void testByteBufferPoolContainment() {
+  public void testByteBufferPoolContainment() throws Exception {
 
     diskProps.setPersistBackup(true);
     diskProps.setRolling(false);
@@ -2421,105 +1024,14 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     for (int i = 0; i < 10; ++i) {
       region.put("" + i, val);
     }
-    try {
-      synchronized (OplogJUnitTest.this) {
-        if (LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER) {
-          OplogJUnitTest.this.wait(9000);
-          assertEquals(false, LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER);
-        }
+    synchronized (OplogJUnitTest.this) {
+      if (LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER) {
+        OplogJUnitTest.this.wait(9000);
+        assertEquals(false, LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER);
       }
-
-    } catch (InterruptedException ie) {
-      fail("interrupted");
     }
-    // ((LocalRegion)region).getDiskRegion().getChild().forceFlush();
-    // int x = ((LocalRegion)region).getDiskRegion().getChild().getAsynchWriter()
-    // .getApproxFreeBuffers();
-    // assertIndexDetailsEquals(10, x);
   }
 
-  // we no longer have a pendingFlushMap
-  // /**
-  // * This test does the following: <br>
-  // * 1)Create a diskRegion with async mode and byte-threshold as 25 bytes. <br>
-  // * 2)Put an entry into the region such that the async-buffer is just over 25
-  // * bytes and the writer-thread is invoked. <br>
-  // * 3)Using CacheObserver.afterSwitchingWriteAndFlushMaps callback, perform a
-  // * put on the same key just after the async writer thread swaps the
-  // * pendingFlushMap and pendingWriteMap for flushing. <br>
-  // * 4)Using CacheObserver.afterWritingBytes, read the value for key
-  // * (LocalRegion.getValueOnDiskOrBuffer) just after the async writer thread has
-  // * flushed to the disk. <br>
-  // * 5) Verify that the value read in step3 is same as the latest value. This
-  // * will ensure that the flushBufferToggle flag is functioning as expected ( It
-  // * prevents the writer thread from setting the oplog-offset in diskId if that
-  // * particular entry has been updated by a put-thread while the
-  // * async-writer-thread is flushing that entry.)
-  // *
-  // * @throws Exception
-  // */
-  // @Test
-  // public void testFlushBufferToggleFlag() throws Exception
-  // {
-  // final int MAX_OPLOG_SIZE = 100000;
-  // diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(false);
-  // diskProps.setSynchronous(false);
-  // diskProps.setOverflow(false);
-  // diskProps.setBytesThreshold(25);
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
-
-  // region = DiskRegionHelperFactory
-  // .getAsyncPersistOnlyRegion(cache, diskProps);
-  // CacheObserver old = CacheObserverHolder
-  // .setInstance(new CacheObserverAdapter() {
-  // public void afterWritingBytes()
-  // {
-  // LocalRegion localregion = (LocalRegion)region;
-  // try {
-  // valueRead = (String)localregion.getValueOnDiskOrBuffer(KEY);
-  // synchronized (OplogJUnitTest.class) {
-  // proceedForValidation = true;
-  // OplogJUnitTest.class.notify();
-  // }
-  // }
-  // catch (EntryNotFoundException e) {
-  // e.printStackTrace();
-  // }
-  // }
-
-  // public void afterSwitchingWriteAndFlushMaps()
-  // {
-  // region.put(KEY, NEW_VALUE);
-  // }
-
-  // });
-
-  // region.put(KEY, OLD_VALUE);
-
-  // if (!proceedForValidation) {
-  // synchronized (OplogJUnitTest.class) {
-  // if (!proceedForValidation) {
-  // try {
-  // OplogJUnitTest.class.wait(9000);
-  // assertIndexDetailsEquals(true, proceedForValidation);
-  // }
-  // catch (InterruptedException e) {
-  // fail("interrupted");
-  // }
-  // }
-  // }
-  // }
-
-  // cache.getLogger().info("valueRead : " + valueRead);
-  // assertIndexDetailsEquals("valueRead is stale, doesnt match with latest PUT", NEW_VALUE,
-  // valueRead);
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
-  // CacheObserverHolder.setInstance(old);
-  // closeDown();
-  // }
-
   /**
    * tests async stats are correctly updated
    */
@@ -2536,7 +1048,9 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
         .timeout(10, TimeUnit.SECONDS).until(() -> assertEquals(100, dss.getQueueSize()));
 
     assertEquals(0, dss.getFlushes());
-    region.writeToDisk();
+
+    DiskRegion diskRegion = ((LocalRegion) region).getDiskRegion();
+    diskRegion.getDiskStore().flush();
     Awaitility.await().pollInterval(10, TimeUnit.MILLISECONDS).pollDelay(10, TimeUnit.MILLISECONDS)
         .timeout(10, TimeUnit.SECONDS).until(() -> assertEquals(0, dss.getQueueSize()));
     Awaitility.await().pollInterval(10, TimeUnit.MILLISECONDS).pollDelay(10, TimeUnit.MILLISECONDS)
@@ -2544,7 +1058,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     put100Int();
     Awaitility.await().pollInterval(10, TimeUnit.MILLISECONDS).pollDelay(10, TimeUnit.MILLISECONDS)
         .timeout(10, TimeUnit.SECONDS).until(() -> assertEquals(100, dss.getQueueSize()));
-    region.writeToDisk();
+    diskRegion.getDiskStore().flush();
     Awaitility.await().pollInterval(10, TimeUnit.MILLISECONDS).pollDelay(10, TimeUnit.MILLISECONDS)
         .timeout(10, TimeUnit.SECONDS).until(() -> assertEquals(0, dss.getQueueSize()));
     Awaitility.await().pollInterval(10, TimeUnit.MILLISECONDS).pollDelay(10, TimeUnit.MILLISECONDS)
@@ -2612,63 +1126,59 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    */
   @Test
   public void testEntryAlreadyWrittenIsCorrectlyUnmarkedForOverflowOnly() throws Exception {
-    try {
-      diskProps.setPersistBackup(false);
-      diskProps.setRolling(false);
-      diskProps.setMaxOplogSize(1024 * 1024);
-      diskProps.setSynchronous(true);
-      diskProps.setOverflow(true);
-      diskProps.setBytesThreshold(10000);
-      diskProps.setTimeInterval(0);
-      diskProps.setOverFlowCapacity(1);
-      region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
-      final byte[] val = new byte[1000];
-      region.put("1", val);
-      region.put("2", val);
-      // "1" should now be on disk
-      region.get("1");
-      // "2" should now be on disk
-      DiskEntry entry1 = (DiskEntry) ((LocalRegion) region).basicGetEntry("1");
-      DiskId did1 = entry1.getDiskId();
-      DiskId.isInstanceofOverflowIntOplogOffsetDiskId(did1);
-      assertTrue(!did1.needsToBeWritten());
-      region.put("1", "3");
-      assertTrue(did1.needsToBeWritten());
-      region.put("2", val);
-      DiskEntry entry2 = (DiskEntry) ((LocalRegion) region).basicGetEntry("2");
-      DiskId did2 = entry2.getDiskId();
-      assertTrue(!did2.needsToBeWritten() || !did1.needsToBeWritten());
-      tearDown();
-      setUp();
-      diskProps.setPersistBackup(false);
-      diskProps.setRolling(false);
-      long opsize = Integer.MAX_VALUE;
-      opsize += 100L;
-      diskProps.setMaxOplogSize(opsize);
-      diskProps.setSynchronous(true);
-      diskProps.setOverflow(true);
-      diskProps.setBytesThreshold(10000);
-      diskProps.setTimeInterval(0);
-      diskProps.setOverFlowCapacity(1);
-      region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
-      region.put("1", val);
-      region.put("2", val);
-      region.get("1");
-      entry1 = (DiskEntry) ((LocalRegion) region).basicGetEntry("1");
-      did1 = entry1.getDiskId();
-      DiskId.isInstanceofOverflowOnlyWithLongOffset(did1);
-      assertTrue(!did1.needsToBeWritten());
-      region.put("1", "3");
-      assertTrue(did1.needsToBeWritten());
-      region.put("2", "3");
-      did2 = entry2.getDiskId();
-      assertTrue(!did2.needsToBeWritten() || !did1.needsToBeWritten());
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.toString());
-    }
+    diskProps.setPersistBackup(false);
+    diskProps.setRolling(false);
+    diskProps.setMaxOplogSize(1024 * 1024);
+    diskProps.setSynchronous(true);
+    diskProps.setOverflow(true);
+    diskProps.setBytesThreshold(10000);
+    diskProps.setTimeInterval(0);
+    diskProps.setOverFlowCapacity(1);
+    region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
+    final byte[] val = new byte[1000];
+    region.put("1", val);
+    region.put("2", val);
+    // "1" should now be on disk
+    region.get("1");
+    // "2" should now be on disk
+    DiskEntry entry1 = (DiskEntry) ((LocalRegion) region).basicGetEntry("1");
+    DiskId did1 = entry1.getDiskId();
+    DiskId.isInstanceofOverflowIntOplogOffsetDiskId(did1);
+    assertTrue(!did1.needsToBeWritten());
+    region.put("1", "3");
+    assertTrue(did1.needsToBeWritten());
+    region.put("2", val);
+    DiskEntry entry2 = (DiskEntry) ((LocalRegion) region).basicGetEntry("2");
+    DiskId did2 = entry2.getDiskId();
+    assertTrue(!did2.needsToBeWritten() || !did1.needsToBeWritten());
+    tearDown();
+    setUp();
+    diskProps.setPersistBackup(false);
+    diskProps.setRolling(false);
+    long opsize = Integer.MAX_VALUE;
+    opsize += 100L;
+    diskProps.setMaxOplogSize(opsize);
+    diskProps.setSynchronous(true);
+    diskProps.setOverflow(true);
+    diskProps.setBytesThreshold(10000);
+    diskProps.setTimeInterval(0);
+    diskProps.setOverFlowCapacity(1);
+    region = DiskRegionHelperFactory.getSyncOverFlowOnlyRegion(cache, diskProps);
+    region.put("1", val);
+    region.put("2", val);
+    region.get("1");
+    entry1 = (DiskEntry) ((LocalRegion) region).basicGetEntry("1");
+    did1 = entry1.getDiskId();
+    DiskId.isInstanceofOverflowOnlyWithLongOffset(did1);
+    assertTrue(!did1.needsToBeWritten());
+    region.put("1", "3");
+    assertTrue(did1.needsToBeWritten());
+    region.put("2", "3");
+    did2 = entry2.getDiskId();
+    assertTrue(!did2.needsToBeWritten() || !did1.needsToBeWritten());
   }
 
+
   /**
    * An persistent or overflow with persistence entry which is evicted to disk, will have the flag
    * already written to disk, appropriately set
@@ -2713,17 +1223,11 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     final byte[] val = new byte[1000];
     DiskRegion dr = ((LocalRegion) region).getDiskRegion();
     region.put("1", val);
-    // region.get("1");
     region.put("2", val);
-    // region.get("2");
     region.put("3", val);
-    // region.get("3");
     DiskEntry entry1 = (DiskEntry) ((LocalRegion) region).basicGetEntry("1");
-    // DiskId did1 = entry1.getDiskId();
     DiskEntry entry2 = (DiskEntry) ((LocalRegion) region).basicGetEntry("2");
-    // DiskId did2 = entry2.getDiskId();
     DiskEntry entry3 = (DiskEntry) ((LocalRegion) region).basicGetEntry("3");
-    // DiskId did3 = entry3.getDiskId();
     assertNull(entry2.getDiskId());
     assertNull(entry3.getDiskId());
     assertNotNull(entry1.getDiskId());
@@ -2778,17 +1282,11 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     final byte[] val = new byte[1000];
     DiskRegion dr = ((LocalRegion) region).getDiskRegion();
     region.put("1", val);
-    // region.get("1");
     region.put("2", val);
-    // region.get("2");
     region.put("3", val);
-    // region.get("3");
     DiskEntry entry1 = (DiskEntry) ((LocalRegion) region).basicGetEntry("1");
-    // DiskId did1 = entry1.getDiskId();
     DiskEntry entry2 = (DiskEntry) ((LocalRegion) region).basicGetEntry("2");
-    // DiskId did2 = entry2.getDiskId();
     DiskEntry entry3 = (DiskEntry) ((LocalRegion) region).basicGetEntry("3");
-    // DiskId did3 = entry3.getDiskId();
     assertNotNull(entry2.getDiskId());
     assertNotNull(entry3.getDiskId());
     assertNotNull(entry1.getDiskId());
@@ -2804,11 +1302,8 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     region = DiskRegionHelperFactory.getSyncOverFlowAndPersistRegion(cache, diskProps);
     dr = ((LocalRegion) region).getDiskRegion();
     entry1 = (DiskEntry) ((LocalRegion) region).basicGetEntry("1");
-    // did1 = entry1.getDiskId();
     entry2 = (DiskEntry) ((LocalRegion) region).basicGetEntry("2");
-    // did2 = entry2.getDiskId();
     entry3 = (DiskEntry) ((LocalRegion) region).basicGetEntry("3");
-    // did3 = entry3.getDiskId();
 
     assertNotNull(entry2.getDiskId());
     assertNotNull(entry3.getDiskId());
@@ -2821,112 +1316,8 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     assertNotNull(DiskEntry.Helper.getValueOnDiskOrBuffer(entry3, dr, (LocalRegion) region));
     assertNotNull(DiskEntry.Helper.getValueOnDiskOrBuffer(entry2, dr, (LocalRegion) region));
     assertNotNull(DiskEntry.Helper.getValueOnDiskOrBuffer(entry1, dr, (LocalRegion) region));
-
   }
 
-  // @todo this test is failing for some reason. Does it need to be fixed?
-  /**
-   * Bug test to reproduce the bug 37261. The scenario which this test depicts is not actually the
-   * cause of Bug 37261. This test validates the case where a synch persist only entry1 is created
-   * in Oplog1. A put operation on entry2 causes the switch , but before Oplog1 is rolled , the
-   * entry1 is modified so that it references Oplog2. Thus in effect roller will skip rolling entry1
-   * when rolling Oplog1.Now entry1 is deleted in Oplog2 and then a rolling happens. There should
-   * not be any error
-   */
-  // @Test
-  // public void testBug37261_1()
-  // {
-  // CacheObserver old = CacheObserverHolder.getInstance();
-  // try {
-  // // Create a persist only region with rolling true
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(true);
-  // diskProps.setCompactionThreshold(100);
-  // diskProps.setMaxOplogSize(1024);
-  // diskProps.setSynchronous(true);
-  // this.proceed = false;
-  // region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache,
-  // diskProps);
-  // // create an entry 1 in oplog1,
-  // region.put("key1", new byte[800]);
-
-  // // Asif the second put will cause a switch to oplog 2 & also cause the
-  // // oplog1
-  // // to be submitted to the roller
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
-  // CacheObserverHolder.setInstance(new CacheObserverAdapter() {
-  // public void beforeGoingToCompact()
-  // {
-  // // modify entry 1 so that it points to the new switched oplog
-  // Thread th = new Thread(new Runnable() {
-  // public void run()
-  // {
-  // region.put("key1", new byte[400]);
-  // }
-  // });
-  // th.start();
-  // try {
-  // DistributedTestCase.join(th, 30 * 1000, null);
-  // }
-  // catch (Exception e) {
-  // e.printStackTrace();
-  // failureCause = e.toString();
-  // failure = true;
-  // }
-  // }
-
-  // public void afterHavingCompacted()
-  // {
-  // synchronized (OplogJUnitTest.this) {
-  // rollerThread = Thread.currentThread();
-  // OplogJUnitTest.this.notify();
-  // OplogJUnitTest.this.proceed = true;
-  // }
-  // }
-  // });
-  // region.put("key2", new byte[300]);
-  // synchronized (this) {
-  // if (!this.proceed) {
-  // this.wait(15000);
-  // assertTrue(this.proceed);
-  // }
-  // }
-  // this.proceed = false;
-  // // Asif Delete the 1st entry
-  // region.destroy("key1");
-
-  // CacheObserverHolder.setInstance(new CacheObserverAdapter() {
-  // public void afterHavingCompacted()
-  // {
-  // synchronized (OplogJUnitTest.this) {
-  // OplogJUnitTest.this.notify();
-  // OplogJUnitTest.this.proceed = true;
-  // }
-  // }
-  // });
-  // // Coz another switch and wait till rolling done
-  // region.put("key2", new byte[900]);
-
-  // synchronized (this) {
-  // if (!this.proceed) {
-  // this.wait(15000);
-  // assertFalse(this.proceed);
-  // }
-  // }
-  // // Check if the roller is stil alive
-  // assertTrue(rollerThread.isAlive());
-  // }
-  // catch (Exception e) {
-  // e.printStackTrace();
-  // fail("Test failed du toe xception" + e);
-  // }
-  // finally {
-  // CacheObserverHolder.setInstance(old);
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
-  // }
-
-  // }
-
   /**
    * Tests the condition when a 'put' is in progress and concurrent 'clear' and 'put'(on the same
    * key) occur. Thus if after Htree ref was set (in 'put'), the region got cleared (and same key
@@ -2935,7 +1326,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    * This put should not proceed. Also, Region creation after closing should not give an exception.
    */
   @Test
-  public void testPutClearPut() {
+  public void testPutClearPut() throws Exception {
     try {
       // Create a persist only region with rolling true
       diskProps.setPersistBackup(true);
@@ -2944,16 +1335,14 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       diskProps.setSynchronous(true);
       this.proceed = false;
       region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
-      final Thread clearOp = new Thread(new Runnable() {
-        public void run() {
-          try {
-            LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
-            region.clear();
-            region.put("key1", "value3");
-          } catch (Exception e) {
-            testFailed = true;
-            failureCause = "Encountered Exception=" + e;
-          }
+      final Thread clearOp = new Thread(() -> {
+        try {
+          LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
+          region.clear();
+          region.put("key1", "value3");
+        } catch (Exception e) {
+          testFailed = true;
+          failureCause = "Encountered Exception=" + e;
         }
       });
       region.getAttributesMutator().setCacheWriter(new CacheWriterAdapter() {
@@ -2978,9 +1367,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       } else {
         fail(failureCause);
       }
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail("Test failed due to exception" + e);
     } finally {
       testFailed = false;
       proceed = false;
@@ -2997,7 +1383,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
    *
    */
   @Test
-  public void testPutClearCreate() {
+  public void testPutClearCreate() throws Exception {
     failure = false;
     try {
       // Create a persist only region with rolling true
@@ -3013,11 +1399,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       CacheObserverHolder.setInstance(new CacheObserverAdapter() {
         @Override
         public void afterSettingDiskRef() {
-          Thread clearTh = new Thread(new Runnable() {
-            public void run() {
-              region.clear();
-            }
-          });
+          Thread clearTh = new Thread(() -> region.clear());
           clearTh.start();
           try {
             ThreadUtils.join(clearTh, 120 * 1000);
@@ -3036,10 +1418,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       region.close();
       region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
       assertEquals(1, region.size());
-      assertEquals("value2", (String) region.get("key1"));
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail("Test failed due to exception" + e);
+      assertEquals("value2", region.get("key1"));
     } finally {
       testFailed = false;
       proceed = false;
@@ -3061,45 +1440,28 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     assertNotNull(region);
     region.put("key", "createValue");
     region.put("key1", "createValue1");
-    try {
-      cache.getCacheTransactionManager().begin();
-      region.destroy("key");
-      cache.getCacheTransactionManager().commit();
-      assertNull("The deleted entry should have been null",
-          ((LocalRegion) region).entries.getEntry("key"));
-    } catch (CommitConflictException e) {
-      testFailed = true;
-      fail("CommitConflitException encountered");
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail("Test failed due to exception" + e);
-    }
+    cache.getCacheTransactionManager().begin();
+    region.destroy("key");
+    cache.getCacheTransactionManager().commit();
+    assertNull("The deleted entry should have been null",
+        ((LocalRegion) region).entries.getEntry("key"));
   }
 
   /**
    * Test to force a recovery to follow the path of switchOutFilesForRecovery and ensuring that
    * IOExceptions do not come as a result. This is also a bug test for bug 37682
-   *
-   * @throws Exception
    */
   @Test
   public void testSwitchFilesForRecovery() throws Exception {
     region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, null, Scope.LOCAL);
     put100Int();
-    region.forceRolling();
+    ((LocalRegion) region).getDiskRegion().forceRolling();
     Thread.sleep(2000);
     put100Int();
     int sizeOfRegion = region.size();
     region.close();
-    // this variable will set to false in the src code itself
-    // NewLBHTreeDiskRegion.setJdbmexceptionOccurredToTrueForTesting = true;
-    try {
-      region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, null, Scope.LOCAL);
-    } catch (Exception e) {
-      fail("failed in recreating region due to" + e);
-    } finally {
-      // NewLBHTreeDiskRegion.setJdbmexceptionOccurredToTrueForTesting = false;
-    }
+    region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, null, Scope.LOCAL);
+
     if (sizeOfRegion != region.size()) {
       fail(" Expected region size to be " + sizeOfRegion + " after recovery but it is "
           + region.size());
@@ -3138,9 +1500,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
         if (this.didBeforeCall) {
           this.didBeforeCall = false;
           LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
-          // assertTrue("Assert failure for DSpaceUsage in afterHavingCompacted ",
-          // diskSpaceUsageStats() == calculatedDiskSpaceUsageStats());
-          // what is the point of this assert?
           checkDiskStats();
         }
       }
@@ -3157,20 +1516,12 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
       LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
       region.put("key1", val);
-      // Disk space should have changed due to 1 put
-      // assertTrue("stats did not increase after put 1 ", diskSpaceUsageStats() ==
-      // calculatedDiskSpaceUsageStats());
       checkDiskStats();
       region.put("key2", val);
-      // assertTrue("stats did not increase after put 2", diskSpaceUsageStats() ==
-      // calculatedDiskSpaceUsageStats());
       checkDiskStats();
       // This put will cause a switch as max-oplog size (500) will be exceeded (600)
       region.put("key3", val);
       synchronized (freezeRoller) {
-        // assertTrue("current disk space usage with Roller thread in wait and put key3 done is
-        // incorrect " + diskSpaceUsageStats() + " " + calculatedDiskSpaceUsageStats(),
-        // diskSpaceUsageStats()== calculatedDiskSpaceUsageStats());
         checkDiskStats();
         assertDone = true;
         freezeRoller.set(true);
@@ -3188,20 +1539,14 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       // "Disk space usage zero when region recreated"
       checkDiskStats();
       region.put("key4", val);
-      // assertTrue("stats did not increase after put 4", diskSpaceUsageStats() ==
-      // calculatedDiskSpaceUsageStats());
       checkDiskStats();
       region.put("key5", val);
-      // assertTrue("stats did not increase after put 5", diskSpaceUsageStats() ==
-      // calculatedDiskSpaceUsageStats());
       checkDiskStats();
       assertDone = false;
       LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
       region.put("key6", val);
       // again we expect a switch in oplog here
       synchronized (freezeRoller) {
-        // assertTrue("current disk space usage with Roller thread in wait and put key6 done is
-        // incorrect", diskSpaceUsageStats()== calculatedDiskSpaceUsageStats());
         checkDiskStats();
         assertDone = true;
         freezeRoller.set(true);
@@ -3317,14 +1662,9 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
       LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
       cache.getLogger().info("putting key1");
       region.put("key1", val);
-      // Disk space should have changed due to 1 put
-      // assertTrue("stats did not increase after put 1 ", diskSpaceUsageStats() ==
-      // calculatedDiskSpaceUsageStats());
       checkDiskStats();
       cache.getLogger().info("putting key2");
       region.put("key2", val);
-      // assertTrue("stats did not increase after put 2", diskSpaceUsageStats() ==
-      // calculatedDiskSpaceUsageStats());
       checkDiskStats();
 
       cache.getLogger().info("removing key1");
@@ -3352,145 +1692,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     }
   }
 
-  // @todo this test is broken; size1 can keep changing since the roller will
-  // keep copying forward forever. Need to change it so copy forward oplogs
-  // will not be compacted so that size1 reaches a steady state
-  /**
-   * Tests stats verification with rolling enabled
-   */
-  // @Test
-  // public void testSizeStatsAfterRecreationWithRollingEnabled() throws Exception
-  // {
-  // final int MAX_OPLOG_SIZE = 500;
-  // diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
-  // diskProps.setPersistBackup(true);
-  // diskProps.setRolling(true);
-  // diskProps.setCompactionThreshold(100);
-  // diskProps.setSynchronous(true);
-  // diskProps.setOverflow(false);
-  // diskProps.setDiskDirsAndSizes(new File[] { dirs[0] }, new int[] { 4000 });
-  // final byte[] val = new byte[200];
-  // region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache,
-  // diskProps);
-  // final DiskRegion dr = ((LocalRegion)region).getDiskRegion();
-  // final Object lock = new Object();
-  // final boolean [] exceptionOccurred = new boolean[] {true};
-  // final boolean [] okToExit = new boolean[] {false};
-
-  // CacheObserver old = CacheObserverHolder
-  // .setInstance(new CacheObserverAdapter() {
-  // private long before = -1;
-  // public void beforeDeletingCompactedOplog(Oplog rolledOplog)
-  // {
-  // if (before == -1) {
-  // // only want to call this once; before the 1st oplog destroy
-  // before = dr.getNextDir().getDirStatsDiskSpaceUsage();
-  // }
-  // }
-  // public void afterHavingCompacted() {
-  // if(before > -1) {
-  // synchronized(lock) {
-  // okToExit[0] = true;
-  // long after = dr.getNextDir().getDirStatsDiskSpaceUsage();;
-  // exceptionOccurred[0] = false;
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
-  // lock.notify();
-  // }
-  // }
-  // }
-  // });
-  // try {
-
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
-  // region.put("key1", val);
-  // region.put("key2", val);
-  // // This put will cause a switch as max-oplog size (500) will be exceeded (600)
-  // region.put("key3", val);
-  // synchronized(lock) {
-  // if (!okToExit[0]) {
-  // lock.wait(9000);
-  // assertTrue(okToExit[0]);
-  // }
-  // assertFalse(exceptionOccurred[0]);
-  // }
-  // while (region.forceCompaction() != null) {
-  // // wait until no more oplogs to compact
-  // Thread.sleep(50);
-  // }
-  // long size1 =0;
-  // for(DirectoryHolder dh:dr.getDirectories()) {
-  // cache.getLogger().info(" dir=" + dh.getDir()
-  // + " size1=" + dh.getDirStatsDiskSpaceUsage());
-  // size1 += dh.getDirStatsDiskSpaceUsage();
-  // }
-  // System.out.println("Size before closing= "+ size1);
-  // region.close();
-  // diskProps.setRolling(false);
-  // region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache,
-  // diskProps);
-
-  // long size2 =0;
-  // for(DirectoryHolder dh:((LocalRegion)region).getDiskRegion().getDirectories()) {
-  // cache.getLogger().info(" dir=" + dh.getDir()
-  // + " size2=" + dh.getDirStatsDiskSpaceUsage());
-  // size2 += dh.getDirStatsDiskSpaceUsage();
-  // }
-  // System.out.println("Size after recreation= "+ size2);
-  // assertIndexDetailsEquals(size1, size2);
-  // region.close();
-
-  // }
-  // finally {
-  // LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
-  // CacheObserverHolder.setInstance(old);
-  // }
-  // }
-
-  // This test is not valid. When listenForDataSerializeChanges is called
-  // it ALWAYS does vrecman writes and a commit. Look at saveInstantiators
-  // and saveDataSerializers to see these commit calls.
-  // These calls can cause the size of the files to change.
-  /**
-   * Tests if without rolling the region size before close is same as after recreation
-   */
-  @Test
-  public void testSizeStatsAfterRecreation() throws Exception {
-    final int MAX_OPLOG_SIZE = 500;
-    diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
-    diskProps.setPersistBackup(true);
-    diskProps.setRolling(false);
-    diskProps.setSynchronous(true);
-    diskProps.setOverflow(false);
-    diskProps.setDiskDirsAndSizes(new File[] {dirs[0], dirs[1]}, new int[] {4000, 4000});
-    final byte[] val = new byte[200];
-    region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
-    DiskRegion dr = ((LocalRegion) region).getDiskRegion();
-
-    try {
-      LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
-      for (int i = 0; i < 8; ++i) {
-        region.put("key" + i, val);
-      }
-      long size1 = 0;
-      for (DirectoryHolder dh : dr.getDirectories()) {
-        size1 += dh.getDirStatsDiskSpaceUsage();
-      }
-      System.out.println("Size before close = " + size1);
-      region.close();
-      region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
-      dr = ((LocalRegion) region).getDiskRegion();
-      long size2 = 0;
-      for (DirectoryHolder dh : dr.getDirectories()) {
-        size2 += dh.getDirStatsDiskSpaceUsage();
-      }
-      System.out.println("Size after recreation= " + size2);
-      assertEquals(size1, size2);
-      region.close();
-    } finally {
-      LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
-    }
-  }
-
   @Test
   public void testUnPreblowOnRegionCreate() throws Exception {
     final int MAX_OPLOG_SIZE = 20000;
@@ -3618,7 +1819,7 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     });
 
     File[] files = dir.listFiles();
-    HashSet<String> verified = new HashSet<String>();
+    HashSet<String> verified = new HashSet<>();
     for (File file : files) {
       String name = file.getName();
       byte[] expect = new byte[Oplog.OPLOG_MAGIC_SEQ_REC_SIZE];
@@ -3704,8 +1905,6 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     }
   }
 
-
-
   @Test
   public void testAsynchModeStatsBehaviour() throws Exception {
     final int MAX_OPLOG_SIZE = 1000;
@@ -3753,15 +1952,12 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
     }
   }
 
-  protected long diskSpaceUsageStats() {
+  private long diskSpaceUsageStats() {
     return ((LocalRegion) region).getDiskRegion().getInfoFileDir().getDirStatsDiskSpaceUsage();
   }
 
-  protected long calculatedDiskSpaceUsageStats() {
-    long oplogSize = oplogSize();
-    // cache.getLogger().info(" oplogSize=" + oplogSize
-    // + " statSize=" + diskSpaceUsageStats());
-    return oplogSize;
+  private long calculatedDiskSpaceUsageStats() {
+    return oplogSize();
   }
 
   private void checkDiskStats() {
@@ -3782,14 +1978,11 @@ public class OplogJUnitTest extends DiskRegionTestingBase {
 
   private long oplogSize() {
     long size = ((LocalRegion) region).getDiskRegion().getDiskStore().undeletedOplogSize.get();
-    // cache.getLogger().info("undeletedOplogSize=" + size);
     Oplog[] opArray =
         ((LocalRegion) region).getDiskRegion().getDiskStore().persistentOplogs.getAllOplogs();
-    if ((opArray != null) && (opArray.length != 0)) {
-      for (int j = 0; j < opArray.length; ++j) {
-        size += opArray[j].getOplogSize();
-        // cache.getLogger().info("oplog#" + opArray[j].getOplogId()
-        // + ".size=" + opArray[j].getOplogSize());
+    if (opArray != null) {
+      for (Oplog log : opArray) {
+        size += log.getOplogSize();
       }
     }
     return size;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/SimpleDiskRegionJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/SimpleDiskRegionJUnitTest.java
index ac5d965..c19a728 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/SimpleDiskRegionJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/SimpleDiskRegionJUnitTest.java
@@ -214,7 +214,7 @@ public class SimpleDiskRegionJUnitTest extends DiskRegionTestingBase {
   // newOplog = dr.getChild();
   // assertIndexDetailsEquals(null, region.get(new Integer(1)));
   // try {
-  // dr.addToOplogSet(id, new File(oplog.getOplogFile()
+  // dr.addToOplogSet(id, new File(oplog.getOplogFileForTest()
   // .getPath()), dr.getNextDir());
   // }
   // catch (Exception e) {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowAsyncRollingOpLogJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowAsyncRollingOpLogJUnitTest.java
index a9f81bd..aa8b46c 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowAsyncRollingOpLogJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowAsyncRollingOpLogJUnitTest.java
@@ -114,7 +114,7 @@ public class DiskRegionOverflowAsyncRollingOpLogJUnitTest extends DiskRegionTest
 
   private void populateSecond10kto20kwrites() {
     afterHavingCompacted = false;
-    DiskRegionTestingBase.setCacheObserverCallBack();
+    setCacheObserverCallBack();
     CacheObserverHolder.setInstance(new CacheObserverAdapter() {
       public void afterHavingCompacted() {
         afterHavingCompacted = true;
@@ -185,7 +185,7 @@ public class DiskRegionOverflowAsyncRollingOpLogJUnitTest extends DiskRegionTest
     log.info(statsGet2);
     if (debug)
       System.out.println("Perf Stats of get which is fauting in from Second OpLog  :" + statsGet2);
-    DiskRegionTestingBase.unSetCacheObserverCallBack();
+    unSetCacheObserverCallBack();
   }
 
   /**
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowSyncRollingOpLogJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowSyncRollingOpLogJUnitTest.java
index 186cced..14fcc0d 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowSyncRollingOpLogJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionOverflowSyncRollingOpLogJUnitTest.java
@@ -106,7 +106,7 @@ public class DiskRegionOverflowSyncRollingOpLogJUnitTest extends DiskRegionTesti
 
     // LRUStatistics lruStats = getLRUStats(region);
 
-    DiskRegionTestingBase.setCacheObserverCallBack();
+    setCacheObserverCallBack();
 
     CacheObserverHolder.setInstance(new CacheObserverAdapter() {
       public void afterHavingCompacted() {
@@ -175,7 +175,7 @@ public class DiskRegionOverflowSyncRollingOpLogJUnitTest extends DiskRegionTesti
     log.info(statsGet2);
     System.out.println("Perf Stats of get which is fauting in from Second OpLog  :" + statsGet2);
 
-    DiskRegionTestingBase.unSetCacheObserverCallBack();
+    unSetCacheObserverCallBack();
 
   }
 
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
index 77f8833..8c017b2 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionPerfJUnitPerformanceTest.java
@@ -494,7 +494,7 @@ public class DiskRegionPerfJUnitPerformanceTest extends DiskRegionTestingBase {
     log.info(stats_ForSameKeyputs);
   }
 
-  protected static void deleteFiles() {
+  protected void deleteFiles() {
     for (int i = 0; i < 4; i++) {
       File[] files = dirs[i].listFiles();
       for (int j = 0; j < files.length; j++) {
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionRollOpLogJUnitPerformanceTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionRollOpLogJUnitPerformanceTest.java
index e0338c6..208dd06 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionRollOpLogJUnitPerformanceTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/diskPerf/DiskRegionRollOpLogJUnitPerformanceTest.java
@@ -558,7 +558,7 @@ public class DiskRegionRollOpLogJUnitPerformanceTest extends DiskRegionTestingBa
     log.info(stats_ForSameKeyputs);
   }
 
-  protected static void deleteFiles() {
+  protected void deleteFiles() {
     for (int i = 0; i < 4; i++) {
       File[] files = dirs[i].listFiles();
       for (int j = 0; j < files.length; j++) {

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