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

[geode] 10/19: GEODE-1279: Rename Bug40299DUnitTest as ClearDuringNetSearchOplogRegressionTest

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 e2e2b904b4688b2fdfd4a8713df66ae63057f7db
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Thu Mar 22 12:15:23 2018 -0700

    GEODE-1279: Rename Bug40299DUnitTest as ClearDuringNetSearchOplogRegressionTest
    
    * Move testNetSearchMessageDoGet from SearchLoadAndWriteProcessor to
    the test
    * Move generateEventID and getMyId from AbstractRegion to
    InternalRegion
    * Remove Thread.sleep and cleanup test
---
 .../geode/internal/cache/AbstractRegion.java       |  13 -
 .../geode/internal/cache/InternalRegion.java       |  13 +
 .../apache/geode/internal/cache/LocalRegion.java   |   2 +-
 .../cache/SearchLoadAndWriteProcessor.java         |  17 +-
 .../geode/internal/cache/Bug40299DUnitTest.java    | 263 ---------------------
 .../ClearDuringNetSearchOplogRegressionTest.java   | 205 ++++++++++++++++
 6 files changed, 222 insertions(+), 291 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
index 46c7a4e..bc5f792 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/AbstractRegion.java
@@ -371,19 +371,6 @@ public abstract class AbstractRegion implements InternalRegion, AttributesMutato
 
   abstract void basicClear(RegionEventImpl regionEvent);
 
-  /**
-   * The default Region implementation will generate EvenTID in the EntryEvent object. This method
-   * is overridden in special Region objects like HARegion or
-   * SingleWriteSingleReadRegionQueue.SingleReadWriteMetaRegion to return false as the event
-   * propagation from those regions do not need EventID objects. This method is made abstract to
-   * directly use it in clear operations. (clear and localclear)
-   *
-   * @return boolean indicating whether to generate eventID or not
-   */
-  abstract boolean generateEventID();
-
-  protected abstract InternalDistributedMember getMyId();
-
   @Override
   public void clear() {
     checkReadiness();
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 eeb2023..7c5d722 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
@@ -225,4 +225,17 @@ public interface InternalRegion extends Region, HasCachePerfStats, RegionEntryCo
   boolean isInitialized();
 
   void cleanupForClient(CacheClientNotifier ccn, ClientProxyMembershipID client);
+
+  InternalDistributedMember getMyId();
+
+  /**
+   * The default Region implementation will generate EvenTID in the EntryEvent object. This method
+   * is overridden in special Region objects like HARegion or
+   * SingleWriteSingleReadRegionQueue.SingleReadWriteMetaRegion to return false as the event
+   * propagation from those regions do not need EventID objects. This method is made abstract to
+   * directly use it in clear operations. (clear and localclear)
+   *
+   * @return boolean indicating whether to generate eventID or not
+   */
+  boolean generateEventID();
 }
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 7519048..b20c838 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
@@ -853,7 +853,7 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
    * @since GemFire 5.0
    */
   @Override
-  protected InternalDistributedMember getMyId() {
+  public InternalDistributedMember getMyId() {
     return this.cache.getInternalDistributedSystem().getDistributedMember();
   }
 
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/SearchLoadAndWriteProcessor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/SearchLoadAndWriteProcessor.java
index c7502b5..b55c9f4 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/SearchLoadAndWriteProcessor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/SearchLoadAndWriteProcessor.java
@@ -1338,17 +1338,6 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
   }
 
   /**
-   * Test method for bug 40299.
-   */
-  @SuppressWarnings("synthetic-access")
-  public void testNetSearchMessageDoGet(String theRegionName, Object theKey, int theTimeoutMs,
-      int theTtl, int theIdleTime) {
-    NetSearchRequestMessage nMsg = new NetSearchRequestMessage();
-    nMsg.initialize(this, theRegionName, theKey, theTimeoutMs, theTtl, theIdleTime);
-    nMsg.doGet((ClusterDistributionManager) this.distributionManager);
-  }
-
-  /**
    * A QueryMessage is broadcast to every node that has the region defined, to find out who has a
    * valid copy of the requested object.
    */
