You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@geode.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/10/23 18:38:01 UTC

[jira] [Commented] (GEODE-1036) CI failure: InitializeIndexEntryDestroyQueryDUnitTest.testAsyncIndexInitDuringEntryDestroyAndQueryOnPR

    [ https://issues.apache.org/jira/browse/GEODE-1036?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16215625#comment-16215625 ] 

ASF GitHub Bot commented on GEODE-1036:
---------------------------------------

nabarunnag closed pull request #936: GEODE-1036: Refactored InitializeIndexEntryDestroyQueryDUnitTest
URL: https://github.com/apache/geode/pull/936
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/InitializeIndexEntryDestroyQueryDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/InitializeIndexEntryDestroyQueryDUnitTest.java
index 246e2c089f..00809d8b8b 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/InitializeIndexEntryDestroyQueryDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/internal/index/InitializeIndexEntryDestroyQueryDUnitTest.java
@@ -14,25 +14,25 @@
  */
 package org.apache.geode.cache.query.internal.index;
 
-import org.junit.experimental.categories.Category;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.junit.categories.DistributedTest;
-
 import static org.apache.geode.cache.query.Utils.createPortfolioData;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
 
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.logging.log4j.Logger;
+import org.awaitility.Awaitility;
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
 
-import org.apache.geode.cache.AttributesFactory;
 import org.apache.geode.cache.Cache;
-import org.apache.geode.cache.CacheException;
-import org.apache.geode.cache.PartitionAttributesFactory;
+import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionFactory;
 import org.apache.geode.cache.Scope;
@@ -43,42 +43,32 @@
 import org.apache.geode.cache.query.data.PortfolioData;
 import org.apache.geode.cache.query.internal.Undefined;
 import org.apache.geode.cache.query.partitioned.PRQueryDUnitHelper;
-import org.apache.geode.cache30.CacheSerializableRunnable;
-import org.apache.geode.cache30.CacheTestCase;
-import org.apache.geode.test.dunit.Assert;
+import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.test.dunit.AsyncInvocation;
 import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.LogWriterUtils;
 import org.apache.geode.test.dunit.ThreadUtils;
 import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.Wait;
-import org.apache.geode.test.junit.categories.FlakyTest;
+import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
 
 /**
  * Test creates a local region. Creates and removes index in a parallel running thread. Then
- * destroys and puts back entries in separated thread in the same region and runs query parallely
+ * destroys and puts back entries in separated thread in the same region and runs query in parallel
  * and checks for UNDEFINED values in result set of the query.
  */
 @Category(DistributedTest.class)
+@RunWith(JUnitParamsRunner.class)
 public class InitializeIndexEntryDestroyQueryDUnitTest extends JUnit4CacheTestCase {
+  private static final Logger logger = LogService.getLogger();
 
   PRQueryDUnitHelper PRQHelp = new PRQueryDUnitHelper();
 
-  String name;
-
-  final int redundancy = 0;
-
   final Portfolio portfolio = new Portfolio(1, 1);
 
   private int cnt = 0;
 
   private int cntDest = 100;
 
-  volatile static boolean hooked = false;
-
-  /**
-   * @param name
-   */
   public InitializeIndexEntryDestroyQueryDUnitTest() {
     super();
   }
@@ -89,373 +79,176 @@ public void setCacheInVMs(VM... vms) {
     }
   }
 
+  private static Scope[] getScope() {
+    return new Scope[] {Scope.LOCAL, null};
+  }
+
   @Test
-  public void testAsyncIndexInitDuringEntryDestroyAndQuery() {
+  @Parameters(method = "getScope")
+  public void testAsyncIndexInitDuringEntryDestroyAndQueryOnPR(Scope scope) {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     setCacheInVMs(vm0);
-    name = "PartionedPortfolios";
-    // Create Local Region
-    vm0.invoke(
-        new CacheSerializableRunnable("Create local region with asynchronous index maintenance") {
-          @Override
-          public void run2() throws CacheException {
-            Cache cache = getCache();
-            Region localRegion = null;
-            try {
-              AttributesFactory attr = new AttributesFactory();
-              attr.setValueConstraint(PortfolioData.class);
-              attr.setScope(Scope.LOCAL);
-              attr.setIndexMaintenanceSynchronous(false);
-              RegionFactory regionFactory = cache.createRegionFactory(attr.create());
-              localRegion = regionFactory.create(name);
-            } catch (IllegalStateException ex) {
-              LogWriterUtils.getLogWriter().warning("Creation caught IllegalStateException", ex);
-            }
-            assertNotNull("Region " + name + " not in cache", cache.getRegion(name));
-            assertNotNull("Region ref null", localRegion);
-            assertTrue("Region ref claims to be destroyed", !localRegion.isDestroyed());
-          }
-        });
-
-
-    final PortfolioData[] portfolio = createPortfolioData(cnt, cntDest);
-    // Putting the data into the PR's created
-    vm0.invoke(PRQHelp.getCacheSerializableRunnableForPRPuts(name, portfolio, cnt, cntDest));
-
-    AsyncInvocation asyInvk0 =
-        vm0.invokeAsync(new CacheSerializableRunnable("Create Index with Hook") {
-
-          @Override
-          public void run2() throws CacheException {
-
-            for (int i = 0; i < cntDest; i++) {
-              // Create Index first to go in hook.
-              Cache cache = getCache();
-              Index index = null;
-              try {
-                index = cache.getQueryService().createIndex("statusIndex", "p.status",
-                    "/" + name + " p");
-              } catch (Exception e1) {
-                e1.printStackTrace();
-                fail("Index creation failed");
-              }
-              assertNotNull(index);
-
-              Wait.pause(100);
-
-              getCache().getQueryService().removeIndex(index);
-
-              Wait.pause(100);
-            }
-          }
-        });
-
-    // Change the value in Region
-    AsyncInvocation asyInvk1 =
-        vm0.invokeAsync(new CacheSerializableRunnable("Change value in region") {
-
-          @Override
-          public void run2() throws CacheException {
-            // Do a put in region.
-            Region r = getCache().getRegion(name);
-
-            for (int i = 0, j = 0; i < 1000; i++, j++) {
-
-              PortfolioData p = (PortfolioData) r.get(j);
-
-              getCache().getLogger().fine("Going to destroy the value" + p);
-              r.destroy(j);
-
-              Wait.pause(100);
-
-              // Put the value back again.
-              getCache().getLogger().fine("Putting the value back" + p);
-              r.put(j, p);
-
-              // Reset j
-              if (j == cntDest - 1) {
-                j = 0;
-              }
-            }
-          }
-        });
-
-    vm0.invoke(new CacheSerializableRunnable("Run query on region") {
-
-      @Override
-      public void run2() throws CacheException {
-        // Do a put in region.
-        Region r = getCache().getRegion(name);
-
-        Query query = getCache().getQueryService()
-            .newQuery("select * from /" + name + " p where p.status = 'active'");
-
-        // Now run the query
-        SelectResults results = null;
-
-
-        for (int i = 0; i < 500; i++) {
-
-          try {
-            getCache().getLogger().fine("Querying the region");
-            results = (SelectResults) query.execute();
-          } catch (Exception e) {
-            e.printStackTrace();
-          }
-
-          for (Object obj : results) {
-            if (obj instanceof Undefined) {
-              fail("Found an undefined element" + Arrays.toString(results.toArray()));
-            }
-          }
-        }
-      }
-    });
-
-    ThreadUtils.join(asyInvk0, 1000 * 1000);
-    if (asyInvk0.exceptionOccurred()) {
-      Assert.fail("asyInvk0 failed", asyInvk0.getException());
-    }
-
-    ThreadUtils.join(asyInvk1, 1000 * 1000);
-    if (asyInvk1.exceptionOccurred()) {
-      Assert.fail("asyInvk1 failed", asyInvk1.getException());
+    String regionName = "PartionedPortfoliosPR";
+    String query = "select * from /" + regionName + " p where p.status = 'active'";
+    try {
+      vm0.invoke(() -> createRegionInVM(regionName, scope));
+      final PortfolioData[] portfolio = createPortfolioData(cnt, cntDest);
+      vm0.invoke(
+          PRQHelp.getCacheSerializableRunnableForPRPuts(regionName, portfolio, cnt, cntDest));
+      AsyncInvocation asyInvk0 =
+          vm0.invokeAsync(() -> consecutivelyCreateAndDestroyIndex(regionName));
+      AsyncInvocation asyInvk1 =
+          vm0.invokeAsync(() -> consecutivelyPutAndDestroyEntries(regionName));
+      vm0.invoke(() -> executeAndValidateQueryResults(query));
+      waitForAsyncThreadsToComplete(asyInvk0, asyInvk1);
+    } finally {
+      vm0.invoke(() -> clearIndexesAndDestroyRegion(regionName));
     }
   }
 
-  @Category(FlakyTest.class) // GEODE-1036: uses PRQueryDUnitHelper, time sensitive, async actions,
-                             // overly long joins (16+ minutes), eats exceptions (fixed 1), thread
-                             // sleeps
   @Test
-  public void testAsyncIndexInitDuringEntryDestroyAndQueryOnPR() {
+  public void testConcurrentRemoveIndexAndQueryOnPR() {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
     setCacheInVMs(vm0);
-    name = "PartionedPortfoliosPR";
-    // Create Local Region
-    vm0.invoke(
-        new CacheSerializableRunnable("Create local region with asynchronous index maintenance") {
-          @Override
-          public void run2() throws CacheException {
-            Cache cache = getCache();
-            Region partitionRegion = null;
-            try {
-              AttributesFactory attr = new AttributesFactory();
-              attr.setValueConstraint(PortfolioData.class);
-              attr.setIndexMaintenanceSynchronous(false);
-              attr.setPartitionAttributes(new PartitionAttributesFactory().create());
-              RegionFactory regionFactory = cache.createRegionFactory(attr.create());
-              partitionRegion = regionFactory.create(name);
-            } catch (IllegalStateException ex) {
-              LogWriterUtils.getLogWriter().warning("Creation caught IllegalStateException", ex);
-            }
-            assertNotNull("Region " + name + " not in cache", cache.getRegion(name));
-            assertNotNull("Region ref null", partitionRegion);
-            assertTrue("Region ref claims to be destroyed", !partitionRegion.isDestroyed());
-          }
-        });
-
-
-    final PortfolioData[] portfolio = createPortfolioData(cnt, cntDest);
-    // Putting the data into the PR's created
-    vm0.invoke(PRQHelp.getCacheSerializableRunnableForPRPuts(name, portfolio, cnt, cntDest));
-
-    AsyncInvocation asyInvk0 =
-        vm0.invokeAsync(new CacheSerializableRunnable("Create Index with Hook") {
-
-          @Override
-          public void run2() throws CacheException {
-            for (int i = 0; i < cntDest; i++) {
-              // Create Index first to go in hook.
-              Cache cache = getCache();
-              Index index = null;
-              try {
-                index = cache.getQueryService().createIndex("statusIndex", "p.status",
-                    "/" + name + " p");
-              } catch (Exception e1) {
-                e1.printStackTrace();
-                Assert.fail("Index creation failed", e1);
-              }
-              assertNotNull(index);
-
-              getCache().getQueryService().removeIndex(index);
-
-            }
-          }
-        });
-
-    // Change the value in Region
-    AsyncInvocation asyInvk1 =
-        vm0.invokeAsync(new CacheSerializableRunnable("Change value in region") {
-
-          @Override
-          public void run2() throws CacheException {
-            // Do a put in region.
-            Region r = getCache().getRegion(name);
-
-            for (int i = 0, j = 0; i < 1000; i++, j++) {
-
-              PortfolioData p = (PortfolioData) r.get(j);
-
-              getCache().getLogger().fine("Going to destroy the value" + p);
-              r.destroy(j);
-
-              Wait.pause(20);
-
-              // Put the value back again.
-              getCache().getLogger().fine("Putting the value back" + p);
-              r.put(j, p);
-
-              // Reset j
-              if (j == cntDest - 1) {
-                j = 0;
-              }
-            }
-          }
-        });
-
-    vm0.invoke(new CacheSerializableRunnable("Run query on region") {
-
-      @Override
-      public void run2() throws CacheException {
-        // Do a put in region.
-        Query query = getCache().getQueryService()
-            .newQuery("select * from /" + name + " p where p.status = 'active'");
-
-        // Now run the query
-        SelectResults results = null;
-
-
-        for (int i = 0; i < 500; i++) {
-
-          try {
-            getCache().getLogger().fine("Querying the region");
-            results = (SelectResults) query.execute();
-          } catch (Exception e) {
-            e.printStackTrace(); // TODO: eats exceptions
-          }
+    String name = "PartionedPortfoliosPR";
+    String query =
+        "select * from /" + name + " p where p.status = 'active' and p.ID > 0 and p.pk != ' ' ";
+    try {
+      vm0.invoke(() -> createRegionInVM(name, null));
+      final PortfolioData[] portfolio = createPortfolioData(cnt, cntDest);
+      vm0.invoke(PRQHelp.getCacheSerializableRunnableForPRPuts(name, portfolio, cnt, cntDest));
+      vm0.invoke(() -> createIndex(name, "statusIndex", "p.status", "/" + name + " p"));
+      vm0.invoke(() -> createIndex(name, "idIndex", "p.ID", "/" + name + " p"));
+      vm0.invoke(() -> createIndex(name, "pkidIndex", "p.pk", "/" + name + " p"));
+      vm0.invoke(() -> executeAndValidateQueryResults(query));
+    } finally {
+      vm0.invoke(() -> clearIndexesAndDestroyRegion(name));
+    }
 
-          for (Object obj : results) {
-            if (obj instanceof Undefined) {
-              fail("Found an undefined element" + Arrays.toString(results.toArray()));
-            }
-          }
-        }
-      }
-    });
+  }
 
+  private void waitForAsyncThreadsToComplete(AsyncInvocation asyInvk0, AsyncInvocation asyInvk1) {
     ThreadUtils.join(asyInvk0, 1000 * 1000); // TODO: this is way too long: 16.67 minutes!
     if (asyInvk0.exceptionOccurred()) {
-      Assert.fail("asyInvk0 failed", asyInvk0.getException());
+      logger.error("Asynchronous thread to create and destroy index failed",
+          asyInvk0.getException());
+      fail();
     }
 
     ThreadUtils.join(asyInvk1, 1000 * 1000); // TODO: this is way too long: 16.67 minutes!
     if (asyInvk1.exceptionOccurred()) {
-      Assert.fail("asyInvk1 failed", asyInvk1.getException());
+      logger.error("Asychronous thread to create and destroy region entry failed with exception",
+          asyInvk1.getException());
+      fail();
     }
   }
 
-  @Test
-  public void testConcurrentRemoveIndexAndQueryOnPR() {
-    Host host = Host.getHost(0);
-    VM vm0 = host.getVM(0);
-    setCacheInVMs(vm0);
-    name = "PartionedPortfoliosPR";
-    // Create Local Region
-    vm0.invoke(
-        new CacheSerializableRunnable("Create local region with asynchronous index maintenance") {
-          @Override
-          public void run2() throws CacheException {
-            Cache cache = getCache();
-            Region partitionRegion = null;
-            try {
-              AttributesFactory attr = new AttributesFactory();
-              attr.setValueConstraint(PortfolioData.class);
-              attr.setIndexMaintenanceSynchronous(false);
-              attr.setPartitionAttributes(new PartitionAttributesFactory().create());
-              RegionFactory regionFactory = cache.createRegionFactory(attr.create());
-              partitionRegion = regionFactory.create(name);
-            } catch (IllegalStateException ex) {
-              LogWriterUtils.getLogWriter().warning("Creation caught IllegalStateException", ex);
-            }
-            assertNotNull("Region " + name + " not in cache", cache.getRegion(name));
-            assertNotNull("Region ref null", partitionRegion);
-            assertTrue("Region ref claims to be destroyed", !partitionRegion.isDestroyed());
-          }
-        });
-
-
-    final PortfolioData[] portfolio = createPortfolioData(cnt, cntDest);
-    // Putting the data into the PR's created
-    vm0.invoke(PRQHelp.getCacheSerializableRunnableForPRPuts(name, portfolio, cnt, cntDest));
-
-    vm0.invoke(new CacheSerializableRunnable("Create Index") {
-
-      @Override
-      public void run2() throws CacheException {
-
-        // Create Index first to go in hook.
-        Cache cache = getCache();
-        Index sindex = null;
-        Index iindex = null;
-        Index pkindex = null;
-        try {
-          sindex =
-              cache.getQueryService().createIndex("statusIndex", "p.status", "/" + name + " p");
-          iindex = cache.getQueryService().createIndex("idIndex", "p.ID", "/" + name + " p");
-          pkindex = cache.getQueryService().createIndex("pkidIndex", "p.pk", "/" + name + " p");
-        } catch (Exception e1) {
-          e1.printStackTrace();
-          fail("Index creation failed");
-        }
-        assertNotNull(sindex);
-        assertNotNull(iindex);
-        assertNotNull(pkindex);
+  private void createRegionInVM(String regionName, Scope scope) {
+    Cache cache = getCache();
+    Region partitionRegion = null;
+    try {
+      RegionFactory regionFactory = cache.createRegionFactory();
+      if (scope != null) {
+        partitionRegion = regionFactory.setValueConstraint(PortfolioData.class)
+            .setIndexMaintenanceSynchronous(false).setScope(scope).create(regionName);
+      } else {
+        partitionRegion = regionFactory.setValueConstraint(PortfolioData.class)
+            .setIndexMaintenanceSynchronous(false).setDataPolicy(DataPolicy.PARTITION)
+            .create(regionName);
       }
-    });
-
-    vm0.invoke(new CacheSerializableRunnable("Run query on region") {
-
-      @Override
-      public void run2() throws CacheException {
-        // Do a put in region.
-        Query query = getCache().getQueryService().newQuery("select * from /" + name
-            + " p where p.status = 'active' and p.ID > 0 and p.pk != ' ' ");
-        // Now run the query
-        SelectResults results = null;
-
-        for (int i = 0; i < 10; i++) {
-
-          try {
-            getCache().getLogger().fine("Querying the region with " + query);
-            results = (SelectResults) query.execute();
-          } catch (Exception e) {
-            Assert.fail("Query: " + query + " execution failed with exception", e);
-          }
+    } catch (IllegalStateException ex) {
+      logger.warn("Creation caught IllegalStateException", ex);
+    }
+    assertNotNull("Region " + regionName + " not in cache", cache.getRegion(regionName));
+    assertNotNull("Region ref null", partitionRegion);
+    assertTrue("Region ref claims to be destroyed", !partitionRegion.isDestroyed());
+  }
 
-          for (Object obj : results) {
-            if (obj instanceof Undefined) {
-              fail("Found an undefined element" + Arrays.toString(results.toArray()));
-            }
-          }
-        }
+  private void consecutivelyCreateAndDestroyIndex(String regionName) {
+    for (int i = 0; i < cntDest; i++) {
+      // Create Index first to go in hook.
+      Cache cache = getCache();
+      Index index = null;
+      try {
+        index =
+            cache.getQueryService().createIndex("statusIndex", "p.status", "/" + regionName + " p");
+      } catch (Exception e1) {
+        logger.error("Index creation failed", e1);
+        fail();
       }
-    });
+      Region region = cache.getRegion(regionName);
+      Awaitility.await().atMost(30, TimeUnit.SECONDS)
+          .until(() -> assertNotNull(cache.getQueryService().getIndex(region, "statusIndex")));
+      getCache().getQueryService().removeIndex(index);
+      Awaitility.await().atMost(60, TimeUnit.SECONDS).until(
+          () -> assertEquals(null, getCache().getQueryService().getIndex(region, "statusIndex")));
+    }
+  }
 
-    vm0.invoke(new CacheSerializableRunnable("Create Index") {
+  private void consecutivelyPutAndDestroyEntries(String regionName) {
+    Region r = getCache().getRegion(regionName);
 
-      @Override
-      public void run2() throws CacheException {
+    for (int i = 0, j = 0; i < 500; i++, j++) {
 
-        Region r = getCache().getRegion(name);
+      PortfolioData p = (PortfolioData) r.get(j);
+      logger.debug("Going to destroy the value" + p);
+      r.destroy(j);
+      final int key = j;
+      Awaitility.await().atMost(20, TimeUnit.SECONDS).until(() -> assertEquals(null, r.get(key)));
 
-        // Create Index first to go in hook.
-        getCache().getQueryService().removeIndexes(r);
+      // Put the value back again.
+      getCache().getLogger().fine("Putting the value back" + p);
+      r.put(j, p);
 
+      // Reset j
+      if (j == cntDest - 1) {
+        j = 0;
+      }
+    }
+  }
 
+  private void executeAndValidateQueryResults(String queryString) {
+    Query query = getCache().getQueryService().newQuery(queryString);
+    // Now run the query
+    SelectResults results = null;
+    for (int i = 0; i < 50; i++) {
+      try {
+        logger.debug("Querying the region");
+        results = (SelectResults) query.execute();
+      } catch (Exception e) {
+        e.printStackTrace();
+        logger.error("Query Execution failed", e);
+        fail();
+      }
+      for (Object obj : results) {
+        if (obj instanceof Undefined) {
+          fail("Found an undefined element" + Arrays.toString(results.toArray()));
+        }
       }
-    });
+    }
+  }
+
+  private void clearIndexesAndDestroyRegion(String regionName) {
+    Region region = getCache().getRegion(regionName);
+    if (region != null) {
+      getCache().getQueryService().removeIndexes(region);
+      region.destroyRegion();
+    }
+  }
 
+  private void createIndex(String regionName, String indexName, String expression,
+      String regionPath) {
+    try {
+      getCache().getQueryService().createIndex(indexName, expression, regionPath);
+    } catch (Exception e) {
+      e.printStackTrace();
+      logger.error("Exception while creating index : " + indexName, e);
+      fail();
+    }
+    Region region = getCache().getRegion(regionName);
+    Awaitility.await().atMost(30, TimeUnit.SECONDS)
+        .until(() -> assertNotNull(getCache().getQueryService().getIndex(region, indexName)));
   }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> CI failure: InitializeIndexEntryDestroyQueryDUnitTest.testAsyncIndexInitDuringEntryDestroyAndQueryOnPR
> ------------------------------------------------------------------------------------------------------
>
>                 Key: GEODE-1036
>                 URL: https://issues.apache.org/jira/browse/GEODE-1036
>             Project: Geode
>          Issue Type: Bug
>          Components: querying
>            Reporter: Jason Huynh
>            Assignee: Jason Huynh
>              Labels: CI, Flaky
>
> java.lang.AssertionError: asyInvk0 failed
> 	at com.gemstone.gemfire.test.dunit.Assert.fail(Assert.java:64)
> 	at com.gemstone.gemfire.cache.query.internal.index.InitializeIndexEntryDestroyQueryDUnitTest.testAsyncIndexInitDuringEntryDestroyAndQueryOnPR(InitializeIndexEntryDestroyQueryDUnitTest.java:338)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:497)
> 	at junit.framework.TestCase.runTest(TestCase.java:176)
> 	at junit.framework.TestCase.runBare(TestCase.java:141)
> 	at junit.framework.TestResult$1.protect(TestResult.java:122)
> 	at junit.framework.TestResult.runProtected(TestResult.java:142)
> 	at junit.framework.TestResult.run(TestResult.java:125)
> 	at junit.framework.TestCase.run(TestCase.java:129)
> 	at junit.framework.TestSuite.runTest(TestSuite.java:252)
> 	at junit.framework.TestSuite.run(TestSuite.java:247)
> 	at org.junit.internal.runners.JUnit38ClassRunner.run(JUnit38ClassRunner.java:86)
> 	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecuter.runTestClass(JUnitTestClassExecuter.java:105)
> 	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecuter.execute(JUnitTestClassExecuter.java:56)
> 	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassProcessor.processTestClass(JUnitTestClassProcessor.java:64)
> 	at org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:50)
> 	at sun.reflect.GeneratedMethodAccessor4.invoke(Unknown Source)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:497)
> 	at org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
> 	at org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
> 	at org.gradle.messaging.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
> 	at org.gradle.messaging.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
> 	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
> 	at org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:106)
> 	at sun.reflect.GeneratedMethodAccessor3.invoke(Unknown Source)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:497)
> 	at org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
> 	at org.gradle.messaging.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
> 	at org.gradle.messaging.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:360)
> 	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:54)
> 	at org.gradle.internal.concurrent.StoppableExecutorImpl$1.run(StoppableExecutorImpl.java:40)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: junit.framework.AssertionFailedError: Index creation failed
> 	at junit.framework.Assert.fail(Assert.java:57)
> 	at junit.framework.TestCase.fail(TestCase.java:227)
> 	at com.gemstone.gemfire.cache.query.internal.index.InitializeIndexEntryDestroyQueryDUnitTest$6.run2(InitializeIndexEntryDestroyQueryDUnitTest.java:259)
> 	at com.gemstone.gemfire.cache30.CacheSerializableRunnable.run(CacheSerializableRunnable.java:57)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:497)
> 	at hydra.MethExecutor.executeObject(MethExecutor.java:268)
> 	at com.gemstone.gemfire.test.dunit.standalone.RemoteDUnitVM.executeMethodOnObject(RemoteDUnitVM.java:84)
> 	at sun.reflect.GeneratedMethodAccessor8.invoke(Unknown Source)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:497)
> 	at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:323)
> 	at sun.rmi.transport.Transport$1.run(Transport.java:200)
> 	at sun.rmi.transport.Transport$1.run(Transport.java:197)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at sun.rmi.transport.Transport.serviceCall(Transport.java:196)
> 	at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:568)
> 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:826)
> 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.lambda$run$95(TCPTransport.java:683)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:682)
> 	... 3 more



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)