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:33 UTC

[geode] 08/19: GEODE-1279: Rename Bug37500JUnitTest as DiskAccessExceptionDisablesServerRegressionTest

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 10eabfb3a89b8996aeeee2786a1f2c988bbfbf85
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Wed Mar 21 16:38:25 2018 -0700

    GEODE-1279: Rename Bug37500JUnitTest as DiskAccessExceptionDisablesServerRegressionTest
---
 .../geode/internal/cache/Bug37500JUnitTest.java    | 163 --------------
 ...ccessExceptionDisablesServerRegressionTest.java | 241 +++++++++++++++++++++
 2 files changed, 241 insertions(+), 163 deletions(-)

diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/Bug37500JUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/Bug37500JUnitTest.java
deleted file mode 100644
index a65b30f..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/Bug37500JUnitTest.java
+++ /dev/null
@@ -1,163 +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 static org.junit.Assert.fail;
-
-import java.io.File;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.cache.Scope;
-import org.apache.geode.test.junit.categories.IntegrationTest;
-
-/**
- * This is a bugtest for bug 37500.
- *
- *
- */
-@Category(IntegrationTest.class)
-public class Bug37500JUnitTest extends DiskRegionTestingBase {
-
-  /** The disk region configuration object for the test */
-  private DiskRegionProperties diskProps = new DiskRegionProperties();
-
-  /** The key for entry1 */
-  static final String KEY1 = "KEY1";
-
-  /** The key for entry2 */
-  static final String KEY2 = "KEY2";
-
-  /** Boolean to indicate the roller thread to proceed */
-  static volatile boolean proceedForRolling = false;
-
-  /**
-   * Boolean to decide whether we want to allow roller to run ( used via CacheObserver callback
-   */
-  static volatile boolean notifyRoller = false;
-
-  /**
-   * This test does the following: <br>
-   * 1. Create a disk-region with following configurations :
-   * <li>dirSize = 2000 bytes
-   * <li>maxOplogSize = 500 bytes
-   * <li>rolling = true
-   * <li>syncMode = true
-   * <li>approx size on disk for operations = 440 bytes<br>
-   *
-   * 2.Make Roller go into WAIT state via CacheObserverAdapter.beforeGoingToCompact callback<br>
-   * 3.Put 440 bytes , it will go in oplog1 <br>
-   * 4.Put another 440 bytes ,it will go in oplog1<br>
-   * 5.Put 440 bytes , switching will be caused, it will go in oplog2, Roller will remained blocked
-   * (step 2)<br>
-   * 6.Put 440 bytes , it will go in oplog2, oplog2 will now be full<br>
-   * 7.Notify the Roller and put 440 bytes , this will try further switching. The put will fail with
-   * exception due to bug 37500. The put thread takes an entry level lock for entry2 ( the one with
-   * KEY2) and tries to write to disk but there is no free space left, so it goes into wait,
-   * expecting Roller to free up the space. The roller, which has now been notified to run, tries to
-   * roll entry2 for which it seeks entry level lock which has been acquired by put-thread. So the
-   * put thread eventually comes out of the wait with DiskAccessException<br>
-   *
-   * Another scenario for this bug is, once the disk space was getting exhausted , the entry
-   * operation threads which had already taken a lock on Entry got stuck trying to seek the Oplog
-   * Lock. The switching thread had acquired the Oplog.lock & was waiting for the roller thread to
-   * free disk space. Since the roller needed to acquire Entry lock to roll, it was unable to do so
-   * because of entry operation threads. This would cause the entry operation threads to get
-   * DiskAccessException after completing the stipulated wait. The Roller was able to free space
-   * only when it has rolled all the relevant entries which could happen only when the entry
-   * operation threads released the entry lock after getting DiskAccessException.
-   *
-   *
-   * @throws Exception
-   */
-  @Test
-  public void testBug37500() throws Exception {
-    final int MAX_OPLOG_SIZE = 1000;
-    diskProps.setMaxOplogSize(MAX_OPLOG_SIZE);
-    diskProps.setPersistBackup(true);
-    diskProps.setRolling(true);
-    diskProps.setSynchronous(false);
-
-    File testdir = new File("bug37500-diskDir");
-    testdir.mkdir();
-    testdir.deleteOnExit();
-    diskProps.setDiskDirsAndSizes(new File[] {testdir}, new int[] {2000});
-
-    LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
-
-    region = DiskRegionHelperFactory.getSyncPersistOnlyRegion(cache, diskProps, Scope.LOCAL);
-    CacheObserver old = CacheObserverHolder.setInstance(new CacheObserverAdapter() {
-      public void beforeGoingToCompact() {
-        if (!proceedForRolling) {
-          synchronized (Bug37500JUnitTest.class) {
-            if (!proceedForRolling) {
-              try {
-                cache.getLogger().info("beforeGoingToCompact :: going into wait");
-                Bug37500JUnitTest.class.wait();
-              } catch (InterruptedException e) {
-                cache.getLogger().info("Roller interrupted");
-                fail("interrupted");
-              }
-              cache.getLogger().info("beforeGoingToCompact :: coming out of wait");
-            }
-          }
-        }
-      }
-
-      public void beforeSwitchingOplog() {
-        if (notifyRoller) {
-          cache.getLogger().info("beforeSwitchingOplog :: going to notify Roller");
-          synchronized (Bug37500JUnitTest.class) {
-            proceedForRolling = true;
-            Bug37500JUnitTest.class.notify();
-            cache.getLogger().info("beforeSwitchingOplog :: notified the Roller");
-          }
-        }
-
-      }
-    });
-
-    cache.getLogger().info("goin to put no. 1");
-    // put 440 bytes , it will go in oplog1
-    region.put(KEY1, new byte[420]);
-
-    cache.getLogger().info("goin to put no. 2");
-    // put another 440 bytes ,it will go in oplog1
-    region.put(KEY2, new byte[420]);
-
-    cache.getLogger().info("goin to put no. 3");
-    // put 440 bytes , switching will be caused, it will go in oplog2 (value
-    // size increased to 432 as key wont be written to disk for UPDATE)
-    region.put(KEY1, new byte[432]);
-
-    cache.getLogger().info("goin to put no. 4");
-    // put 440 bytes , it will go in oplog2
-    region.put(KEY1, new byte[432]);
-
-    notifyRoller = true;
-    cache.getLogger().info("goin to put no. 5");
-    // put 440 bytes , this will try further switching
-    region.put(KEY2, new byte[432]);
-
-    LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
-    CacheObserverHolder.setInstance(old);
-    closeDown();
-  }
-
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/DiskAccessExceptionDisablesServerRegressionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskAccessExceptionDisablesServerRegressionTest.java
new file mode 100644
index 0000000..6104204
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/DiskAccessExceptionDisablesServerRegressionTest.java
@@ -0,0 +1,241 @@
+/*
+ * 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.distributed.ConfigurationProperties.LOCATORS;
+import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
+
+import java.io.File;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+import org.junit.rules.TestName;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.DiskStore;
+import org.apache.geode.cache.DiskStoreFactory;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionFactory;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+/**
+ * This is a bugtest for bug 37500.
+ *
+ * <p>
+ * TRAC #37500: A DiskAccessException brings cache server to a stall
+ *
+ * <pre>
+ * com.gemstone.gemfire.cache.DiskAccessException:  Unable to get free space for creating an oplog after waiting for 20 seconds
+ *     at com.gemstone.gemfire.internal.cache.ComplexDiskRegion.getNextDir(ComplexDiskRegion.java:150)
+ *     at com.gemstone.gemfire.internal.cache.Oplog.switchOpLog(Oplog.java:2020)
+ *     at com.gemstone.gemfire.internal.cache.Oplog.basicModify(Oplog.java:2423)
+ *     at com.gemstone.gemfire.internal.cache.Oplog.modify(Oplog.java:2339)
+ *     at com.gemstone.gemfire.internal.cache.DiskRegion.put(DiskRegion.java:321)
+ *     at com.gemstone.gemfire.internal.cache.DiskEntry$Helper.writeToDisk(DiskEntry.java:362)
+ *     at com.gemstone.gemfire.internal.cache.DiskEntry$Helper.overflowToDisk(DiskEntry.java:532)
+ *     at com.gemstone.gemfire.internal.cache.AbstractLRURegionMap.evictEntry(AbstractLRURegionMap.java:164)
+ *     at com.gemstone.gemfire.internal.cache.AbstractLRURegionMap.lruUpdateCallback(AbstractLRURegionMap.java:240)
+ *     at com.gemstone.gemfire.internal.cache.AbstractRegionMap.basicPut(AbstractRegionMap.java:928)
+ *     at com.gemstone.gemfire.internal.cache.LocalRegion.virtualPut(LocalRegion.java:3605)
+ *     at com.gemstone.gemfire.internal.cache.DistributedRegion.virtualPut(DistributedRegion.java:151)
+ *     at com.gemstone.gemfire.internal.cache.LocalRegion.basicUpdate(LocalRegion.java:3591)
+ *     at com.gemstone.gemfire.internal.cache.AbstractUpdateOperation.doPutOrCreate(AbstractUpdateOperation.java:100)
+ *     at com.gemstone.gemfire.internal.cache.AbstractUpdateOperation$AbstractUpdateMessage.basicOperateOnRegion(AbstractUpdateOperation.java:171)
+ *     at com.gemstone.gemfire.internal.cache.AbstractUpdateOperation$AbstractUpdateMessage.operateOnRegion(AbstractUpdateOperation.java:154)
+ *     at com.gemstone.gemfire.internal.cache.DistributedCacheOperation$CacheOperationMessage.basicProcess(DistributedCacheOperation.java:487)
+ *     at com.gemstone.gemfire.internal.cache.DistributedCacheOperation$CacheOperationMessage.process(DistributedCacheOperation.java:404)
+ *     at com.gemstone.gemfire.distributed.internal.DistributionMessage.scheduleAction(DistributionMessage.java:242)
+ *     at com.gemstone.gemfire.distributed.internal.DistributionMessage.schedule(DistributionMessage.java:287)
+ *     at com.gemstone.gemfire.distributed.internal.DistributionManager.scheduleIncomingMessage(DistributionManager.java:2732)
+ *     at com.gemstone.gemfire.distributed.internal.DistributionManager.handleIncomingDMsg(DistributionManager.java:2424)
+ *     at com.gemstone.gemfire.distributed.internal.DistributionManager$MyListener.messageReceived(DistributionManager.java:3585)
+ *     at com.gemstone.gemfire.distributed.internal.membership.jgroup.JGroupMembershipManager.processMessage(JGroupMembershipManager.java:1349)
+ *     at com.gemstone.gemfire.distributed.internal.membership.jgroup.JGroupMembershipManager.handleOrDeferMessage(JGroupMembershipManager.java:1289)
+ *     at com.gemstone.gemfire.distributed.internal.membership.jgroup.JGroupMembershipManager$MyDCReceiver.messageReceived(JGroupMembershipManager.java:449)
+ *     at com.gemstone.gemfire.distributed.internal.direct.DirectChannel.receive(DirectChannel.java:535)
+ *     at com.gemstone.gemfire.internal.tcp.TCPConduit.messageReceived(TCPConduit.java:483)
+ *     at com.gemstone.gemfire.internal.tcp.Connection.dispatchMessage(Connection.java:3026)
+ *     at com.gemstone.gemfire.internal.tcp.Connection.processNIOBuffer(Connection.java:2861)
+ *     at com.gemstone.gemfire.internal.tcp.Connection.runNioReader(Connection.java:1332)
+ *     at com.gemstone.gemfire.internal.tcp.Connection.run(Connection.java:1257)
+ *     at java.lang.Thread.run(Thread.java:595)
+ * </pre>
+ */
+@Category(IntegrationTest.class)
+public class DiskAccessExceptionDisablesServerRegressionTest {
+
+  private static final int MAX_OPLOG_SIZE = 1000;
+  private static final String KEY1 = "KEY1";
+  private static final String KEY2 = "KEY2";
+
+  private Cache cache;
+  private Region<String, byte[]> region;
+  private MyCacheObserver observer;
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Rule
+  public TestName testName = new TestName();
+
+  @Before
+  public void setUp() throws Exception {
+    String uniqueName = getClass().getSimpleName() + "_" + testName.getMethodName();
+    File temporaryDirectory = temporaryFolder.newFolder(uniqueName);
+
+    DirectoryHolder.SET_DIRECTORY_SIZE_IN_BYTES_FOR_TESTING_PURPOSES = true;
+    DiskStoreImpl.SET_IGNORE_PREALLOCATE = true;
+    LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
+
+    observer = new MyCacheObserver();
+    CacheObserverHolder.setInstance(observer);
+
+    cache = new CacheFactory().set(LOCATORS, "").set(MCAST_PORT, "0").create();
+
+    DiskStoreFactory dsf = cache.createDiskStoreFactory();
+    dsf.setDiskDirsAndSizes(new File[] {temporaryDirectory}, new int[] {2000});
+    ((DiskStoreFactoryImpl) dsf).setMaxOplogSizeInBytes(MAX_OPLOG_SIZE);
+
+    DiskStore diskStore = dsf.create(uniqueName);
+
+    RegionFactory<String, byte[]> regionFactory =
+        cache.createRegionFactory(RegionShortcut.LOCAL_PERSISTENT);
+    regionFactory.setDiskStoreName(diskStore.getName());
+
+    region = regionFactory.create(uniqueName);
+  }
+
+  @After
+  public void tearDown() {
+    CacheObserverHolder.setInstance(null);
+
+    DirectoryHolder.SET_DIRECTORY_SIZE_IN_BYTES_FOR_TESTING_PURPOSES = false;
+    DiskStoreImpl.SET_IGNORE_PREALLOCATE = false;
+    LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = false;
+
+    cache.close();
+  }
+
+  /**
+   * This test does the following: <br>
+   * 1. Create a disk-region with following configurations:
+   * <ul>
+   * <li>dirSize = 2000 bytes
+   * <li>maxOplogSize = 500 bytes
+   * <li>rolling = true
+   * <li>syncMode = true
+   * <li>approx size on disk for operations = 440 bytes
+   * </ul>
+   *
+   * <p>
+   * 2.Make Roller go into WAIT state via CacheObserverAdapter.beforeGoingToCompact callback
+   *
+   * <p>
+   * 3.Put 440 bytes , it will go in oplog1
+   *
+   * <p>
+   * 4.Put another 440 bytes ,it will go in oplog1
+   *
+   * <p>
+   * 5.Put 440 bytes , switching will be caused, it will go in oplog2, Roller will remained blocked
+   * (step 2)
+   *
+   * <p>
+   * 6.Put 440 bytes , it will go in oplog2, oplog2 will now be full
+   *
+   * <p>
+   * 7.Notify the Roller and put 440 bytes , this will try further switching. The put will fail with
+   * exception due to bug 37500. The put thread takes an entry level lock for entry2 ( the one with
+   * KEY2) and tries to write to disk but there is no free space left, so it goes into wait,
+   * expecting Roller to free up the space. The roller, which has now been notified to run, tries to
+   * roll entry2 for which it seeks entry level lock which has been acquired by put-thread. So the
+   * put thread eventually comes out of the wait with DiskAccessException
+   *
+   * <p>
+   * Another scenario for this bug is, once the disk space was getting exhausted , the entry
+   * operation threads which had already taken a lock on Entry got stuck trying to seek the Oplog
+   * Lock. The switching thread had acquired the Oplog.lock & was waiting for the roller thread to
+   * free disk space. Since the roller needed to acquire Entry lock to roll, it was unable to do so
+   * because of entry operation threads. This would cause the entry operation threads to get
+   * DiskAccessException after completing the stipulated wait. The Roller was able to free space
+   * only when it has rolled all the relevant entries which could happen only when the entry
+   * operation threads released the entry lock after getting DiskAccessException.
+   */
+  @Test
+  public void testBug37500() throws Exception {
+    // put 440 bytes , it will go in oplog1
+    region.put(KEY1, new byte[420]);
+
+    // put another 440 bytes ,it will go in oplog1
+    region.put(KEY2, new byte[420]);
+
+    // put 440 bytes , switching will be caused, it will go in oplog2 (value
+    // size increased to 432 as key wont be written to disk for UPDATE)
+    region.put(KEY1, new byte[432]);
+
+    // put 440 bytes , it will go in oplog2
+    region.put(KEY1, new byte[432]);
+
+    observer.notifyRoller();
+
+    // put 440 bytes , this will try further switching
+    region.put(KEY2, new byte[432]);
+  }
+
+  private static class MyCacheObserver extends CacheObserverAdapter {
+
+    private final Object notification = new Object();
+
+    /**
+     * Flag to decide whether we want to allow roller to run
+     */
+    private volatile boolean notifyRoller = false;
+
+    private volatile boolean proceedForRolling = false;
+
+    void notifyRoller() {
+      notifyRoller = true;
+    }
+
+    @Override
+    public void beforeGoingToCompact() {
+      synchronized (notification) {
+        while (!proceedForRolling) {
+          try {
+            notification.wait();
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void beforeSwitchingOplog() {
+      if (notifyRoller) {
+        synchronized (notification) {
+          proceedForRolling = true;
+          notification.notifyAll();
+        }
+      }
+    }
+  }
+}

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