You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2018/03/26 17:37:32 UTC

[geode] 07/19: GEODE-1279: Rename Bug37377DUnitTest as ClearDuringGiiOplogWithMissingCreateRegressionTest

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

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

commit 3e2619fe630f4b6bcb124908a7354e7bc10e4b4c
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Wed Mar 21 15:16:42 2018 -0700

    GEODE-1279: Rename Bug37377DUnitTest as ClearDuringGiiOplogWithMissingCreateRegressionTest
---
 .../geode/internal/cache/Bug37377DUnitTest.java    | 317 ---------------------
 ...ingGiiOplogWithMissingCreateRegressionTest.java | 229 +++++++++++++++
 2 files changed, 229 insertions(+), 317 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/Bug37377DUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/Bug37377DUnitTest.java
deleted file mode 100644
index 35e00c1..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/Bug37377DUnitTest.java
+++ /dev/null
@@ -1,317 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
- * agreements. See the NOTICE file distributed with this work for additional information regarding
- * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License. You may obtain a
- * copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software distributed under the License
- * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
- * or implied. See the License for the specific language governing permissions and limitations under
- * the License.
- */
-package org.apache.geode.internal.cache;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.util.Properties;
-import java.util.concurrent.CountDownLatch;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.cache.AttributesFactory;
-import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheFactory;
-import org.apache.geode.cache.DataPolicy;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.RegionAttributes;
-import org.apache.geode.cache.Scope;
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.internal.cache.entries.VersionedThinDiskRegionEntryHeapObjectKey;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.junit.categories.DistributedTest;
-
-/**
- * Bug37377 DUNIT Test: The Clear operation during a GII in progress can leave a Entry in the Oplog
- * due to a race condition wherein the clearFlag getting set after the entry gets written to the
- * disk, The Test verifies the existence of the scenario.
- *
- */
-
-@Category(DistributedTest.class)
-public class Bug37377DUnitTest extends JUnit4CacheTestCase {
-
-  protected static String regionName = "TestRegion";
-
-  static Properties props = new Properties();
-
-  protected static DistributedSystem distributedSystem = null;
-
-  VM vm0, vm1;
-
-  protected static Cache cache = null;
-
-  protected static File[] dirs = null;
-
-  private static final int maxEntries = 10000;
-
-  private static transient CountDownLatch clearLatch = new CountDownLatch(1);
-
-  static Boolean clearOccurred = false;
-
-  public Bug37377DUnitTest() {
-    super();
-    File file1 = new File(getTestMethodName() + "1");
-    file1.mkdir();
-    file1.deleteOnExit();
-    File file2 = new File(getTestMethodName() + "2");
-    file2.mkdir();
-    file2.deleteOnExit();
-    dirs = new File[2];
-    dirs[0] = file1;
-    dirs[1] = file2;
-  }
-
-  @Override
-  public final void postSetUp() throws Exception {
-    final Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-    vm1 = host.getVM(1);
-  }
-
-  @Override
-  public final void preTearDownCacheTestCase() throws Exception {
-    vm1.invoke(() -> destroyRegion());
-    vm0.invoke(() -> destroyRegion());
-  }
-
-  /**
-   * This method is used to create Cache in VM0
-   */
-
-  @SuppressWarnings("deprecation")
-  private void createCacheForVM0() {
-    try {
-
-      distributedSystem = (new Bug37377DUnitTest()).getSystem(props);
-      assertTrue(distributedSystem != null);
-      cache = CacheFactory.create(distributedSystem);
-      assertTrue(cache != null);
-      AttributesFactory factory = new AttributesFactory();
-      factory.setScope(Scope.DISTRIBUTED_ACK);
-      factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
-      factory.setDiskSynchronous(false);
-      factory.setDiskStoreName(
-          cache.createDiskStoreFactory().setDiskDirs(dirs).create("Bug37377DUnitTest").getName());
-      RegionAttributes attr = factory.create();
-      cache.createRegion(regionName, attr);
-    } catch (Exception ex) {
-      ex.printStackTrace();
-      fail("Error Creating cache / region ");
-    }
-  }
-
-  /**
-   * This method is used to create Cache in VM1
-   */
-  @SuppressWarnings("deprecation")
-  private void createCacheForVM1() {
-    try {
-      distributedSystem = (new Bug37377DUnitTest()).getSystem(props);
-      assertTrue(distributedSystem != null);
-      cache = CacheFactory.create(distributedSystem);
-      assertTrue("cache found null", cache != null);
-
-      AttributesFactory factory = new AttributesFactory();
-      factory.setScope(Scope.DISTRIBUTED_ACK);
-      factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
-      factory.setDiskSynchronous(false);
-      factory.setDiskStoreName(
-          cache.createDiskStoreFactory().setDiskDirs(dirs).create("Bug37377DUnitTest").getName());
-      RegionAttributes attr = factory.create();
-      DistributedRegion distRegion = new DistributedRegion(regionName, attr, null,
-          (GemFireCacheImpl) cache, new InternalRegionArguments().setDestroyLockFlag(true)
-              .setRecreateFlag(false).setSnapshotInputStream(null).setImageTarget(null));
-      // assertTrue("Distributed Region is null", distRegion != null); (cannot be null)
-
-      TestAbstractDiskRegionEntry.setMembers(vm1, vm0); // vm1 is thisVM, vm0 is otherVM
-
-      ((AbstractRegionMap) distRegion.entries)
-          .setEntryFactory(TestAbstractDiskRegionEntry.getEntryFactory());
-
-      LocalRegion region = (LocalRegion) ((GemFireCacheImpl) cache).createVMRegion(regionName, attr,
-          new InternalRegionArguments().setInternalMetaRegion(distRegion).setDestroyLockFlag(true)
-              .setSnapshotInputStream(null).setImageTarget(null));
-      assertTrue("Local Region is null", region != null);
-
-    } catch (Exception ex) {
-      ex.printStackTrace();
-      fail("Error Creating cache / region " + ex);
-    }
-  }
-
-  /**
-   * This method puts in maxEntries in the Region
-   */
-  private void putSomeEntries() {
-    assertTrue("Cache is found as null ", cache != null);
-    Region rgn = cache.getRegion(regionName);
-    for (int i = 0; i < maxEntries; i++) {
-      rgn.put(new Long(i), new Long(i));
-    }
-  }
-
-  /**
-   * This method clears the region and notifies the other member when complete
-   */
-  private static void invokeRemoteClearAndWait(VM remoteVM, VM thisVM) {
-    remoteVM.invoke(() -> clearRegionAndNotify(thisVM));
-    try {
-      clearLatch.await();
-    } catch (InterruptedException e) {
-      fail("wait for remote clear to complete failed");
-    }
-  }
-
-  /**
-   * This method clears the region and notifies the other member when complete
-   */
-  private static void clearRegionAndNotify(VM otherVM) {
-    assertTrue("Cache is found as null ", cache != null);
-    Region rgn = cache.getRegion(regionName);
-    rgn.clear();
-    otherVM.invoke(() -> notifyClearComplete());
-  }
-
-  /**
-   * Decrement countdown latch to notify clear complete
-   */
-  private static void notifyClearComplete() {
-    clearLatch.countDown();
-  }
-
-  /**
-   * This method destroys the Region
-   */
-  private void destroyRegion() {
-    try {
-      assertTrue("Cache is found as null ", cache != null);
-      Region rgn = cache.getRegion(regionName);
-      rgn.localDestroyRegion();
-      cache.close();
-    } catch (Exception ex) {
-    }
-  }
-
-  /**
-   * This method closes the cache on the specified VM
-   */
-  private void closeCacheForVM(final int vmNo) {
-    if (vmNo == 0) {
-      cache.getRegion(regionName).localDestroyRegion();
-    }
-    assertTrue("Cache is found as null ", cache != null);
-    cache.close();
-  }
-
-  /**
-   * This method verifies that the reintialized region size is zero
-   */
-  private void verifyExtraEntryFromOpLogs() {
-    assertTrue("Cache is found as null ", cache != null);
-    Region rgn = cache.getRegion(regionName);
-    // should be zero after clear
-    assertEquals(0, rgn.size());
-  }
-
-  /**
-   * The Clear operation during a GII in progress can leave a Entry in the Oplog due to a race
-   * condition wherein the clearFlag getting set after the entry gets written to the disk, The Test
-   * verifies the existence of the scenario.
-   *
-   */
-
-  @Test
-  public void testGIIputWithClear() {
-    vm0.invoke(() -> createCacheForVM0());
-    vm0.invoke(() -> putSomeEntries());
-
-    vm1.invoke(() -> createCacheForVM1());
-
-    vm0.invoke(() -> closeCacheForVM(0));
-    vm1.invoke(() -> closeCacheForVM(1));
-
-    vm1.invoke(() -> createCacheForVM1());
-    vm1.invoke(() -> verifyExtraEntryFromOpLogs());
-  }
-
-  static class TestAbstractDiskRegionEntry extends VersionedThinDiskRegionEntryHeapObjectKey {
-    private static VM thisVM, otherVM;
-
-    static void setMembers(VM localVM, VM remoteVM) {
-      thisVM = localVM;
-      otherVM = remoteVM;
-    }
-
-    protected TestAbstractDiskRegionEntry(RegionEntryContext r, Object key, Object value) {
-      super(r, key, value);
-    }
-
-    private static RegionEntryFactory factory = new RegionEntryFactory() {
-
-      public RegionEntry createEntry(RegionEntryContext r, Object key, Object value) {
-        return new TestAbstractDiskRegionEntry(r, key, value);
-      }
-
-      public Class getEntryClass() {
-        return TestAbstractDiskRegionEntry.class;
-      }
-
-      public RegionEntryFactory makeVersioned() {
-        return this;
-      }
-
-      public RegionEntryFactory makeOnHeap() {
-        return this;
-      }
-    };
-
-    /**
-     * Overridden setValue method to call clear Region before actually writing the entry
-     */
-    @Override
-    public boolean initialImageInit(final InternalRegion r, final long lastModifiedTime,
-        final Object newValue, final boolean create, final boolean wasRecovered,
-        final boolean versionTagAccepted) throws RegionClearedException {
-      synchronized (clearOccurred) {
-        if (!clearOccurred) {
-          // Force other member to perform a clear during our GII
-          invokeRemoteClearAndWait(otherVM, thisVM);
-          clearOccurred = true;
-        }
-      }
-
-      // Continue GII processing, which should throw RegionClearedException after the clear
-      try {
-        boolean result = super.initialImageInit(r, lastModifiedTime, newValue, create, wasRecovered,
-            versionTagAccepted);
-      } catch (RegionClearedException rce) {
-        throw rce;
-      } catch (Exception ex) {
-        fail("Caught exception during initialImageInit: " + ex);
-      }
-
-      return true;
-    }
-
-    public static RegionEntryFactory getEntryFactory() {
-      return factory;
-    }
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/ClearDuringGiiOplogWithMissingCreateRegressionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/ClearDuringGiiOplogWithMissingCreateRegressionTest.java
new file mode 100644
index 0000000..066de28
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/ClearDuringGiiOplogWithMissingCreateRegressionTest.java
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.geode.internal.cache;
+
+import static org.apache.geode.test.dunit.Host.getHost;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.AttributesFactory;
+import org.apache.geode.cache.DataPolicy;
+import org.apache.geode.cache.DiskStore;
+import org.apache.geode.cache.DiskStoreFactory;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.internal.cache.entries.VersionedThinDiskRegionEntryHeapObjectKey;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.cache.CacheTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.rules.serializable.SerializableTemporaryFolder;
+import org.apache.geode.test.junit.rules.serializable.SerializableTestName;
+
+/**
+ * Bug37377 DUNIT Test: The Clear operation during a GII in progress can leave a Entry in the Oplog
+ * due to a race condition wherein the clearFlag getting set after the entry gets written to the
+ * disk, The Test verifies the existence of the scenario.
+ *
+ * <p>
+ * TRAC #37377: Clear operation with GII in progress may result in a deleted entry to be logged in
+ * the oplog without accompanying create
+ */
+@Category(DistributedTest.class)
+public class ClearDuringGiiOplogWithMissingCreateRegressionTest extends CacheTestCase {
+
+  private static final int PUT_COUNT = 10000;
+
+  private String uniqueName;
+  private String regionName;
+  private File[] foldersForServer1;
+  private File[] foldersForServer2;
+
+  private VM server1;
+  private VM server2;
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Rule
+  public SerializableTestName testName = new SerializableTestName();
+
+  @Before
+  public void setUp() throws Exception {
+    server1 = getHost(0).getVM(0);
+    server2 = getHost(0).getVM(1);
+
+    uniqueName = getClass().getSimpleName() + "_" + testName.getMethodName();
+    regionName = uniqueName;
+
+    File server1Disk1 = temporaryFolder.newFolder(uniqueName + "_server1_disk1");
+    foldersForServer1 = new File[] {server1Disk1};
+
+    File server2Disk1 = temporaryFolder.newFolder(uniqueName + "_server2_disk1");
+    foldersForServer2 = new File[] {server2Disk1};
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    disconnectAllFromDS();
+  }
+
+  /**
+   * The Clear operation during a GII in progress can leave a Entry in the Oplog due to a race
+   * condition wherein the clearFlag getting set after the entry gets written to the disk, The Test
+   * verifies the existence of the scenario.
+   */
+  @Test
+  public void clearDuringGiiShouldOplogCreateAndDelete() {
+    server1.invoke(() -> createCacheForVM0());
+    server1.invoke(() -> {
+      Region<Integer, Integer> region = getCache().getRegion(regionName);
+      for (int i = 0; i < PUT_COUNT; i++) {
+        region.put(i, i);
+      }
+    });
+
+    server2.invoke(() -> createCacheForVM1());
+
+    server1.invoke(() -> {
+      getCache().getRegion(regionName).localDestroyRegion();
+      getCache().close();
+    });
+
+    server2.invoke(() -> getCache().close());
+
+    server2.invoke(() -> createCacheForVM1());
+    server2.invoke(() -> assertThatRegionSizeIsZero());
+  }
+
+  /**
+   * This method is used to create Cache in VM0
+   */
+  private void createCacheForVM0() {
+    DiskStoreFactory dsf = getCache().createDiskStoreFactory();
+    dsf.setDiskDirs(foldersForServer1);
+
+    DiskStore diskStore = dsf.create(uniqueName);
+
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+    factory.setDiskSynchronous(false);
+    factory.setDiskStoreName(diskStore.getName());
+
+    getCache().createRegion(regionName, factory.create());
+  }
+
+  /**
+   * This method is used to create Cache in VM1
+   */
+  private void createCacheForVM1() throws IOException, ClassNotFoundException {
+    DiskStoreFactory dsf = getCache().createDiskStoreFactory();
+    dsf.setDiskDirs(foldersForServer2);
+
+    DiskStore diskStore = dsf.create(uniqueName);
+
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+    factory.setDiskSynchronous(false);
+    factory.setDiskStoreName(diskStore.getName());
+
+    DistributedRegion distRegion = new DistributedRegion(regionName, factory.create(), null,
+        getCache(), new InternalRegionArguments().setDestroyLockFlag(true).setRecreateFlag(false)
+            .setSnapshotInputStream(null).setImageTarget(null));
+
+    distRegion.entries.setEntryFactory(new TestableDiskRegionEntryFactory());
+
+    getCache().createVMRegion(regionName, factory.create(),
+        new InternalRegionArguments().setInternalMetaRegion(distRegion).setDestroyLockFlag(true)
+            .setSnapshotInputStream(null).setImageTarget(null));
+  }
+
+  /**
+   * This method clears the region and notifies the other member when complete
+   */
+  private void invokeRemoteClearAndWait() {
+    server1.invoke(() -> {
+      Region region = getCache().getRegion(regionName);
+      region.clear();
+    });
+  }
+
+  private void assertThatRegionSizeIsZero() {
+    assertThat(getCache().getRegion(regionName).size()).isZero();
+  }
+
+  private class TestableDiskRegionEntry extends VersionedThinDiskRegionEntryHeapObjectKey {
+
+    TestableDiskRegionEntry(RegionEntryContext context, Object key, Object value) {
+      super(context, key, value);
+    }
+
+    /**
+     * Overridden setValue method to call clear Region before actually writing the entry
+     */
+    @Override
+    public boolean initialImageInit(final InternalRegion region, final long lastModified,
+        final Object newValue, final boolean create, final boolean wasRecovered,
+        final boolean acceptedVersionTag) throws RegionClearedException {
+
+      invokeRemoteClearAndWait();
+
+      // Continue GII processing, which should throw RegionClearedException after the clear
+      boolean result;
+      try {
+        result = super.initialImageInit(region, lastModified, newValue, create, wasRecovered,
+            acceptedVersionTag);
+      } catch (RegionClearedException e) {
+        throw e;
+      } catch (Exception e) {
+        throw new RuntimeException("initialImageInit threw " + e.getClass().getSimpleName(), e);
+      }
+
+      return result;
+    }
+  }
+
+  private class TestableDiskRegionEntryFactory implements RegionEntryFactory {
+
+    @Override
+    public RegionEntry createEntry(RegionEntryContext r, Object key, Object value) {
+      return new TestableDiskRegionEntry(r, key, value);
+    }
+
+    @Override
+    public Class getEntryClass() {
+      return TestableDiskRegionEntry.class;
+    }
+
+    @Override
+    public RegionEntryFactory makeVersioned() {
+      return this;
+    }
+
+    @Override
+    public RegionEntryFactory makeOnHeap() {
+      return this;
+    }
+  }
+}

-- 
To stop receiving notification emails like this one, please contact
klund@apache.org.