@@ -1769,8 +1758,8 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
 
     }
 
-    private void initialize(SearchLoadAndWriteProcessor processor, String theRegionName,
-        Object theKey, int timeoutMS, int ttlMS, int idleTimeMS) {
+    void initialize(SearchLoadAndWriteProcessor processor, String theRegionName, Object theKey,
+        int timeoutMS, int ttlMS, int idleTimeMS) {
       this.processorId = processor.processorId;
       this.regionName = theRegionName;
       this.key = theKey;
@@ -1842,7 +1831,7 @@ public class SearchLoadAndWriteProcessor implements MembershipListener {
           + "\" in region \"" + this.regionName + "\", processorId " + processorId;
     }
 
-    private void doGet(ClusterDistributionManager dm) {
+    void doGet(ClusterDistributionManager dm) {
       long startTime = dm.cacheTimeMillis();
       // boolean retVal = true;
       byte[] ebv = null;
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/Bug40299DUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/Bug40299DUnitTest.java
deleted file mode 100755
index fe0f796..0000000
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/Bug40299DUnitTest.java
+++ /dev/null
@@ -1,263 +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 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.EvictionAction;
-import org.apache.geode.cache.EvictionAttributes;
-import org.apache.geode.cache.Operation;
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.RegionAttributes;
-import org.apache.geode.cache.Scope;
-import org.apache.geode.cache30.CacheSerializableRunnable;
-import org.apache.geode.distributed.DistributedSystem;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.IgnoredException;
-import org.apache.geode.test.dunit.SerializableRunnable;
-import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.junit.categories.DistributedTest;
-
-/**
- * Bug40299 DUNIT Test: The Clear operation during a NetSearchMessage.doGet() in progress can cause
- * DiskAccessException by accessing cleared oplogs and eventually destroy region. The Test verifies
- * that fix prevents this.
- */
-@Category(DistributedTest.class)
-public class Bug40299DUnitTest extends JUnit4CacheTestCase {
-
-  protected static String regionName = "TestRegion";
-
-  static Properties props = new Properties();
-
-  protected static DistributedSystem distributedSystem = null;
-
-  private static VM vm0 = null;
-
-  protected static Cache cache = null;
-
-  @Override
-  public final void postSetUp() throws Exception {
-    final Host host = Host.getHost(0);
-    vm0 = host.getVM(0);
-  }
-
-  @Override
-  public final void preTearDownCacheTestCase() throws Exception {
-    vm0.invoke(destroyRegion());
-  }
-
-  /**
-   * This method is used to create Cache in VM0
-   *
-   * @return CacheSerializableRunnable
-   */
-
-  private CacheSerializableRunnable createCacheForVM0() {
-    SerializableRunnable createCache = new CacheSerializableRunnable("createCache") {
-      public void run2() {
-        try {
-
-          distributedSystem = (new Bug40299DUnitTest()).getSystem(props);
-          assertTrue(distributedSystem != null);
-          cache = CacheFactory.create(distributedSystem);
-          assertTrue(cache != null);
-          AttributesFactory factory = new AttributesFactory();
-          factory.setScope(Scope.DISTRIBUTED_ACK);
-          File dir = new File("testingDirectoryDefault");
-          dir.mkdir();
-          dir.deleteOnExit();
-          File[] dirs = {dir};
-          factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
-          factory.setDiskSynchronous(false);
-          factory.setDiskStoreName(cache.createDiskStoreFactory()
-              .setDiskDirsAndSizes(dirs, new int[] {Integer.MAX_VALUE}).setQueueSize(1)
-              .setMaxOplogSize(60) // does the test want 60 bytes or 60M?
-              .setAutoCompact(false).setTimeInterval(1000).create("Bug40299DUnitTest").getName());
-          factory.setEvictionAttributes(
-              EvictionAttributes.createLRUEntryAttributes(2, EvictionAction.OVERFLOW_TO_DISK));
-          RegionAttributes attr = factory.create();
-          cache.createRegion(regionName, attr);
-        } catch (Exception ex) {
-          ex.printStackTrace();
-          fail("Error Creating cache / region ");
-        }
-      }
-    };
-    return (CacheSerializableRunnable) createCache;
-  }
-
-  /**
-   * This method puts in 7 in the Region
-   *
-   * @return CacheSerializableRunnable
-   */
-  private CacheSerializableRunnable putSomeEntries() {
-    SerializableRunnable puts = new CacheSerializableRunnable("putSomeEntries") {
-      public void run2() {
-        assertTrue("Cache is found as null ", cache != null);
-        Region rgn = cache.getRegion(regionName);
-        for (int i = 0; i < 7; i++) {
-          rgn.put("key" + i, new Long(i));
-        }
-      }
-    };
-    return (CacheSerializableRunnable) puts;
-  }
-
-  /**
-   * This method does concurrent NetSearch.doGet with clear in the Region
-   *
-   * @return CacheSerializableRunnable
-   */
-  private CacheSerializableRunnable concurrentNetSearchGetAndClear() {
-    SerializableRunnable getAndClear =
-        new CacheSerializableRunnable("concurrentNetSearchGetAndClear") {
-          public void run2() {
-            assertTrue("Cache is found as null ", cache != null);
-            Region rgn = cache.getRegion(regionName);
-            assertTrue("Region size expected to be 7 but is " + rgn.size(), rgn.size() == 7);
-
-            Thread getThread1 = null;
-            LocalRegion lr = (LocalRegion) rgn;
-            lr.getDiskRegion().acquireWriteLock();
-            // got writeLock from diskregion
-            try {
-              getThread1 = new Thread(new getThread((LocalRegion) rgn));
-
-              // start getThread
-              getThread1.start();
-
-              // sleep for a while to allow getThread to wait for readLock.
-              Thread.sleep(1000);
-
-              // This test appears to be testing a problem with the non-RVV
-              // based clear. So we'll use that functionality here.
-              // Region.clear uses an RVV, and will deadlock if called while
-              // the write lock is held.
-              RegionEventImpl regionEvent = new RegionEventImpl(lr, Operation.REGION_CLEAR, null,
-                  false, lr.getMyId(), lr.generateEventID());
-              // clearRegion to remove entry that getThread has reference of
-              lr.cmnClearRegion(regionEvent, true, false);
-            } catch (InterruptedException e) {
-              if (cache.getLogger().fineEnabled()) {
-                cache.getLogger().fine("InterruptedException in run of localClearThread");
-              }
-            } finally {
-              ((LocalRegion) rgn).getDiskRegion().releaseWriteLock();
-            }
-            // allow getThread to join to set getAfterClearSuccessful
-            try {
-              getThread1.join();
-            } catch (InterruptedException ie) {
-              if (cache.getLogger().fineEnabled()) {
-                cache.getLogger().fine("InterruptedException in join of getThread");
-              }
-            }
-          }
-        };
-
-    return (CacheSerializableRunnable) getAndClear;
-  }
-
-  /**
-   *
-   * getThread
-   *
-   */
-  protected class getThread implements Runnable {
-    LocalRegion region = null;
-
-    getThread(LocalRegion rgn) {
-      super();
-      this.region = rgn;
-    }
-
-    public void run() {
-
-      SearchLoadAndWriteProcessor processor = SearchLoadAndWriteProcessor.getProcessor();
-      processor.initialize((LocalRegion) region, "key1", null);
-      processor.testNetSearchMessageDoGet(region.getName(), "key1", 1500, 1500, 1500);
-    }
-  }
-
-  /**
-   * This method verifies that region is not destroyed
-   *
-   * @return CacheSerializableRunnable
-   */
-  private CacheSerializableRunnable verifyRegionNotDestroyed() {
-    SerializableRunnable verifyR = new CacheSerializableRunnable("verifyRegionNotDestroyed") {
-      public void run2() {
-        assertTrue("Cache is found as null ", cache != null);
-        Region region = cache.getRegion(regionName);
-        assertTrue("Region was destroyed", region != null);
-      }
-    };
-    return (CacheSerializableRunnable) verifyR;
-  }
-
-  /**
-   * This method destroys the Region
-   *
-   * @return CacheSerializableRunnable
-   */
-  private CacheSerializableRunnable destroyRegion() {
-    SerializableRunnable destroyR = new CacheSerializableRunnable("destroyRegion") {
-      public void run2() {
-        try {
-          assertTrue("Cache is found as null ", cache != null);
-
-          Region rgn = cache.getRegion(regionName);
-          rgn.localDestroyRegion();
-          cache.close();
-        } catch (Exception ex) {
-
-        }
-      }
-    };
-    return (CacheSerializableRunnable) destroyR;
-  }
-
-  /**
-   * The Clear operation during a NetSearchMessage.doGet() in progress can cause DiskAccessException
-   * by accessing cleared oplogs and eventually destroy region. The Test verifies that fix prevents
-   * this.
-   */
-
-  @Test
-  public void testQueryGetWithClear() {
-    IgnoredException.addIgnoredException("Entry has been cleared and is not present on disk");
-    // create region in VM0
-    vm0.invoke(createCacheForVM0());
-    // Do puts to region.
-    vm0.invoke(putSomeEntries());
-    // call NetSearchMessage.doGet() after region.clear()
-    vm0.invoke(concurrentNetSearchGetAndClear());
-    // verify that region is not destroyed
-    vm0.invoke(verifyRegionNotDestroyed());
-  }
-}
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/ClearDuringNetSearchOplogRegressionTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/ClearDuringNetSearchOplogRegressionTest.java
new file mode 100755
index 0000000..ff6674a
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/ClearDuringNetSearchOplogRegressionTest.java
@@ -0,0 +1,205 @@
+/*
+ * 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 java.util.concurrent.TimeUnit.MINUTES;
+import static org.apache.geode.cache.EvictionAttributes.createLRUEntryAttributes;
+import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.awaitility.Awaitility.await;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+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.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.EvictionAction;
+import org.apache.geode.cache.Operation;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.Scope;
+import org.apache.geode.distributed.internal.ClusterDistributionManager;
+import org.apache.geode.internal.cache.SearchLoadAndWriteProcessor.NetSearchRequestMessage;
+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;
+
+/**
+ * The Clear operation during a NetSearchMessage.doGet() in progress can cause DiskAccessException
+ * by accessing cleared oplogs and eventually destroy region. The Test verifies that fix prevents
+ * this.
+ *
+ * <p>
+ * Test must be DistributedTest because it requires ClusterDistributionManager.
+ *
+ * <p>
+ * TRAC #40299: Suspect String - DiskAccessException : Data for DiskEntry could not be obtained
+ * from Disk. A clear operation may have deleted the oplogs (logged as error)
+ */
+@Category(DistributedTest.class)
+public class ClearDuringNetSearchOplogRegressionTest extends CacheTestCase {
+
+  private String uniqueName;
+  private String regionName;
+  private File[] diskDirs;
+  private transient CacheObserver observer;
+
+  @Rule
+  public SerializableTemporaryFolder temporaryFolder = new SerializableTemporaryFolder();
+
+  @Rule
+  public SerializableTestName testName = new SerializableTestName();
+
+  @Before
+  public void setUp() throws Exception {
+    uniqueName = getClass().getSimpleName() + "_" + testName.getMethodName();
+    regionName = uniqueName;
+
+    diskDirs = new File[] {temporaryFolder.newFolder(uniqueName)};
+
+    LocalRegion.ISSUE_CALLBACKS_TO_CACHE_OBSERVER = true;
+
+    observer = spy(CacheObserver.class);
+
+    addIgnoredException("Entry has been cleared and is not present on disk");
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    disconnectAllFromDS();
+  }
+
+  /**
+   * The Clear operation during a NetSearchMessage.doGet() in progress can cause DiskAccessException
+   * by accessing cleared oplogs and eventually destroy region. The Test verifies that fix prevents
+   * this.
+   */
+  @Test
+  public void testQueryGetWithClear() throws Exception {
+    // create region
+    createCacheForVM0();
+
+    // Do puts to region
+    putSevenEntries();
+
+    // call NetSearchMessage.doGet() after region.clear()
+    concurrentNetSearchGetAndClear();
+
+    // verify that region is not destroyed
+    verifyRegionNotDestroyed();
+  }
+
+  private void createCacheForVM0() {
+    DiskStoreFactory diskStoreFactory = getCache().createDiskStoreFactory();
+    diskStoreFactory.setDiskDirsAndSizes(diskDirs, new int[] {Integer.MAX_VALUE});
+    diskStoreFactory.setQueueSize(1);
+    diskStoreFactory.setMaxOplogSize(60); // does the test want 60 bytes or 60M?
+    diskStoreFactory.setAutoCompact(false).setTimeInterval(1000);
+
+    DiskStore diskStore = diskStoreFactory.create(uniqueName);
+
+    AttributesFactory factory = new AttributesFactory();
+    factory.setScope(Scope.DISTRIBUTED_ACK);
+    factory.setDataPolicy(DataPolicy.PERSISTENT_REPLICATE);
+    factory.setDiskSynchronous(false);
+    factory.setDiskStoreName(diskStore.getName());
+    factory.setEvictionAttributes(createLRUEntryAttributes(2, EvictionAction.OVERFLOW_TO_DISK));
+
+    getCache().createRegion(regionName, factory.create());
+  }
+
+  private void putSevenEntries() {
+    Region<String, Integer> region = getCache().getRegion(regionName);
+    for (int i = 0; i < 7; i++) {
+      region.put("key" + i, i);
+    }
+  }
+
+  private void concurrentNetSearchGetAndClear() throws InterruptedException {
+    InternalRegion region = (InternalRegion) getCache().getRegion(regionName);
+    assertThat(region.size()).isEqualTo(7);
+
+    Thread getter = new Thread(new Getter(region));
+
+    region.getDiskRegion().acquireWriteLock();
+    try {
+      doConcurrentNetSearchGetAndClear(getter, region);
+    } finally {
+      region.getDiskRegion().releaseWriteLock();
+    }
+
+    // allow getThread to join to set getAfterClearSuccessful
+    getter.join();
+  }
+
+  private void doConcurrentNetSearchGetAndClear(Thread getter, InternalRegion region) {
+    CacheObserverHolder.setInstance(observer);
+
+    // start getThread
+    getter.start();
+
+    await().atMost(1, MINUTES).until(() -> verify(observer, times(1)).afterSettingDiskRef());
+
+    // This test appears to be testing a problem with the non-RVV
+    // based clear. So we'll use that functionality here.
+    // Region.clear uses an RVV, and will deadlock if called while
+    // the write lock is held.
+    RegionEventImpl regionEvent = new RegionEventImpl(region, Operation.REGION_CLEAR, null, false,
+        region.getMyId(), region.generateEventID());
+
+    // clearRegion to remove entry that getter has reference of
+    ((LocalRegion) region).cmnClearRegion(regionEvent, true, false);
+  }
+
+  private void verifyRegionNotDestroyed() {
+    Region region = getCache().getRegion(regionName);
+    assertThat(region).isNotNull();
+    assertThat(region.isDestroyed()).isFalse();
+  }
+
+  private static class Getter implements Runnable {
+
+    private final InternalRegion region;
+
+    Getter(InternalRegion region) {
+      super();
+      this.region = region;
+    }
+
+    @Override
+    public void run() {
+      SearchLoadAndWriteProcessor processor = SearchLoadAndWriteProcessor.getProcessor();
+      processor.initialize((LocalRegion) region, "key1", null);
+      sendNetSearchRequestMessage(processor, "key1", 1500, 1500, 1500);
+    }
+
+    private void sendNetSearchRequestMessage(SearchLoadAndWriteProcessor processor, Object key,
+        int timeoutMillis, int ttlMillis, int idleMillis) {
+      NetSearchRequestMessage message = new SearchLoadAndWriteProcessor.NetSearchRequestMessage();
+      message.initialize(processor, region.getName(), key, timeoutMillis, ttlMillis, idleMillis);
+      message.doGet((ClusterDistributionManager) region.getDistributionManager());
+    }
+  }
+}

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