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 2016/05/20 21:17:20 UTC

[1/5] incubator-geode git commit: GEODE-988: Added wait for events to arrive on stopped CQ before it is stopped. The test stops the CQ and verifies no events are recieved in stopped state; it was doing that by checking operations/events that happend befo

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-835 7e559224a -> 52f21b0fc


GEODE-988: Added wait for events to arrive on stopped CQ before it is stopped.
The test stops the CQ and verifies no events are recieved in stopped state; it was doing that by checking operations/events that happend before stop and not counting the changes happened during stop.
In a slower environment, it could so happen that the CQ may not have recieved all the events before it is stopped, thus causing the validation to fail.


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/425581c6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/425581c6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/425581c6

Branch: refs/heads/feature/GEODE-835
Commit: 425581c650aeb472f38b814ca1617786e8c4fc9c
Parents: 8a3c351
Author: Anil <ag...@pivotal.io>
Authored: Thu May 19 14:02:54 2016 -0700
Committer: Anil <ag...@pivotal.io>
Committed: Fri May 20 10:50:11 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/query/cq/dunit/CqPerfUsingPoolDUnitTest.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/425581c6/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqPerfUsingPoolDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqPerfUsingPoolDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqPerfUsingPoolDUnitTest.java
index 130f924..2a91abb 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqPerfUsingPoolDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqPerfUsingPoolDUnitTest.java
@@ -429,7 +429,6 @@ public class CqPerfUsingPoolDUnitTest extends CacheTestCase {
     // Create.
     cqDUnitTest.createValues(server, cqDUnitTest.regions[0], size);
     cqDUnitTest.waitForCreated(client, "testMatchingCqs_0", CqQueryUsingPoolDUnitTest.KEY+size);
-    cqDUnitTest.waitForCreated(client, "testMatchingCqs_3", CqQueryUsingPoolDUnitTest.KEY+size);
 
     // Close one of the CQ.
     cqDUnitTest.closeCQ(client, "testMatchingCqs_0");      
@@ -438,6 +437,7 @@ public class CqPerfUsingPoolDUnitTest extends CacheTestCase {
     // Update.
     cqDUnitTest.createValues(server, cqDUnitTest.regions[0], size);
     cqDUnitTest.waitForUpdated(client, "testMatchingCqs_3", CqQueryUsingPoolDUnitTest.KEY+size);
+    cqDUnitTest.waitForUpdated(client, "testMatchingCqs_1", CqQueryUsingPoolDUnitTest.KEY+size);
     
     // Stop one of the CQ.
     cqDUnitTest.stopCQ(client, "testMatchingCqs_1");      


[4/5] incubator-geode git commit: GEODE-1415 Disable logging of banners in distributedTests

Posted by kl...@apache.org.
GEODE-1415 Disable logging of banners in distributedTests

Banners are normally logged during creation of an InternalDistributedSystem.
In DUnit tests the output is filled with these banners and they all have
the same information, so they are just clutter.

This change-set disables banners in non-locator VMs launched by
DUnitLauncher/ProcessManager.  Precheckin testing shows that the
amount of output for geode-core/distributedTests alone used to be over 1gb
and is now less than 500mb.


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/c674e357
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/c674e357
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/c674e357

Branch: refs/heads/feature/GEODE-835
Commit: c674e35717c4f358ccfdfa04d65b0c3a65885342
Parents: 0014fd4
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri May 20 13:57:39 2016 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri May 20 13:57:39 2016 -0700

----------------------------------------------------------------------
 .../internal/AdminDistributedSystemImpl.java    |  13 +-
 .../com/gemstone/gemfire/internal/Banner.java   | 252 +++++++++----------
 .../internal/logging/LogWriterFactory.java      |  14 +-
 ...rRemoteWithCustomLoggingIntegrationTest.java |   0
 .../DistributedSystemLogFileJUnitTest.java      |   2 +-
 .../test/dunit/standalone/DUnitLauncher.java    |   4 +
 .../test/dunit/standalone/ProcessManager.java   |   4 +
 7 files changed, 151 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
index 21a9ae3..2522a0f 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/admin/internal/AdminDistributedSystemImpl.java
@@ -27,6 +27,7 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.FutureTask;
 
+import com.gemstone.gemfire.distributed.internal.*;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.CancelException;
@@ -56,10 +57,6 @@ import com.gemstone.gemfire.admin.SystemMembershipListener;
 import com.gemstone.gemfire.cache.persistence.PersistentID;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.FutureCancelledException;
-import com.gemstone.gemfire.distributed.internal.DM;
-import com.gemstone.gemfire.distributed.internal.DistributionConfig;
-import com.gemstone.gemfire.distributed.internal.DistributionManager;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
 import com.gemstone.gemfire.internal.Banner;
@@ -215,8 +212,12 @@ implements com.gemstone.gemfire.admin.AdminDistributedSystem,
     } else {      
       // LOG: create LogWriterLogger
       this.logWriter = LogWriterFactory.createLogWriterLogger(false, false, this.config.createLogConfig(), false);
-      // LOG: changed statement from config to info
-      this.logWriter.info(Banner.getString(null));
+      if (!Boolean.getBoolean(InternalLocator.INHIBIT_DM_BANNER)) {
+        // LOG: changed statement from config to info
+        this.logWriter.info(Banner.getString(null));
+      } else {
+        logger.debug("skipping banner - " + InternalLocator.INHIBIT_DM_BANNER + " is set to true");
+      }
       // Set this log writer in DistributedSystemConfigImpl
       this.config.setInternalLogWriter(this.logWriter);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/main/java/com/gemstone/gemfire/internal/Banner.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/Banner.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/Banner.java
old mode 100644
new mode 100755
index 4b01e54..b6791ca
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/Banner.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/Banner.java
@@ -31,142 +31,142 @@ import java.util.*;
  */
 public class Banner {
 
-    private Banner() {
-	// everything is static so don't allow instance creation
+  private Banner() {
+    // everything is static so don't allow instance creation
+  }
+
+  private static void prettyPrintPath(String path, PrintWriter out) {
+    if (path != null) {
+      StringTokenizer st =
+          new StringTokenizer(path, System.getProperty("path.separator"));
+      while (st.hasMoreTokens()) {
+        out.println("  " + st.nextToken());
+      }
+    }
+  }
+  /**
+   * Print information about this process to the specified stream.
+   * @param args possibly null list of command line arguments
+   */
+  private static void print(PrintWriter out, String args[]) {
+    Map sp = new TreeMap((Properties)System.getProperties().clone()); // fix for 46822
+    int processId = -1;
+    final String SEPERATOR = "---------------------------------------------------------------------------";
+    try {
+      processId = OSProcess.getId();
+    }
+    catch (VirtualMachineError err) {
+      SystemFailure.initiateFailure(err);
+      // If this ever returns, rethrow the error.  We're poisoned
+      // now, so don't let this thread continue.
+      throw err;
     }
-    
-    private static void prettyPrintPath(String path, PrintWriter out) {
-	if (path != null) {
-	    StringTokenizer st =
-		new StringTokenizer(path, System.getProperty("path.separator"));
-	    while (st.hasMoreTokens()) {
-		out.println("  " + st.nextToken());
-	    }
-	}
+    catch (Throwable t) {
+      // Whenever you catch Error or Throwable, you must also
+      // catch VirtualMachineError (see above).  However, there is
+      // _still_ a possibility that you are dealing with a cascading
+      // error condition, so you also need to check to see if the JVM
+      // is still usable:
+      SystemFailure.checkFailure();
     }
-    /** 
-     * Print information about this process to the specified stream.
-     * @param args possibly null list of command line arguments
-     */
-    private static void print(PrintWriter out, String args[]) {
-        Map sp = new TreeMap((Properties)System.getProperties().clone()); // fix for 46822
-	int processId = -1;
-	final String SEPERATOR = "---------------------------------------------------------------------------";
-	try {
-	    processId = OSProcess.getId();
-	} 
-	catch (VirtualMachineError err) {
-	  SystemFailure.initiateFailure(err);
-	  // If this ever returns, rethrow the error.  We're poisoned
-	  // now, so don't let this thread continue.
-	  throw err;
-	}
-	catch (Throwable t) {
-	     // Whenever you catch Error or Throwable, you must also
-	     // catch VirtualMachineError (see above).  However, there is
-	     // _still_ a possibility that you are dealing with a cascading
-	     // error condition, so you also need to check to see if the JVM
-	     // is still usable:
-	     SystemFailure.checkFailure();
-	}
-	out.println();
+    out.println();
 
-        final String productName = GemFireVersion.getProductName();
-        
-	out.println(SEPERATOR);
+    final String productName = GemFireVersion.getProductName();
 
-        out.println("  ");
-        out.println("  Licensed to the Apache Software Foundation (ASF) under one or more");
-        out.println("  contributor license agreements.  See the NOTICE file distributed with this");
-        out.println("  work for additional information regarding copyright ownership.");
-        out.println("   ");
-        out.println("  The ASF licenses this file to You under the Apache License, Version 2.0");
-        out.println("  (the \"License\"); you may not use this file except in compliance with the");
-        out.println("  License.  You may obtain a copy of the License at");
-        out.println("  ");
-        out.println("  http://www.apache.org/licenses/LICENSE-2.0");
-        out.println("  ");
-        out.println("  Unless required by applicable law or agreed to in writing, software");
-        out.println("  distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT");
-        out.println("  WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.  See the");
-        out.println("  License for the specific language governing permissions and limitations");
-        out.println("  under the License.");
-        out.println("  ");
+    out.println(SEPERATOR);
 
-        out.println(SEPERATOR);
+    out.println("  ");
+    out.println("  Licensed to the Apache Software Foundation (ASF) under one or more");
+    out.println("  contributor license agreements.  See the NOTICE file distributed with this");
+    out.println("  work for additional information regarding copyright ownership.");
+    out.println("   ");
+    out.println("  The ASF licenses this file to You under the Apache License, Version 2.0");
+    out.println("  (the \"License\"); you may not use this file except in compliance with the");
+    out.println("  License.  You may obtain a copy of the License at");
+    out.println("  ");
+    out.println("  http://www.apache.org/licenses/LICENSE-2.0");
+    out.println("  ");
+    out.println("  Unless required by applicable law or agreed to in writing, software");
+    out.println("  distributed under the License is distributed on an \"AS IS\" BASIS, WITHOUT");
+    out.println("  WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.  See the");
+    out.println("  License for the specific language governing permissions and limitations");
+    out.println("  under the License.");
+    out.println("  ");
 
-	GemFireVersion.print(out);
+    out.println(SEPERATOR);
 
-	out.println("Process ID: " + processId);
-	out.println("User: " + sp.get("user.name"));
-        sp.remove("user.name");
-        sp.remove("os.name");
-        sp.remove("os.arch");
-	out.println("Current dir: " + sp.get("user.dir"));
-        sp.remove("user.dir");
-	out.println("Home dir: " + sp.get("user.home"));
-        sp.remove("user.home");
-        List<String> allArgs = new ArrayList<String>();
-        {
-          RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
-          if (runtimeBean != null) {
-            allArgs.addAll(runtimeBean.getInputArguments()); // fixes  45353
-          }
-        }
-        
-	if (args != null && args.length != 0) {
-	    for (int i=0; i < args.length; i++) {
-	      allArgs.add(args[i]);
-	    }
-	}
-	if (!allArgs.isEmpty()) {
-	  out.println("Command Line Parameters:");
-	  for (String arg: allArgs) {
-	    out.println("  " + arg);
-	  }
-	}
-	out.println("Class Path:");
-	prettyPrintPath((String)sp.get("java.class.path"), out);
-        sp.remove("java.class.path");
-	out.println("Library Path:");
-	prettyPrintPath((String)sp.get("java.library.path"), out);
-        sp.remove("java.library.path");
+    GemFireVersion.print(out);
+
+    out.println("Process ID: " + processId);
+    out.println("User: " + sp.get("user.name"));
+    sp.remove("user.name");
+    sp.remove("os.name");
+    sp.remove("os.arch");
+    out.println("Current dir: " + sp.get("user.dir"));
+    sp.remove("user.dir");
+    out.println("Home dir: " + sp.get("user.home"));
+    sp.remove("user.home");
+    List<String> allArgs = new ArrayList<String>();
+    {
+      RuntimeMXBean runtimeBean = ManagementFactory.getRuntimeMXBean();
+      if (runtimeBean != null) {
+        allArgs.addAll(runtimeBean.getInputArguments()); // fixes  45353
+      }
+    }
 
-        if (Boolean.getBoolean("gemfire.disableSystemPropertyLogging")) {
-          out.println("System property logging disabled.");
+    if (args != null && args.length != 0) {
+      for (int i=0; i < args.length; i++) {
+        allArgs.add(args[i]);
+      }
+    }
+    if (!allArgs.isEmpty()) {
+      out.println("Command Line Parameters:");
+      for (String arg: allArgs) {
+        out.println("  " + arg);
+      }
+    }
+    out.println("Class Path:");
+    prettyPrintPath((String)sp.get("java.class.path"), out);
+    sp.remove("java.class.path");
+    out.println("Library Path:");
+    prettyPrintPath((String)sp.get("java.library.path"), out);
+    sp.remove("java.library.path");
+
+    if (Boolean.getBoolean("gemfire.disableSystemPropertyLogging")) {
+      out.println("System property logging disabled.");
+    } else {
+      out.println("System Properties:");
+      Iterator it = sp.entrySet().iterator();
+      while (it.hasNext()) {
+        Map.Entry me = (Map.Entry)it.next();
+        String key = me.getKey().toString();
+        // SW: Filter out the security properties since they may contain
+        // sensitive information.
+        if (!key.startsWith(DistributionConfig.GEMFIRE_PREFIX
+            + DistributionConfig.SECURITY_PREFIX_NAME)
+            && !key.startsWith(DistributionConfigImpl.SECURITY_SYSTEM_PREFIX
+            + DistributionConfig.SECURITY_PREFIX_NAME)
+            && !key.toLowerCase().contains("password") /* bug 45381 */) {
+          out.println("    " + key + " = " + me.getValue());
         } else {
-	out.println("System Properties:");
-        Iterator it = sp.entrySet().iterator();
-        while (it.hasNext()) {
-          Map.Entry me = (Map.Entry)it.next();
-          String key = me.getKey().toString();
-          // SW: Filter out the security properties since they may contain
-          // sensitive information.
-          if (!key.startsWith(DistributionConfig.GEMFIRE_PREFIX
-              + DistributionConfig.SECURITY_PREFIX_NAME)
-              && !key.startsWith(DistributionConfigImpl.SECURITY_SYSTEM_PREFIX
-                  + DistributionConfig.SECURITY_PREFIX_NAME)
-              && !key.toLowerCase().contains("password") /* bug 45381 */) {
-            out.println("    " + key + " = " + me.getValue());
-          } else {
-            out.println("    " + key + " = " + "********");
-          }
+          out.println("    " + key + " = " + "********");
         }
-        out.println("Log4J 2 Configuration:");
-        out.println("    " + LogService.getConfigInformation());
-        }
-	out.println(SEPERATOR);
+      }
+      out.println("Log4J 2 Configuration:");
+      out.println("    " + LogService.getConfigInformation());
     }
+    out.println(SEPERATOR);
+  }
 
-    /**
-     * Return a string containing the banner information.
-     * @param args possibly null list of command line arguments
-     */
-    public static String getString(String args[]) {
-	StringWriter sw = new StringWriter();
-	PrintWriter pw = new PrintWriter(sw);
-	print(pw, args);
-	pw.close();
-	return sw.toString();
-    }
+  /**
+   * Return a string containing the banner information.
+   * @param args possibly null list of command line arguments
+   */
+  public static String getString(String args[]) {
+    StringWriter sw = new StringWriter();
+    PrintWriter pw = new PrintWriter(sw);
+    print(pw, args);
+    pw.close();
+    return sw.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LogWriterFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LogWriterFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LogWriterFactory.java
old mode 100644
new mode 100755
index d9352c0..63ed4b0
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LogWriterFactory.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/logging/LogWriterFactory.java
@@ -85,11 +85,15 @@ public class LogWriterFactory {
     }
     
     // log the banner
-    if (InternalDistributedSystem.getReconnectAttemptCounter() == 0 // avoid filling up logs during auto-reconnect
-        && !isSecure && (!isLoner /* do this on a loner to fix bug 35602 */
-        || !Boolean.getBoolean(InternalLocator.INHIBIT_DM_BANNER))) {
-      // LOG:CONFIG:
-      logger.info(LogMarker.CONFIG, Banner.getString(null));
+    if (!Boolean.getBoolean(InternalLocator.INHIBIT_DM_BANNER)) {
+      if (InternalDistributedSystem.getReconnectAttemptCounter() == 0 // avoid filling up logs during auto-reconnect
+          && !isSecure //&& !isLoner /* do this on a loner to fix bug 35602 */
+          ) {
+        // LOG:CONFIG:
+        logger.info(LogMarker.CONFIG, Banner.getString(null));
+      }
+    } else {
+      logger.debug("skipping banner - " + InternalLocator.INHIBIT_DM_BANNER + " is set to true");
     }
 
     // log the config

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteWithCustomLoggingIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteWithCustomLoggingIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteWithCustomLoggingIntegrationTest.java
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
old mode 100644
new mode 100755
index 69df873..061ea97
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
@@ -127,7 +127,7 @@ public class DistributedSystemLogFileJUnitTest {
     // assert not empty
     FileInputStream fis = new FileInputStream(logFile);
     try {
-      assertTrue(fis.available() > 0);
+      assertTrue("log file is empty: " + logFile.getAbsoluteFile(), fis.available() > 0);
     } finally {
       fis.close();
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
index 0c600ab..8249996 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/DUnitLauncher.java
@@ -38,6 +38,7 @@ import java.rmi.server.UnicastRemoteObject;
 import java.util.List;
 import java.util.Properties;
 
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.core.LoggerContext;
@@ -160,6 +161,9 @@ public class DUnitLauncher {
     master = new Master(registry, processManager);
     registry.bind(MASTER_PARAM, master);
 
+    // inhibit banners to make logs smaller
+    System.setProperty(InternalLocator.INHIBIT_DM_BANNER, "true");
+
     Runtime.getRuntime().addShutdownHook(new Thread() {
       public void run() {
 //        System.out.println("shutting down DUnit JVMs");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c674e357/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
old mode 100644
new mode 100755
index 5dbdfd6..1e78bff
--- a/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/test/dunit/standalone/ProcessManager.java
@@ -33,6 +33,7 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
 import org.apache.commons.io.FileUtils;
 
 import com.gemstone.gemfire.internal.FileUtil;
@@ -169,6 +170,9 @@ public class ProcessManager {
     cmds.add("-D" + DUnitLauncher.RMI_PORT_PARAM + "=" + namingPort);
     cmds.add("-D" + DUnitLauncher.VM_NUM_PARAM + "=" + vmNum);
     cmds.add("-D" + DUnitLauncher.WORKSPACE_DIR_PARAM + "=" + new File(".").getAbsolutePath());
+    if (vmNum >= 0) { // let the locator print a banner
+      cmds.add("-D" + InternalLocator.INHIBIT_DM_BANNER + "=true");
+    }
     cmds.add("-DlogLevel=" + DUnitLauncher.LOG_LEVEL);
     if (DUnitLauncher.LOG4J!=null) {
       cmds.add("-Dlog4j.configurationFile="+DUnitLauncher.LOG4J);


[5/5] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-835

Posted by kl...@apache.org.
Merge remote-tracking branch 'origin/develop' into feature/GEODE-835


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/52f21b0f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/52f21b0f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/52f21b0f

Branch: refs/heads/feature/GEODE-835
Commit: 52f21b0fce4fd259c05b30c1b8efac04588819d4
Parents: 7e55922 c674e35
Author: Kirk Lund <kl...@pivotal.io>
Authored: Fri May 20 14:17:11 2016 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Fri May 20 14:17:11 2016 -0700

----------------------------------------------------------------------
 .../internal/AdminDistributedSystemImpl.java    |  13 +-
 .../com/gemstone/gemfire/internal/Banner.java   | 252 +++++++++----------
 .../internal/logging/LogWriterFactory.java      |  14 +-
 .../gemfire/cache30/ReconnectDUnitTest.java     |  21 +-
 ...rRemoteWithCustomLoggingIntegrationTest.java |   0
 .../DistributedSystemLogFileJUnitTest.java      |   2 +-
 .../test/dunit/standalone/DUnitLauncher.java    |   4 +
 .../test/dunit/standalone/ProcessManager.java   |   4 +
 .../cq/dunit/CqPerfUsingPoolDUnitTest.java      |   2 +-
 .../LuceneIndexCreationIntegrationTest.java     |  38 +--
 ...IndexCreationPersistenceIntegrationTest.java |  53 ++++
 .../LuceneIndexMaintenanceIntegrationTest.java  | 119 +++++++++
 .../cache/lucene/LuceneIntegrationTest.java     |   8 +
 .../LuceneServiceImplIntegrationTest.java       | 205 +--------------
 14 files changed, 375 insertions(+), 360 deletions(-)
----------------------------------------------------------------------



[2/5] incubator-geode git commit: GEODE-11: Added transaction tests for Lucene indexes

Posted by kl...@apache.org.
GEODE-11: Added transaction tests for Lucene indexes

* Moved tests from LuceneServiceImplIntegrationTest to better locations
* Added LuceneIndexMaintenanceIntegrationTest


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/afc5dde7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/afc5dde7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/afc5dde7

Branch: refs/heads/feature/GEODE-835
Commit: afc5dde7a62ba55c214c40b97538ba262a8fe509
Parents: 425581c
Author: Jason Huynh <hu...@gmail.com>
Authored: Tue May 17 14:33:38 2016 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Fri May 20 13:47:18 2016 -0700

----------------------------------------------------------------------
 .../LuceneIndexCreationIntegrationTest.java     |  38 ++--
 ...IndexCreationPersistenceIntegrationTest.java |  53 +++++
 .../LuceneIndexMaintenanceIntegrationTest.java  | 119 +++++++++++
 .../cache/lucene/LuceneIntegrationTest.java     |   8 +
 .../LuceneServiceImplIntegrationTest.java       | 205 +------------------
 5 files changed, 214 insertions(+), 209 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afc5dde7/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
index 4c28938..d1cd8ac 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationIntegrationTest.java
@@ -19,9 +19,12 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.RegionShortcut.*;
 import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static junitparams.JUnitParamsRunner.*;
 import static org.junit.Assert.*;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
@@ -42,11 +45,15 @@ import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.jayway.awaitility.Awaitility;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.core.KeywordTokenizer;
+import org.apache.lucene.queryparser.classic.ParseException;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 
 import junitparams.JUnitParamsRunner;
@@ -65,6 +72,8 @@ import junitparams.Parameters;
 @RunWith(JUnitParamsRunner.class)
 public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
 
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
 
   @Test
   public void shouldCreateIndexWriterWithAnalyzersWhenSettingPerFieldAnalyzers()
@@ -136,20 +145,6 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
-  public void shouldCreateInternalRegionsForIndex() {
-    createIndex("field1", "field2");
-
-    // Create partitioned region
-    createRegion();
-
-    verifyInternalRegions(region -> {
-      region.isInternalRegion();
-      assertNotNull(region.getAttributes().getPartitionAttributes().getColocatedWith());
-      cache.rootRegions().contains(region);
-    });
-  }
-
-  @Test
   public void shouldUseFixedPartitionsForInternalRegions() {
     createIndex("text");
 
@@ -167,13 +162,26 @@ public class LuceneIndexCreationIntegrationTest extends LuceneIntegrationTest {
     });
   }
 
+  @Test(expected = IllegalStateException.class)
+  public void cannotCreateLuceneIndexAfterRegionHasBeenCreated() throws IOException, ParseException {
+    createRegion();
+    createIndex("field1", "field2", "field3");
+  }
+
+  @Test
+  public void cannotCreateLuceneIndexForReplicateRegion() throws IOException, ParseException {
+    expectedException.expect(UnsupportedOperationException.class);
+    expectedException.expectMessage("Lucene indexes on replicated regions are not supported");
+    createIndex("field1", "field2", "field3");
+    this.cache.createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
+  }
 
   private void verifyInternalRegions(Consumer<LocalRegion> verify) {
     LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
   }
 
   private Region createRegion() {
-    return this.cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
+    return createRegion(REGION_NAME, RegionShortcut.PARTITION);
   }
 
   private void createIndex(String ... fieldNames) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afc5dde7/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
index d6bf116..c43b592 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
@@ -18,7 +18,9 @@
  */
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.RegionShortcut.*;
 import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static junitparams.JUnitParamsRunner.$;
 import static org.junit.Assert.*;
 
 import java.io.File;
@@ -130,7 +132,58 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
     assertEquals(1, query.search().size());
   }
 
+  @Test
+  @Parameters(method = "getRegionShortcuts")
+  public void shouldHandleMultipleIndexes(RegionShortcut shortcut) throws ParseException {
+    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME+"_1", REGION_NAME, "field1");
+    LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME+"_2", REGION_NAME, "field2");
+    Region region = cache.createRegionFactory(shortcut).create(REGION_NAME);
+    region.put("key1", new TestObject());
+    verifyQueryResultSize(INDEX_NAME+"_1", REGION_NAME, "field1:world", 1);
+    verifyQueryResultSize(INDEX_NAME+"_2", REGION_NAME, "field2:field", 1);
+  }
+
+  @Test
+  @Parameters(method = "getRegionShortcuts")
+  public void shouldCreateInternalRegionsForIndex(RegionShortcut shortcut) {
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1", "field2");
+
+    // Create partitioned region
+    createRegion(REGION_NAME, shortcut);
+
+    verifyInternalRegions(region -> {
+      region.isInternalRegion();
+      assertTrue(region.isInternalRegion());
+
+      assertNotNull(region.getAttributes().getPartitionAttributes().getColocatedWith());
+      cache.rootRegions().contains(region);
+      assertFalse(cache.rootRegions().contains(region));
+    });
+  }
+
+  private void verifyQueryResultSize(String indexName, String regionName, String queryString, int size) throws ParseException {
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(indexName, regionName, queryString);
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
+      assertEquals(size, query.search().size());
+    });
+  }
+
   private void verifyInternalRegions(Consumer<LocalRegion> verify) {
     LuceneTestUtilities.verifyInternalRegions(luceneService, cache, verify);
   }
+
+
+  private static final Object[] getRegionShortcuts() {
+    return $(
+      new Object[] { PARTITION },
+      new Object[] { PARTITION_REDUNDANT },
+      new Object[] { PARTITION_PERSISTENT },
+      new Object[] { PARTITION_REDUNDANT_PERSISTENT },
+      new Object[] { PARTITION_OVERFLOW },
+      new Object[] { PARTITION_REDUNDANT_OVERFLOW },
+      new Object[] { PARTITION_PERSISTENT_OVERFLOW },
+      new Object[] { PARTITION_REDUNDANT_PERSISTENT_OVERFLOW }
+    );
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afc5dde7/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
new file mode 100644
index 0000000..07780ca
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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 com.gemstone.gemfire.cache.lucene;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static org.junit.Assert.*;
+
+import java.io.Serializable;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(IntegrationTest.class)
+public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest {
+
+  private static int WAIT_FOR_FLUSH_TIME = 10000;
+
+  @Test
+  public void indexIsNotUpdatedIfTransactionHasNotCommittedYet() throws Exception {
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+
+    Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
+    region.put("object-1", new TestObject("title 1", "hello world"));
+    region.put("object-2", new TestObject("title 2", "this will not match"));
+    region.put("object-3", new TestObject("title 3", "hello world"));
+    region.put("object-4", new TestObject("hello world", "hello world"));
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "description:\"hello world\"");
+    LuceneQueryResults<Integer, TestObject> results = query.search();
+    assertEquals(3, results.size());
+
+    //begin transaction
+    cache.getCacheTransactionManager().begin();
+    region.put("object-1", new TestObject("title 1", "updated"));
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+    assertEquals(3, query.search().size());
+  }
+
+  @Test
+  public void indexIsUpdatedAfterTransactionHasCommitted() throws Exception {
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+
+    Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
+    region.put("object-1", new TestObject("title 1", "hello world"));
+    region.put("object-2", new TestObject("title 2", "this will not match"));
+    region.put("object-3", new TestObject("title 3", "hello world"));
+    region.put("object-4", new TestObject("hello world", "hello world"));
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "description:\"hello world\"");
+    LuceneQueryResults<Integer, TestObject> results = query.search();
+    assertEquals(3, results.size());
+
+    cache.getCacheTransactionManager().begin();
+    region.put("object-1", new TestObject("title 1", "updated"));
+    cache.getCacheTransactionManager().commit();
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+
+    assertEquals(2, query.search().size());
+  }
+
+  @Test
+  public void indexIsNotUpdatedAfterTransactionRollback() throws Exception {
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, "title", "description");
+
+    Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION);
+    region.put("object-1", new TestObject("title 1", "hello world"));
+    region.put("object-2", new TestObject("title 2", "this will not match"));
+    region.put("object-3", new TestObject("title 3", "hello world"));
+    region.put("object-4", new TestObject("hello world", "hello world"));
+
+    LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+    LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "description:\"hello world\"");
+    LuceneQueryResults<Integer, TestObject> results = query.search();
+    assertEquals(3, results.size());
+
+    cache.getCacheTransactionManager().begin();
+    region.put("object-1", new TestObject("title 1", "updated"));
+    cache.getCacheTransactionManager().rollback();
+    index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
+
+    assertEquals(3, query.search().size());
+  }
+
+  private static class TestObject implements Serializable {
+
+    String title;
+    String description;
+
+    public TestObject(String title, String description) {
+      this.title = title;
+      this.description = description;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afc5dde7/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
index c302460..6d8e370 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIntegrationTest.java
@@ -19,10 +19,14 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
+
 import java.io.File;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
@@ -60,4 +64,8 @@ public class LuceneIntegrationTest {
     cf.set("locators", "");
     return cf;
   }
+
+  protected Region createRegion(String regionName, RegionShortcut shortcut) {
+    return this.cache.createRegionFactory(shortcut).create(regionName);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/afc5dde7/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplIntegrationTest.java
index fa3392c..a078910 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneServiceImplIntegrationTest.java
@@ -16,34 +16,13 @@
  * specific language governing permissions and limitations
  * under the License.
  */
+
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import static org.junit.Assert.*;
 
-import java.io.IOException;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-
-import com.gemstone.gemfire.cache.Region;
-
-import org.apache.logging.log4j.Logger;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.core.KeywordAnalyzer;
-import org.apache.lucene.analysis.miscellaneous.PerFieldAnalyzerWrapper;
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.queryparser.classic.ParseException;
-import org.junit.After;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.rules.ExpectedException;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
 import com.gemstone.gemfire.cache.execute.Function;
@@ -51,31 +30,26 @@ import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
-import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
-import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
+
 @Category(IntegrationTest.class)
 public class LuceneServiceImplIntegrationTest {
 
   Cache cache;
   ClientCache clientCache;
-  private LuceneIndexImpl repo;
-  private HeterogeneousLuceneSerializer mapper;
-  private StandardAnalyzer analyzer = new StandardAnalyzer();
-  private IndexWriter writer;
   LuceneServiceImpl service = null;
-  private static final Logger logger = LogService.getLogger();
 
   @Rule
   public ExpectedException expectedException = ExpectedException.none();
 
   @Test
-  public void luceneServiceProviderGetShouldAcceptClientCacheAsAParameter(){
+  public void luceneServiceProviderGetShouldAcceptClientCacheAsAParameter() {
     clientCache = getClientCache();
     LuceneService luceneService = LuceneServiceProvider.get(clientCache);
     assertNotNull(luceneService);
@@ -107,7 +81,7 @@ public class LuceneServiceImplIntegrationTest {
       cache.close();
       cache = null;
     }
-    if (null != clientCache  && !clientCache.isClosed()) {
+    if (null != clientCache && !clientCache.isClosed()) {
       clientCache.close();
       clientCache = null;
     }
@@ -117,7 +91,7 @@ public class LuceneServiceImplIntegrationTest {
     if (null == clientCache) {
       clientCache = new ClientCacheFactory().set("mcast-port", "0").create();
     }
-    else{
+    else {
       return clientCache;
     }
     return clientCache;
@@ -130,163 +104,6 @@ public class LuceneServiceImplIntegrationTest {
     return cache;
   }
 
-  private LuceneService getService() {
-    if (null == cache) {
-      getCache();
-    }
-    if (null == service) {
-      service = (LuceneServiceImpl)LuceneServiceProvider.get(cache);
-    }
-    return service;
-  }
-
-
-  private Region createRegion(String regionName, RegionShortcut shortcut) {
-    return cache.createRegionFactory(shortcut).create(regionName);
-  }
-
-  private LocalRegion createPR(String regionName, boolean isSubRegion) {
-    if (isSubRegion) {
-      LocalRegion root = (LocalRegion) cache.createRegionFactory(RegionShortcut.PARTITION).create("root");
-      LocalRegion region = (LocalRegion) cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT).
-        createSubregion(root, regionName);
-      return region;
-    }
-    else {
-      LocalRegion region = (LocalRegion) createRegion(regionName, RegionShortcut.PARTITION_PERSISTENT);
-      return region;
-    }
-  }
-
-  private LocalRegion createRR(String regionName, boolean isSubRegion) {
-    if (isSubRegion) {
-
-      LocalRegion root = (LocalRegion) cache.createRegionFactory(RegionShortcut.REPLICATE).create("root");
-      LocalRegion region = (LocalRegion) cache.createRegionFactory(RegionShortcut.REPLICATE_PERSISTENT).
-        createSubregion(root, regionName);
-      return region;
-    }
-    else {
-      LocalRegion region = (LocalRegion) createRegion(regionName, RegionShortcut.REPLICATE_PERSISTENT);
-      return region;
-    }
-  }
-
-  @Test(expected = IllegalStateException.class)
-  public void cannotCreateLuceneIndexAfterRegionHasBeenCreated() throws IOException, ParseException {
-    getService();
-
-    LocalRegion userRegion = createPR("PR1", false);
-    service.createIndex("index1", "PR1", "field1", "field2", "field3");
-  }
-
-  @Test
-  public void canCreateLuceneIndexForPRWithAnalyzer() throws IOException, ParseException {
-    getService();
-    StandardAnalyzer sa = new StandardAnalyzer();
-    KeywordAnalyzer ka = new KeywordAnalyzer();
-    Map<String, Analyzer> analyzerPerField = new HashMap<String, Analyzer>();
-    analyzerPerField.put("field1", ka);
-    analyzerPerField.put("field2", sa);
-    analyzerPerField.put("field3", sa);
-    //  field2 and field3 will use StandardAnalyzer
-    PerFieldAnalyzerWrapper analyzer2 = new PerFieldAnalyzerWrapper(sa, analyzerPerField);
-
-    service.createIndex("index1", "PR1", analyzerPerField);
-    createPR("PR1", false);
-    LuceneIndexImpl index1 = (LuceneIndexImpl) service.getIndex("index1", "PR1");
-    assertTrue(index1 instanceof LuceneIndexForPartitionedRegion);
-    LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion) index1;
-    assertEquals("index1", index1.getName());
-    assertEquals("/PR1", index1.getRegionPath());
-    String[] fields1 = index1.getFieldNames();
-    assertEquals(3, fields1.length);
-    Analyzer analyzer = index1PR.getAnalyzer();
-    assertTrue(analyzer instanceof PerFieldAnalyzerWrapper);
-    RepositoryManager RepositoryManager = index1PR.getRepositoryManager();
-    assertTrue(RepositoryManager != null);
-
-    final String fileRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".files";
-    final String chunkRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".chunks";
-    PartitionedRegion filePR = (PartitionedRegion) cache.getRegion(fileRegionName);
-    PartitionedRegion chunkPR = (PartitionedRegion) cache.getRegion(chunkRegionName);
-    assertTrue(filePR != null);
-    assertTrue(chunkPR != null);
-  }
-
-  @Test
-  public void cannotCreateLuceneIndexForReplicateRegion() throws IOException, ParseException {
-    expectedException.expect(UnsupportedOperationException.class);
-    expectedException.expectMessage("Lucene indexes on replicated regions are not supported");
-    getService();
-    service.createIndex("index1", "RR1", "field1", "field2", "field3");
-    createRR("RR1", false);
-  }
-
-  @Test
-  public void canCreateIndexForAllNonProxyPartitionRegionTypes() {
-    for (RegionShortcut shortcut : RegionShortcut.values()) {
-      String sname = shortcut.name().toLowerCase();
-      if (sname.contains("partition") && !sname.contains("proxy")) {
-        canCreateLuceneIndexForPRType(shortcut);
-        //Destroying cache and service for now because aeq's are not completely being cleaned up correctly after
-        // being destroyed.  Instead we should close the aeq and clean up any regions associated with this lucene
-        //index but only after aeq destroy works properly
-        destroyCache();
-        destroyService();
-      }
-    }
-  }
-
-  private void canCreateLuceneIndexForPRType(RegionShortcut regionShortcut) {
-    getService();
-    service.createIndex("index1", "PR1", "field1", "field2", "field3");
-    Region region = null;
-    AsyncEventQueueImpl aeq = null;
-    try {
-      region = createRegion("PR1", regionShortcut);
-      LuceneIndexImpl index1 = (LuceneIndexImpl) service.getIndex("index1", "PR1");
-      assertTrue(index1 instanceof LuceneIndexForPartitionedRegion);
-      LuceneIndexForPartitionedRegion index1PR = (LuceneIndexForPartitionedRegion) index1;
-      assertEquals("index1", index1.getName());
-      assertEquals("/PR1", index1.getRegionPath());
-      String[] fields1 = index1.getFieldNames();
-      assertEquals(3, fields1.length);
-      Analyzer analyzer = index1PR.getAnalyzer();
-      assertTrue(analyzer instanceof StandardAnalyzer);
-      RepositoryManager RepositoryManager = index1PR.getRepositoryManager();
-      assertTrue(RepositoryManager != null);
-
-      final String fileRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".files";
-      final String chunkRegionName = LuceneServiceImpl.getUniqueIndexName("index1", "/PR1") + ".chunks";
-      PartitionedRegion filePR = (PartitionedRegion) cache.getRegion(fileRegionName);
-      PartitionedRegion chunkPR = (PartitionedRegion) cache.getRegion(chunkRegionName);
-      assertTrue(filePR != null);
-      assertTrue(chunkPR != null);
+}
 
-      String aeqId = LuceneServiceImpl.getUniqueIndexName(index1.getName(), index1.getRegionPath());
-      aeq = (AsyncEventQueueImpl) cache.getAsyncEventQueue(aeqId);
-      assertTrue(aeq != null);
 
-      //Make sure our queue doesn't show up in the list of async event queues
-      assertEquals(Collections.emptySet(), cache.getAsyncEventQueues());
-    }
-    finally {
-      String aeqId = LuceneServiceImpl.getUniqueIndexName("index1", "PR1");
-      PartitionedRegion chunkRegion = (PartitionedRegion) cache.getRegion(aeqId + ".chunks");
-      if (chunkRegion != null) {
-        chunkRegion.destroyRegion();
-      }
-      PartitionedRegion fileRegion = (PartitionedRegion) cache.getRegion(aeqId + ".files");
-      if (fileRegion != null) {
-        fileRegion.destroyRegion();
-      }
-      ((GemFireCacheImpl) cache).removeAsyncEventQueue(aeq);
-      if (aeq != null) {
-        aeq.destroy();
-      }
-      region.destroyRegion();
-    }
-  }
-
-}


[3/5] incubator-geode git commit: GEODE-1407: CI Failure: ReconnectDUnitTest.testReconnectALocator

Posted by kl...@apache.org.
GEODE-1407: CI Failure: ReconnectDUnitTest.testReconnectALocator

Added a FlakyTest annotation to the test.  Modified the flaky runnable
to use fail() instead of logging symptoms so that we can tell more about
why it's flaky.

The test never fails when run by itself so it may be an infection from
another test.


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/0014fd44
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/0014fd44
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/0014fd44

Branch: refs/heads/feature/GEODE-835
Commit: 0014fd444332ba05b55c95ce1d220613dbe9d580
Parents: afc5dde
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri May 20 13:51:40 2016 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri May 20 13:55:03 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache30/ReconnectDUnitTest.java     | 21 +++++++++-----------
 1 file changed, 9 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0014fd44/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
index 6c63def..948e75a 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache30/ReconnectDUnitTest.java
@@ -36,6 +36,8 @@ import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlGenerator;
 import com.gemstone.gemfire.test.dunit.*;
+import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+import org.junit.experimental.categories.Category;
 
 import java.io.File;
 import java.io.FileWriter;
@@ -452,7 +454,8 @@ public class ReconnectDUnitTest extends CacheTestCase
     });
   }
   
-  
+
+  @Category(FlakyTest.class) // GEODE-1407
   public void testReconnectALocator() throws Exception {
     Host host = Host.getHost(0);
     VM vm0 = host.getVM(0);
@@ -523,8 +526,8 @@ public class ReconnectDUnitTest extends CacheTestCase
       newdm = waitForReconnect(vm0);
       assertGfshWaitingThreadAlive(vm0);
 
-      boolean running = (Boolean)vm0.invoke(new SerializableCallable("check for running locator") {
-        public Object call() {
+      vm0.invoke(new SerializableRunnable("check for running locator") {
+        public void run() {
           WaitCriterion wc = new WaitCriterion() {
             public boolean done() {
               return Locator.getLocator() != null;
@@ -535,20 +538,14 @@ public class ReconnectDUnitTest extends CacheTestCase
           };
           Wait.waitForCriterion(wc, 30000, 1000, false);
           if (Locator.getLocator() == null) {
-            LogWriterUtils.getLogWriter().error("expected to find a running locator but getLocator() returns null");
-            return false;
+            fail("expected to find a running locator but getLocator() returns null");
           }
           if (((InternalLocator)Locator.getLocator()).isStopped()) {
-            LogWriterUtils.getLogWriter().error("found a stopped locator");
-            return false;
+            fail("found a stopped locator");
           }
-          return true;
         }
       });
-      if (!running) {
-        fail("expected the restarted member to be hosting a running locator");
-      }
-      
+
       assertNotSame("expected a reconnect to occur in the locator", dm, newdm);
 
       // the log should have been opened and appended with a new view