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 2017/06/21 23:15:56 UTC

[01/25] geode git commit: GEODE-3071: Provide capability to parallelize distributedTests [Forced Update!]

Repository: geode
Updated Branches:
  refs/heads/feature/GEODE-3062-2 1320328dc -> 6384d8738 (forced update)


GEODE-3071: Provide capability to parallelize distributedTests

Herewith the ability to leverage Gradle's parallel test execution
capability to run dunits in parallel. This is combined with launching
tests in Docker containers to provide process, network and filesystem
isolation. Depending on the size of your system, this can speed up
running the distributedTest task 2-5 times.

The capability is enabled by launching gradle with '-PparallelDunit'

Tunables, enabled as gradle parametrs (-P option) are:

- dunitDockerImage: The docker image which will be used to launch
  tests. The image must have the JAVA_HOME environment variable set. The
  image must be pulled locally before starting the tests.
- dunitParallelForks: The number of parallel docker containers to be
  launched.
- dunitDockerUser: The docker user which will run the tests. Because of
  the way that the containers map the build directory into them, the
  test artifacts, will be written with this user id. By default this is
  'root'.


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 588c3ed8af4b9a1b46737c2faebafa589a396e12
Parents: a4d790c
Author: Jens Deppe <jd...@pivotal.io>
Authored: Wed Jun 14 08:03:22 2017 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Wed Jun 14 08:03:22 2017 -0700

----------------------------------------------------------------------
 build.gradle         |  2 ++
 gradle.properties    |  7 ++++
 gradle/docker.gradle | 83 +++++++++++++++++++++++++++++++++++++++++++++++
 3 files changed, 92 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/588c3ed8/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index ec6b920..57372b3 100755
--- a/build.gradle
+++ b/build.gradle
@@ -26,6 +26,7 @@ buildscript {
     classpath 'org.sonarsource.scanner.gradle:sonarqube-gradle-plugin:2.0.1'
     classpath "com.diffplug.gradle.spotless:spotless:2.2.0"
     classpath "me.champeau.gradle:jmh-gradle-plugin:0.3.1"
+    classpath "com.pedjak.gradle.plugins:dockerized-test:0.4.2"
   }
 }
 
@@ -82,6 +83,7 @@ apply from: "${scriptDir}/code-analysis.gradle"
 apply from: "${scriptDir}/sonar.gradle"
 apply from: "${scriptDir}/ide.gradle"
 apply from: "${scriptDir}/rat.gradle"
+apply from: "${scriptDir}/docker.gradle"
 
 subprojects {
   // Make sure clean task for rootProject runs last

http://git-wip-us.apache.org/repos/asf/geode/blob/588c3ed8/gradle.properties
----------------------------------------------------------------------
diff --git a/gradle.properties b/gradle.properties
index ca79a38..9462295 100755
--- a/gradle.properties
+++ b/gradle.properties
@@ -49,3 +49,10 @@ buildRoot=
 
 # We want signing to be on by default. Signing requires GPG to be set up.
 nexusSignArchives = true
+
+# Control how many concurrent dunit (using docker) tests will be run
+dunitParallelForks = 8
+# This is the name of the Docker image for running parallel dunits
+dunitDockerImage = openjdk:8
+# Docker user for parallel dunit tests
+dunitDockerUser = root

http://git-wip-us.apache.org/repos/asf/geode/blob/588c3ed8/gradle/docker.gradle
----------------------------------------------------------------------
diff --git a/gradle/docker.gradle b/gradle/docker.gradle
new file mode 100644
index 0000000..7a74c5d
--- /dev/null
+++ b/gradle/docker.gradle
@@ -0,0 +1,83 @@
+/*
+ * Configuration for running (dunit) tests in parallel in Docker containers.
+ * The container used must hava JAVA_HOME set in it's environment and must
+ * have 'java' defined on the path. For example, the relevant Dockerfile
+ * content could be:
+ * 
+ *   ENV JAVA_HOME=/opt/jdk1.8.0_u101
+ *   ENV PATH=$PATH:$JAVA_HOME/bin
+ *
+ * In addition, the container must have docker installed.
+ *
+ * The plugin can be activated with the Gradle property 'parallelDunit'.
+ * Additional properties that can be set are:
+ *
+ *  dunitDockerImage   - The docker image used for running parallel dunits. The
+ *                       default image is 'openjdk:8'. The image is required to
+ *                       have 'JAVA_HOME' set as an environment variable.
+ *  dunitParallelForks - The number of parallel containers that will be
+ *                       launched. The default is 8.
+ *  dunitDockerUser    - The user used within the docker container to run tests.
+ *                       The default is 'root'.
+ */
+
+def dockerConfig = {
+  maxParallelForks = dunitParallelForks.toInteger()
+
+  docker {
+    // base image for creating docker containers that execute the tests
+    image = dunitDockerImage
+
+    // volumes mounted to the containers
+    // in a form: host_dir : container_dir
+    def pwd = System.getenv('PWD')
+    def gradleHome = System.getenv('GRADLE_USER_HOME') ?: "${System.getenv('HOME')}/.gradle"
+    volumes = ["${gradleHome}":gradleHome]
+
+    volumes << ["${pwd}":pwd]
+
+    // specify the user for starting Gradle test worker within the container.
+    user = dunitDockerUser
+
+    argsInspect = { List args ->
+      def javaHomeIdx = 0
+      def i = args.iterator()
+      def j = 0
+      while (i.hasNext()) {
+        if (i.next() == '-e') {
+          def x = i.next()
+          j++
+          if (x.startsWith('JAVA_HOME')) {
+            javaHomeIdx = j
+          }
+        }
+        j++
+      }
+
+      // Remove JAVA_HOME env variable - it might not be the same as the container needs
+      if (javaHomeIdx > 0) {
+        args.removeAt(javaHomeIdx-1)
+        args.removeAt(javaHomeIdx-1)
+      }
+
+      // Infer the index of this invocation
+      def matcher = (args[args.size - 1] =~ /.*Executor (\d*).*/)
+
+      args[3] = args[3] + matcher[0][1]
+      def workdir = new File(args[3])
+      println "dockerize: making ${workdir}"
+      workdir.mkdirs()
+      // println args
+
+      args
+    }
+  }
+}
+
+subprojects {
+  apply plugin: 'com.github.pedjak.dockerized-test'
+
+  if (project.hasProperty('parallelDunit')) {
+    distributedTest.configure(dockerConfig)
+  }
+}


[25/25] geode git commit: Revert "Revert "GEODE-3062: create new SecurityService after receiving cluster config""

Posted by kl...@apache.org.
Revert "Revert "GEODE-3062: create new SecurityService after receiving cluster config""

This reverts commit a79d2cc1621bd8c36531519df5e1edbb0faabf96.


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 6384d8738da9ae73cb37e525052009df20983f7f
Parents: b7f5391
Author: Kirk Lund <kl...@apache.org>
Authored: Thu Jun 15 10:04:02 2017 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Wed Jun 21 16:14:59 2017 -0700

----------------------------------------------------------------------
 .../internal/InternalDistributedSystem.java     |  4 ++
 .../cache/ClusterConfigurationLoader.java       |  7 ++-
 .../geode/internal/cache/GemFireCacheImpl.java  | 47 ++++++++++---------
 .../security/IntegratedSecurityService.java     | 40 +++++++++-------
 .../security/LegacySecurityService.java         | 15 +-----
 .../internal/security/SecurityService.java      | 48 ++++++++++----------
 .../security/SecurityServiceFactory.java        | 10 ++--
 .../ClusterConfigDeployJarDUnitTest.java        | 11 ++---
 .../ClusterConfigDistributionDUnitTest.java     |  5 +-
 .../ClusterConfigImportDUnitTest.java           |  9 ++--
 .../ClusterConfigStartMemberDUnitTest.java      |  4 +-
 .../ClusterConfigWithSecurityDUnitTest.java     | 32 ++++++++++---
 .../test/dunit/rules/MemberStarterRule.java     | 12 ++++-
 .../test/dunit/rules/ServerStarterRule.java     | 31 +++++++++----
 14 files changed, 155 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index 1572355..a4b3a50 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -540,6 +540,10 @@ public class InternalDistributedSystem extends DistributedSystem
     return this.securityService;
   }
 
+  public void setSecurityService(SecurityService securityService) {
+    this.securityService = securityService;
+  }
+
   /**
    * Registers a listener to the system
    * 

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java b/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
index 4f4881f..92cfd96 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/ClusterConfigurationLoader.java
@@ -152,13 +152,12 @@ public class ClusterConfigurationLoader {
 
   /***
    * Apply the gemfire properties cluster configuration on this member
-   *
-   * @param cache Cache created for this member
+   * 
    * @param response {@link ConfigurationResponse} containing the requested {@link Configuration}
    * @param config this member's config
    */
-  public static void applyClusterPropertiesConfiguration(Cache cache,
-      ConfigurationResponse response, DistributionConfig config) {
+  public static void applyClusterPropertiesConfiguration(ConfigurationResponse response,
+      DistributionConfig config) {
     if (response == null || response.getRequestedConfiguration().isEmpty()) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index fe97685..2dda38c 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -77,6 +77,7 @@ import javax.transaction.TransactionManager;
 import com.sun.jna.Native;
 import com.sun.jna.Platform;
 import org.apache.commons.lang.StringUtils;
+import org.apache.geode.internal.security.SecurityServiceFactory;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelCriterion;
@@ -323,6 +324,8 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
   private static final Pattern DOUBLE_BACKSLASH = Pattern.compile("\\\\");
 
+  private volatile ConfigurationResponse configurationResponse;
+
   /** To test MAX_QUERY_EXECUTION_TIME option. */
   public int testMaxQueryExecutionTime = -1;
 
@@ -809,7 +812,16 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
       this.system = system;
       this.dm = this.system.getDistributionManager();
 
-      this.securityService = this.system.getSecurityService();
+      this.configurationResponse = requestSharedConfiguration();
+
+      // apply the cluster's properties configuration and initialize security using that
+      // configuration
+      ClusterConfigurationLoader.applyClusterPropertiesConfiguration(this.configurationResponse,
+          this.system.getConfig());
+
+      this.securityService =
+          SecurityServiceFactory.create(this.system.getConfig().getSecurityProps(), cacheConfig);
+      this.system.setSecurityService(this.securityService);
 
       if (!this.isClient && PoolManager.getAll().isEmpty()) {
         // We only support management on members of a distributed system
@@ -1035,17 +1047,6 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     }
   }
 
-  private void deployJarsReceivedFromClusterConfiguration(ConfigurationResponse response) {
-    try {
-      ClusterConfigurationLoader.deployJarsReceivedFromClusterConfiguration(this, response);
-    } catch (IOException | ClassNotFoundException e) {
-      throw new GemFireConfigException(
-          LocalizedStrings.GemFireCache_EXCEPTION_OCCURRED_WHILE_DEPLOYING_JARS_FROM_SHARED_CONDFIGURATION
-              .toLocalizedString(),
-          e);
-    }
-  }
-
   /**
    * When called, clusterProps and serverProps and key could not be null
    */
@@ -1150,15 +1151,18 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
       listener.cacheCreated(this);
     }
 
+    // set ClassPathLoader and then deploy cluster config jars
     ClassPathLoader.setLatestToDefault(this.system.getConfig().getDeployWorkingDir());
 
-    // request and check cluster configuration
-    ConfigurationResponse configurationResponse = requestSharedConfiguration();
-    deployJarsReceivedFromClusterConfiguration(configurationResponse);
-
-    // apply the cluster's properties configuration and initialize security using that configuration
-    ClusterConfigurationLoader.applyClusterPropertiesConfiguration(this, configurationResponse,
-        this.system.getConfig());
+    try {
+      ClusterConfigurationLoader.deployJarsReceivedFromClusterConfiguration(this,
+          this.configurationResponse);
+    } catch (IOException | ClassNotFoundException e) {
+      throw new GemFireConfigException(
+          LocalizedStrings.GemFireCache_EXCEPTION_OCCURRED_WHILE_DEPLOYING_JARS_FROM_SHARED_CONDFIGURATION
+              .toLocalizedString(),
+          e);
+    }
 
     SystemMemberCacheEventProcessor.send(this, Operation.CACHE_CREATE);
     this.resourceAdvisor.initializationGate();
@@ -1182,11 +1186,11 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
     boolean completedCacheXml = false;
     try {
-      if (configurationResponse == null) {
+      if (this.configurationResponse == null) {
         // Deploy all the jars from the deploy working dir.
         ClassPathLoader.getLatest().getJarDeployer().loadPreviouslyDeployedJarsFromDisk();
       }
-      ClusterConfigurationLoader.applyClusterXmlConfiguration(this, configurationResponse,
+      ClusterConfigurationLoader.applyClusterXmlConfiguration(this, this.configurationResponse,
           this.system.getConfig());
       initializeDeclarativeCache();
       completedCacheXml = true;
@@ -1199,6 +1203,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
           // I don't want init to throw an exception that came from the close.
           // I want it to throw the original exception that came from initializeDeclarativeCache.
         }
+        this.configurationResponse = null;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java b/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java
index 171cfb7..323ab67 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/IntegratedSecurityService.java
@@ -14,22 +14,8 @@
  */
 package org.apache.geode.internal.security;
 
-import java.io.IOException;
-import java.security.AccessController;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.Callable;
-
 import org.apache.commons.lang.SerializationException;
 import org.apache.commons.lang.StringUtils;
-import org.apache.logging.log4j.Logger;
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.ShiroException;
-import org.apache.shiro.subject.Subject;
-import org.apache.shiro.subject.support.SubjectThreadState;
-import org.apache.shiro.util.ThreadContext;
-import org.apache.shiro.util.ThreadState;
-
 import org.apache.geode.GemFireIOException;
 import org.apache.geode.internal.cache.EntryEventImpl;
 import org.apache.geode.internal.logging.LogService;
@@ -46,6 +32,20 @@ import org.apache.geode.security.ResourcePermission;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
 import org.apache.geode.security.ResourcePermission.Target;
+import org.apache.geode.security.SecurityManager;
+import org.apache.logging.log4j.Logger;
+import org.apache.shiro.SecurityUtils;
+import org.apache.shiro.ShiroException;
+import org.apache.shiro.subject.Subject;
+import org.apache.shiro.subject.support.SubjectThreadState;
+import org.apache.shiro.util.ThreadContext;
+import org.apache.shiro.util.ThreadState;
+
+import java.io.IOException;
+import java.security.AccessController;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Callable;
 
 /**
  * Security service with SecurityManager and an optional PostProcessor.
@@ -54,7 +54,7 @@ public class IntegratedSecurityService implements SecurityService {
   private static Logger logger = LogService.getLogger(LogService.SECURITY_LOGGER_NAME);
 
   private final PostProcessor postProcessor;
-  private final org.apache.geode.security.SecurityManager securityManager;
+  private final SecurityManager securityManager;
 
   /**
    * this creates a security service using a SecurityManager
@@ -72,11 +72,13 @@ public class IntegratedSecurityService implements SecurityService {
     this.postProcessor = postProcessor;
   }
 
+  @Override
   public PostProcessor getPostProcessor() {
     return this.postProcessor;
   }
 
-  public org.apache.geode.security.SecurityManager getSecurityManager() {
+  @Override
+  public SecurityManager getSecurityManager() {
     return this.securityManager;
   }
 
@@ -187,6 +189,7 @@ public class IntegratedSecurityService implements SecurityService {
     return threadState;
   }
 
+  @Override
   public void authorizeClusterManage() {
     authorize(Resource.CLUSTER, Operation.MANAGE, Target.ALL, ResourcePermission.ALL);
   }
@@ -266,14 +269,17 @@ public class IntegratedSecurityService implements SecurityService {
     authorize(Resource.DATA, Operation.READ, regionName, key);
   }
 
+  @Override
   public void authorize(Resource resource, Operation operation, Target target, String key) {
     authorize(resource, operation, target.getName(), key);
   }
 
+  @Override
   public void authorize(Resource resource, Operation operation, Target target) {
     authorize(resource, operation, target, ResourcePermission.ALL);
   }
 
+  @Override
   public void authorize(Resource resource, Operation operation, String target, String key) {
     authorize(new ResourcePermission(resource, operation, target, key));
   }
@@ -300,7 +306,7 @@ public class IntegratedSecurityService implements SecurityService {
   @Override
   public void close() {
     if (this.securityManager != null) {
-      securityManager.close();
+      this.securityManager.close();
     }
     if (this.postProcessor != null) {
       this.postProcessor.close();

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/main/java/org/apache/geode/internal/security/LegacySecurityService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/LegacySecurityService.java b/geode-core/src/main/java/org/apache/geode/internal/security/LegacySecurityService.java
index c594bf9..66b6876 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/security/LegacySecurityService.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/LegacySecurityService.java
@@ -14,17 +14,7 @@
  */
 package org.apache.geode.internal.security;
 
-import java.util.Properties;
-import java.util.concurrent.Callable;
-
 import org.apache.commons.lang.StringUtils;
-import org.apache.shiro.subject.Subject;
-import org.apache.shiro.util.ThreadState;
-
-import org.apache.geode.management.internal.security.ResourceOperation;
-import org.apache.geode.security.PostProcessor;
-import org.apache.geode.security.ResourcePermission;
-import org.apache.geode.security.SecurityManager;
 
 /**
  * implementing SecurityService when only legacy authenticators are specified
@@ -32,12 +22,11 @@ import org.apache.geode.security.SecurityManager;
 public class LegacySecurityService implements SecurityService {
 
   private final boolean hasClientAuthenticator;
-
   private final boolean hasPeerAuthenticator;
 
   LegacySecurityService() {
-    hasClientAuthenticator = false;
-    hasPeerAuthenticator = false;
+    this.hasClientAuthenticator = false;
+    this.hasPeerAuthenticator = false;
   }
 
   LegacySecurityService(final String clientAuthenticator, final String peerAuthenticator) {

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/main/java/org/apache/geode/internal/security/SecurityService.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/SecurityService.java b/geode-core/src/main/java/org/apache/geode/internal/security/SecurityService.java
index feea899..b00e0f4 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/security/SecurityService.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/SecurityService.java
@@ -14,7 +14,6 @@
  */
 package org.apache.geode.internal.security;
 
-import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.PostProcessor;
 import org.apache.geode.security.ResourcePermission;
 import org.apache.geode.security.ResourcePermission.Resource;
@@ -28,6 +27,7 @@ import java.util.Properties;
 import java.util.concurrent.Callable;
 
 public interface SecurityService {
+
   default ThreadState bindSubject(Subject subject) {
     return null;
   }
@@ -40,57 +40,57 @@ public interface SecurityService {
     return null;
   }
 
-  default void logout() {};
+  default void logout() {}
 
   default Callable associateWith(Callable callable) {
     return callable;
   }
 
-  default void authorize(Resource resource, Operation operation, String target, String key) {};
+  default void authorize(Resource resource, Operation operation, String target, String key) {}
 
-  default void authorize(Resource resource, Operation operation, Target target, String key) {};
+  default void authorize(Resource resource, Operation operation, Target target, String key) {}
 
-  default void authorize(Resource resource, Operation operation, Target target) {};
+  default void authorize(Resource resource, Operation operation, Target target) {}
 
-  default void authorizeClusterManage() {};
+  default void authorizeClusterManage() {}
 
-  default void authorizeClusterWrite() {};
+  default void authorizeClusterWrite() {}
 
-  default void authorizeClusterRead() {};
+  default void authorizeClusterRead() {}
 
-  default void authorizeDataManage() {};
+  default void authorizeDataManage() {}
 
-  default void authorizeDataWrite() {};
+  default void authorizeDataWrite() {}
 
-  default void authorizeDataRead() {};
+  default void authorizeDataRead() {}
 
-  default void authorizeDiskManage() {};
+  default void authorizeDiskManage() {}
 
-  default void authorizeGatewayManage() {};
+  default void authorizeGatewayManage() {}
 
-  default void authorizeJarManage() {};
+  default void authorizeJarManage() {}
 
-  default void authorizeQueryManage() {};
+  default void authorizeQueryManage() {}
 
-  default void authorizeRegionManage(String regionName) {};
+  default void authorizeRegionManage(String regionName) {}
 
-  default void authorizeRegionManage(String regionName, String key) {};
+  default void authorizeRegionManage(String regionName, String key) {}
 
-  default void authorizeRegionWrite(String regionName) {};
+  default void authorizeRegionWrite(String regionName) {}
 
-  default void authorizeRegionWrite(String regionName, String key) {};
+  default void authorizeRegionWrite(String regionName, String key) {}
 
-  default void authorizeRegionRead(String regionName) {};
+  default void authorizeRegionRead(String regionName) {}
 
-  default void authorizeRegionRead(String regionName, String key) {};
+  default void authorizeRegionRead(String regionName, String key) {}
 
-  default void authorize(ResourcePermission context) {};
+  default void authorize(ResourcePermission context) {}
 
-  default void close() {};
+  default void close() {}
 
   default boolean needPostProcess() {
     return false;
-  };
+  }
 
   default Object postProcess(String regionPath, Object key, Object value,
       boolean valueIsSerialized) {

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceFactory.java b/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceFactory.java
index 2e0ad95..19645b6 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/security/SecurityServiceFactory.java
@@ -18,18 +18,17 @@ import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIE
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_PEER_AUTHENTICATOR;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_SHIRO_INIT;
 
-import java.util.Properties;
-
 import org.apache.commons.lang.StringUtils;
-import org.apache.shiro.SecurityUtils;
-import org.apache.shiro.UnavailableSecurityManagerException;
-
 import org.apache.geode.internal.cache.CacheConfig;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.security.shiro.SecurityManagerProvider;
 import org.apache.geode.security.PostProcessor;
 import org.apache.geode.security.SecurityManager;
+import org.apache.shiro.SecurityUtils;
+import org.apache.shiro.UnavailableSecurityManagerException;
+
+import java.util.Properties;
 
 public class SecurityServiceFactory {
 
@@ -106,5 +105,4 @@ public class SecurityServiceFactory {
     }
   }
 
-
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java
index 3781c98..3b105a4 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDeployJarDUnitTest.java
@@ -12,13 +12,11 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.management.internal.configuration;
 
 import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
-import static org.apache.geode.test.dunit.Host.getHost;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
@@ -31,11 +29,14 @@ import org.junit.experimental.categories.Category;
 
 @Category(DistributedTest.class)
 public class ClusterConfigDeployJarDUnitTest extends ClusterConfigTestBase {
+
+  private String clusterJar;
+  private String group1Jar;
+  private String group2Jar;
+
   @Rule
   public GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule();
 
-  private String clusterJar, group1Jar, group2Jar;
-
   @Before
   public void before() throws Exception {
     clusterJar = createJarFileWithClass("Cluster", "cluster.jar", lsRule.getTempFolder().getRoot());
@@ -88,7 +89,6 @@ public class ClusterConfigDeployJarDUnitTest extends ClusterConfigTestBase {
     expectedClusterConfig.verify(locator3);
   }
 
-
   @Test
   public void testDeploy() throws Exception {
     // set up the locator/servers
@@ -134,7 +134,6 @@ public class ClusterConfigDeployJarDUnitTest extends ClusterConfigTestBase {
     expectedGroup1and2Config.verify(server3);
   }
 
-
   @Test
   public void testUndeploy() throws Exception {
     // set up the locator/servers

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDistributionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDistributionDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDistributionDUnitTest.java
index 52a1f9b..4b003a8 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDistributionDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigDistributionDUnitTest.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.management.internal.configuration;
 
 import static org.apache.commons.io.FileUtils.writeByteArrayToFile;
@@ -44,6 +43,7 @@ import java.util.Properties;
 
 @Category(DistributedTest.class)
 public class ClusterConfigDistributionDUnitTest {
+
   private static final String REPLICATE_REGION = "ReplicateRegion1";
   private static final String PARTITION_REGION = "PartitionRegion1";
   private static final String INDEX1 = "ID1";
@@ -67,7 +67,6 @@ public class ClusterConfigDistributionDUnitTest {
     lsRule.startServerVM(1, locator.getPort());
   }
 
-
   @Test
   public void testIndexAndAsyncEventQueueCommands() throws Exception {
     final String DESTROY_REGION = "regionToBeDestroyed";
@@ -88,7 +87,6 @@ public class ClusterConfigDistributionDUnitTest {
     String asyncEventQueueJarPath = createAsyncEventQueueJar();
     gfshConnector.executeAndVerifyCommand("deploy --jar=" + asyncEventQueueJarPath);
 
-
     CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_ASYNC_EVENT_QUEUE);
     csb.addOptionWithValueCheck(CliStrings.CREATE_ASYNC_EVENT_QUEUE__ID, AsyncEventQueue1);
     csb.addOptionWithValueCheck(CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER,
@@ -138,7 +136,6 @@ public class ClusterConfigDistributionDUnitTest {
     });
   }
 
-
   private String createAsyncEventQueueJar() throws IOException {
     String queueCommandsJarName = this.lsRule.getTempFolder().getRoot().getCanonicalPath()
         + File.separator + "testEndToEndSC-QueueCommands.jar";

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
index 521e084..066f882 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigImportDUnitTest.java
@@ -13,7 +13,6 @@
  * the License.
  *
  */
-
 package org.apache.geode.management.internal.configuration;
 
 import static org.apache.geode.distributed.ConfigurationProperties.GROUPS;
@@ -43,13 +42,14 @@ import java.util.zip.ZipFile;
 
 @Category(DistributedTest.class)
 public class ClusterConfigImportDUnitTest extends ClusterConfigTestBase {
-  @Rule
-  public GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule();
 
-  public static final ClusterConfig INITIAL_CONFIG = new ClusterConfig(new ConfigGroup("cluster"));
+  private static final ClusterConfig INITIAL_CONFIG = new ClusterConfig(new ConfigGroup("cluster"));
 
   private MemberVM locatorVM;
 
+  @Rule
+  public GfshShellConnectionRule gfshConnector = new GfshShellConnectionRule();
+
   @Before
   public void before() throws Exception {
     locatorVM = lsRule.startLocatorVM(0, locatorProps);
@@ -156,7 +156,6 @@ public class ClusterConfigImportDUnitTest extends ClusterConfigTestBase {
   public void testExportClusterConfig(String zipFilePath) throws Exception {
     MemberVM server1 = lsRule.startServerVM(1, serverProps, locatorVM.getPort());
 
-
     gfshConnector.executeAndVerifyCommand("create region --name=myRegion --type=REPLICATE");
 
     ConfigGroup cluster = new ConfigGroup("cluster").regions("myRegion");

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java
index 1cdda4c..c84a7c1 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigStartMemberDUnitTest.java
@@ -13,7 +13,6 @@
  * the License.
  *
  */
-
 package org.apache.geode.management.internal.configuration;
 
 import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_CONFIGURATION_DIR;
@@ -34,7 +33,8 @@ import java.util.Properties;
 
 @Category(DistributedTest.class)
 public class ClusterConfigStartMemberDUnitTest extends ClusterConfigTestBase {
-  protected MemberVM locator;
+
+  private MemberVM locator;
 
   @Before
   public void before() throws Exception {

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigWithSecurityDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigWithSecurityDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigWithSecurityDUnitTest.java
index c551ca9..bb45b0d 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigWithSecurityDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/configuration/ClusterConfigWithSecurityDUnitTest.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.management.internal.configuration;
 
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
@@ -24,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThat;
 import org.apache.commons.io.FileUtils;
 import org.apache.geode.distributed.internal.ClusterConfigurationService;
 import org.apache.geode.distributed.internal.InternalLocator;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.management.internal.cli.i18n.CliStrings;
 import org.apache.geode.management.internal.configuration.utils.ZipUtils;
 import org.apache.geode.security.SimpleTestSecurityManager;
@@ -43,7 +43,10 @@ import java.util.Properties;
 
 @Category({DistributedTest.class, SecurityTest.class})
 public class ClusterConfigWithSecurityDUnitTest {
-  public String clusterConfigZipPath;
+
+  private String clusterConfigZipPath;
+  private MemberVM locator0;
+  private Properties locatorProps;
 
   @Rule
   public LocatorServerStartupRule lsRule = new LocatorServerStartupRule();
@@ -51,9 +54,6 @@ public class ClusterConfigWithSecurityDUnitTest {
   @Rule
   public GfshShellConnectionRule connector = new GfshShellConnectionRule();
 
-  MemberVM locator0;
-  Properties locatorProps;
-
   @Before
   public void before() throws Exception {
     clusterConfigZipPath = buildSecureClusterConfigZip();
@@ -64,8 +64,8 @@ public class ClusterConfigWithSecurityDUnitTest {
   }
 
   @Test
-  @Ignore("GEODE-2315")
-  public void testSecurityPropsInheritance() throws Exception {
+  @Ignore("Fails until GEODE-2315 is implemented")
+  public void testSecurityPropsInheritanceOnLocator() throws Exception {
     locatorProps.clear();
     locatorProps.setProperty(LOCATORS, "localhost[" + locator0.getPort() + "]");
     locatorProps.setProperty("security-username", "cluster");
@@ -104,6 +104,24 @@ public class ClusterConfigWithSecurityDUnitTest {
     });
   }
 
+  @Test // fails due to GEODE-3062
+  public void testSecurityPropsInheritanceOnServer() throws Exception {
+    Properties serverProps = new Properties();
+    serverProps.setProperty(LOCATORS, "localhost[" + locator0.getPort() + "]");
+    serverProps.setProperty("security-username", "cluster");
+    serverProps.setProperty("security-password", "cluster");
+    MemberVM server = lsRule.startServerVM(1, serverProps);
+
+    // cluster config specifies a security-manager so integrated security should be enabled
+    server.invoke(() -> {
+      InternalCache cache = LocatorServerStartupRule.serverStarter.getCache();
+      Properties properties = cache.getDistributedSystem().getSecurityProperties();
+      assertThat(properties.getProperty(SECURITY_MANAGER))
+          .isEqualTo(SimpleTestSecurityManager.class.getName());
+      assertThat(cache.getSecurityService().isIntegratedSecurity()).isTrue();
+    });
+  }
+
   private String buildSecureClusterConfigZip() throws Exception {
     File clusterDir = lsRule.getTempFolder().newFolder("cluster");
     File clusterSubDir = new File(clusterDir, "cluster");

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberStarterRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberStarterRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberStarterRule.java
index e5e598e..008ced7 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberStarterRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/MemberStarterRule.java
@@ -27,6 +27,8 @@ import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 import static org.apache.geode.distributed.ConfigurationProperties.NAME;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
 
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.security.SecurityManager;
 import org.junit.rules.ExternalResource;
@@ -79,12 +81,20 @@ public abstract class MemberStarterRule<T> extends ExternalResource implements M
 
   @Override
   public void after() {
+    // invoke stopMember() first and then ds.disconnect
     stopMember();
+
+    DistributedSystem ds = InternalDistributedSystem.getConnectedInstance();
+    if (ds != null) {
+      ds.disconnect();
+    }
+
     if (oldUserDir == null) {
       System.clearProperty("user.dir");
     } else {
       System.setProperty("user.dir", oldUserDir);
     }
+
     if (temporaryFolder != null) {
       temporaryFolder.delete();
     }
@@ -116,7 +126,7 @@ public abstract class MemberStarterRule<T> extends ExternalResource implements M
     this.name = name;
     properties.setProperty(NAME, name);
     // if log-file is not already set
-    properties.putIfAbsent(LOG_FILE, new File(name + ".log").getAbsolutePath().toString());
+    properties.putIfAbsent(LOG_FILE, new File(name + ".log").getAbsolutePath());
     return (T) this;
   }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/6384d873/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
index fa3e8c0..6ea2d03 100644
--- a/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
+++ b/geode-core/src/test/java/org/apache/geode/test/dunit/rules/ServerStarterRule.java
@@ -12,7 +12,6 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.geode.test.dunit.rules;
 
 import static org.apache.geode.distributed.ConfigurationProperties.HTTP_SERVICE_BIND_ADDRESS;
@@ -22,6 +21,7 @@ import static org.apache.geode.distributed.ConfigurationProperties.START_DEV_RES
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.server.CacheServer;
+import org.apache.geode.distributed.DistributedSystem;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.AvailablePortHelper;
@@ -34,7 +34,6 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
-
 /**
  * This is a rule to start up a server in your current VM. It's useful for your Integration Tests.
  *
@@ -51,6 +50,7 @@ import java.util.Properties;
  * use {@link LocatorServerStartupRule}.
  */
 public class ServerStarterRule extends MemberStarterRule<ServerStarterRule> implements Server {
+
   private transient InternalCache cache;
   private transient CacheServer server;
   private int embeddedLocatorPort = -1;
@@ -62,7 +62,9 @@ public class ServerStarterRule extends MemberStarterRule<ServerStarterRule> impl
    * Default constructor, if used, the rule will create a temporary folder as the server's working
    * dir, and will delete it when the test is done.
    */
-  public ServerStarterRule() {}
+  public ServerStarterRule() {
+    // nothing
+  }
 
   /**
    * if constructed this way, the rule won't be deleting the workingDir after the test is done. It's
@@ -106,16 +108,26 @@ public class ServerStarterRule extends MemberStarterRule<ServerStarterRule> impl
 
   @Override
   public void stopMember() {
+    // stop CacheServer and then close cache -- cache.close() will stop any running CacheServers
+    if (server != null) {
+      try {
+        server.stop();
+      } catch (Exception e) {
+      } finally {
+        server = null;
+      }
+    }
+
     // make sure this cache is the one currently open. A server cache can be recreated due to
     // importing a new set of cluster configuration.
     cache = GemFireCacheImpl.getInstance();
     if (cache != null) {
-      cache.close();
-      cache = null;
-    }
-    if (server != null) {
-      server.stop();
-      server = null;
+      try {
+        cache.close();
+      } catch (Exception e) {
+      } finally {
+        cache = null;
+      }
     }
   }
 
@@ -125,7 +137,6 @@ public class ServerStarterRule extends MemberStarterRule<ServerStarterRule> impl
   }
 
 
-
   public ServerStarterRule withEmbeddedLocator() {
     embeddedLocatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
     properties.setProperty("start-locator", "localhost[" + embeddedLocatorPort + "]");


[21/25] geode git commit: GEODE-2861: Remove dead code

Posted by kl...@apache.org.
GEODE-2861: Remove dead code

* code removed related to shutting down DiskStoreTaskPool
* this closes #594


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 5039e62af85795e5448fa746969d001782922cc9
Parents: b517ef8
Author: Nick Reich <nr...@pivotal.io>
Authored: Fri Jun 16 15:52:00 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Wed Jun 21 08:28:06 2017 -0700

----------------------------------------------------------------------
 .../geode/internal/cache/DiskStoreImpl.java     | 26 ------------------
 .../geode/internal/cache/GemFireCacheImpl.java  | 28 --------------------
 2 files changed, 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/5039e62a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
index 5a121a8..3e97d0e 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DiskStoreImpl.java
@@ -4615,32 +4615,6 @@ public class DiskStoreImpl implements DiskStore {
     return false;
   }
 
-  private void stopDiskStoreTaskPool() {
-    if (logger.isDebugEnabled()) {
-      logger.debug("Stopping DiskStoreTaskPool");
-    }
-    shutdownPool(diskStoreTaskPool);
-
-    // Allow the delayed writes to complete
-    delayedWritePool.shutdown();
-    try {
-      delayedWritePool.awaitTermination(1, TimeUnit.SECONDS);
-    } catch (InterruptedException ignore) {
-      Thread.currentThread().interrupt();
-    }
-  }
-
-  private void shutdownPool(ThreadPoolExecutor pool) {
-    // All the regions have already been closed
-    // so this pool shouldn't be doing anything.
-    List<Runnable> l = pool.shutdownNow();
-    for (Runnable runnable : l) {
-      if (l instanceof DiskStoreTask) {
-        ((DiskStoreTask) l).taskCancelled();
-      }
-    }
-  }
-
   public void writeRVVGC(DiskRegion dr, LocalRegion region) {
     acquireReadLock(dr);
     try {

http://git-wip-us.apache.org/repos/asf/geode/blob/5039e62a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index 5eaa5a4..fe97685 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -2357,7 +2357,6 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
         if (this.queryMonitor != null) {
           this.queryMonitor.stopMonitoring();
         }
-        stopDiskStoreTaskPool();
 
       } finally {
         // NO DISTRIBUTED MESSAGING CAN BE DONE HERE!
@@ -2475,16 +2474,6 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     }
   }
 
-  /**
-   * Used to guard access to compactorPool and set to true when cache is shutdown.
-   */
-  private final AtomicBoolean diskStoreTaskSync = new AtomicBoolean(false);
-
-  /**
-   * Lazily initialized. TODO: this is always null
-   */
-  private ThreadPoolExecutor diskStoreTaskPool = null;
-
   private final ConcurrentMap<String, DiskStoreImpl> diskStores = new ConcurrentHashMap<>();
 
   private final ConcurrentMap<String, DiskStoreImpl> regionOwnedDiskStores =
@@ -2593,23 +2582,6 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     return allDiskStores;
   }
 
-  private void stopDiskStoreTaskPool() {
-    synchronized (this.diskStoreTaskSync) {
-      this.diskStoreTaskSync.set(true);
-      // All the regions have already been closed
-      // so this pool shouldn't be doing anything.
-      if (this.diskStoreTaskPool != null) {
-        List<Runnable> listOfRunnables = this.diskStoreTaskPool.shutdownNow();
-        for (Runnable runnable : listOfRunnables) {
-          // TODO: fix this for-loop and the one in DiskStoreImpl
-          if (listOfRunnables instanceof DiskStoreTask) {
-            ((DiskStoreTask) listOfRunnables).taskCancelled();
-          }
-        }
-      }
-    }
-  }
-
   private void stopServers() {
     final boolean isDebugEnabled = logger.isDebugEnabled();
     if (isDebugEnabled) {


[14/25] geode git commit: Update geode-book/README.md

Posted by kl...@apache.org.
Update geode-book/README.md


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 8db7862758cfb79d5097f1fefa4e230186668f22
Parents: c6d88f7
Author: Dave Barnes <db...@pivotal.io>
Authored: Mon Jun 19 14:07:31 2017 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Mon Jun 19 14:07:31 2017 -0700

----------------------------------------------------------------------
 geode-book/README.md | 16 +++++++++++++---
 1 file changed, 13 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/8db78627/geode-book/README.md
----------------------------------------------------------------------
diff --git a/geode-book/README.md b/geode-book/README.md
index f381b16..a542ce2 100644
--- a/geode-book/README.md
+++ b/geode-book/README.md
@@ -78,12 +78,22 @@ where `XY` is the product version of your documentation (e.g., `{geode-site}/web
 
 2. Navigate to the User Guide you have built in the Geode repository: `{geode-project-dir}/geode-book/final_app/public/docs/guide/XY`.
 
-3. Use `tar` to copy the directory in order to preserve links and other filesystem niceties. Create the tarfile in your Desktop for easy access on the retrieval side.
-  
-  To copy the directory, enter:
+3. Use `tar` to copy the directory in order to preserve links and other filesystem niceties.
+
+  a. Create the tarfile in your Desktop for easy access on the retrieval side.
 
     ```
     $ tar cvf ~/Desktop/new-guide-content.tar .
+    ```
+  b. Create the destination directory in the `geode-site` repo:
+
+    ```
+    $ mkdir -p {geode-site}/website/content/docs/guide/XY
+    ```
+
+  c. Navigate to the target directory and un-tar the userguide archive:
+
+    ```
     $ cd {geode-site}/website/content/docs/guide/XY
     $ tar xvf ~/Desktop/new-guide-content.tar
     ```


[03/25] geode git commit: GEODE_3071: Add Apache license header

Posted by kl...@apache.org.
GEODE_3071: Add Apache license header


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: b26fa518c8ab04f60697a5b61c36a1222a68b515
Parents: ee88cd2
Author: Jens Deppe <jd...@pivotal.io>
Authored: Fri Jun 16 08:05:32 2017 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Fri Jun 16 08:05:32 2017 -0700

----------------------------------------------------------------------
 gradle/docker.gradle | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/b26fa518/gradle/docker.gradle
----------------------------------------------------------------------
diff --git a/gradle/docker.gradle b/gradle/docker.gradle
index 5c2ef77..b67c073 100644
--- a/gradle/docker.gradle
+++ b/gradle/docker.gradle
@@ -1,4 +1,21 @@
 /*
+ * 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.
+ */
+
+/*
  * Configuration for running (dunit) tests in parallel in Docker containers.
  * The container used must hava JAVA_HOME set in it's environment and must
  * have 'java' defined on the path. For example, the relevant Dockerfile


[12/25] geode git commit: GEODE-2601: Updated based on feedback

Posted by kl...@apache.org.
GEODE-2601: Updated based on feedback

This closes #582


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 87c26be495de5a23290231e97826cb434277c66b
Parents: 2d3e05e
Author: YehEmily <em...@gmail.com>
Authored: Fri Jun 16 09:06:56 2017 -0700
Committer: Ken Howe <kh...@pivotal.io>
Committed: Mon Jun 19 10:05:12 2017 -0700

----------------------------------------------------------------------
 .../geode/distributed/internal/InternalDistributedSystem.java  | 3 +--
 .../org/apache/geode/distributed/internal/InternalLocator.java | 6 ++----
 2 files changed, 3 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/87c26be4/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index 4ed58af..1572355 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -2189,9 +2189,8 @@ public class InternalDistributedSystem extends DistributedSystem
    * @param resource the actual resource object.
    */
   private void notifyResourceEventListeners(ResourceEvent event, Object resource) {
-    for (Iterator<ResourceEventsListener> iter = resourceListeners.iterator(); iter.hasNext();) {
+    for (ResourceEventsListener listener : resourceListeners) {
       try {
-        ResourceEventsListener listener = iter.next();
         listener.handleEvent(event, resource);
       } catch (CancelException e) {
         // ignore

http://git-wip-us.apache.org/repos/asf/geode/blob/87c26be4/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
index 3ff27ea..6eaaec2 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
@@ -469,15 +469,13 @@ public class InternalLocator extends Locator implements ConnectListener {
         // if security-log-file then create securityLogWriterAppender
         LogWriterAppenders.getOrCreateAppender(LogWriterAppenders.Identifier.SECURITY, true, false,
             this.config, false);
-
       }
-        // do not create a LogWriterAppender for security -- let it go through to logWriterAppender
+      // do not create a LogWriterAppender for security -- let it go through to logWriterAppender
     }
 
     // LOG: create LogWriters for GemFireTracer (or use whatever was passed in)
     if (logWriter == null) {
-      logWriter = LogWriterFactory.createLogWriterLogger(false, false, this.config,
-          false);
+      logWriter = LogWriterFactory.createLogWriterLogger(false, false, this.config, false);
       if (logger.isDebugEnabled()) {
         logger.debug("LogWriter for locator is created.");
       }


[22/25] geode git commit: GEODE-1958: Working on removing PasswordUtil and all related commands, classes, etc. Keeping decrypt() method to maintain backwards compatibility.

Posted by kl...@apache.org.
GEODE-1958: Working on removing PasswordUtil and all related commands, classes, etc. Keeping decrypt() method to maintain backwards compatibility.

* this closes #578


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 43b00bd84984a7c9c28536cf4a13dc104834bf0b
Parents: 5039e62
Author: YehEmily <em...@gmail.com>
Authored: Mon Jun 12 11:42:15 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Wed Jun 21 11:35:47 2017 -0700

----------------------------------------------------------------------
 .../org/apache/geode/internal/SystemAdmin.java  | 167 ++++++++++---------
 .../geode/internal/i18n/LocalizedStrings.java   |   3 -
 .../geode/internal/net/SocketCreator.java       |  72 ++++----
 .../geode/internal/util/PasswordUtil.java       |  76 ++-------
 .../geode/management/internal/cli/Launcher.java |  19 +--
 .../internal/cli/commands/ShellCommands.java    |  67 ++++----
 .../internal/cli/i18n/CliStrings.java           |  11 +-
 .../geode/cache/util/PasswordUtilJUnitTest.java |  42 -----
 .../internal/util/PasswordUtilJUnitTest.java    |  41 +++++
 .../internal/security/TestCommand.java          |   3 +-
 10 files changed, 224 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/43b00bd8/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java b/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
index ceae486..ccc64e3 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/SystemAdmin.java
@@ -14,42 +14,80 @@
  */
 package org.apache.geode.internal;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
+import static org.apache.geode.distributed.ConfigurationProperties.START_LOCATOR;
 
-import org.apache.geode.*;
+import java.io.BufferedInputStream;
+import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.PrintStream;
+import java.io.PrintWriter;
+import java.io.RandomAccessFile;
+import java.io.StringWriter;
+import java.net.InetAddress;
+import java.net.NetworkInterface;
+import java.net.SocketException;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.net.UnknownHostException;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.zip.GZIPInputStream;
+
+import org.apache.geode.GemFireException;
+import org.apache.geode.GemFireIOException;
+import org.apache.geode.InternalGemFireException;
+import org.apache.geode.NoSystemException;
+import org.apache.geode.SystemFailure;
+import org.apache.geode.UncreatedSystemException;
+import org.apache.geode.UnstartedSystemException;
 import org.apache.geode.admin.AdminException;
 import org.apache.geode.admin.BackupStatus;
 import org.apache.geode.admin.internal.AdminDistributedSystemImpl;
 import org.apache.geode.cache.persistence.PersistentID;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.*;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionConfigImpl;
+import org.apache.geode.distributed.internal.HighPriorityAckedMessage;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
-import org.apache.geode.distributed.internal.tcpserver.*;
-import org.apache.geode.internal.statistics.StatArchiveReader;
-import org.apache.geode.internal.statistics.StatArchiveReader.ResourceInst;
-import org.apache.geode.internal.statistics.StatArchiveReader.StatValue;
+import org.apache.geode.distributed.internal.tcpserver.TcpClient;
 import org.apache.geode.internal.admin.remote.TailLogResponse;
 import org.apache.geode.internal.cache.DiskStoreImpl;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.DateFormatter;
 import org.apache.geode.internal.logging.MergeLogFiles;
 import org.apache.geode.internal.net.SocketCreator;
+import org.apache.geode.internal.statistics.StatArchiveReader;
+import org.apache.geode.internal.statistics.StatArchiveReader.ResourceInst;
+import org.apache.geode.internal.statistics.StatArchiveReader.StatValue;
 import org.apache.geode.internal.util.JavaCommandBuilder;
-import org.apache.geode.internal.util.PasswordUtil;
 import org.apache.geode.internal.util.PluckStacks;
 import org.apache.geode.internal.util.PluckStacks.ThreadStack;
 
-import java.io.*;
-import java.net.*;
-import java.text.DateFormat;
-import java.text.ParseException;
-import java.util.*;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import java.util.zip.GZIPInputStream;
-
-import static org.apache.geode.distributed.ConfigurationProperties.START_LOCATOR;
-
 /**
  * Provides static methods for various system administation tasks.
  */
@@ -163,7 +201,7 @@ public class SystemAdmin {
             String msg = tailFile(logFile, false);
             throw new GemFireIOException(
                 LocalizedStrings.SystemAdmin_START_OF_LOCATOR_FAILED_THE_END_OF_0_CONTAINED_THIS_MESSAGE_1
-                    .toLocalizedString(new Object[] {logFile, msg}),
+                    .toLocalizedString(logFile, msg),
                 null);
           } catch (IOException ignore) {
             throw new GemFireIOException(
@@ -335,13 +373,11 @@ public class SystemAdmin {
       ManagerInfo mi = ManagerInfo.loadLocatorInfo(directory);
       if (statusCode == ManagerInfo.KILLED_STATUS_CODE) {
         return LocalizedStrings.SystemAdmin_LOCATOR_IN_0_WAS_KILLED_WHILE_IT_WAS_1_LOCATOR_PROCESS_ID_WAS_2
-            .toLocalizedString(
-                new Object[] {directory, ManagerInfo.statusToString(mi.getManagerStatus()),
-                    Integer.valueOf(mi.getManagerProcessId())});
+            .toLocalizedString(directory, ManagerInfo.statusToString(mi.getManagerStatus()),
+                Integer.valueOf(mi.getManagerProcessId()));
       } else {
         return LocalizedStrings.SystemAdmin_LOCATOR_IN_0_IS_1_LOCATOR_PROCESS_ID_IS_2
-            .toLocalizedString(
-                new Object[] {directory, statusString, Integer.valueOf(mi.getManagerProcessId())});
+            .toLocalizedString(directory, statusString, Integer.valueOf(mi.getManagerProcessId()));
       }
     } catch (UnstartedSystemException ex) {
       return LocalizedStrings.SystemAdmin_LOCATOR_IN_0_IS_STOPPED.toLocalizedString(directory);
@@ -535,7 +571,7 @@ public class SystemAdmin {
       File outputFile = null;
 
       if (cmdLine.size() > 0) {
-        outputFile = new File((String) cmdLine.get(0));
+        outputFile = new File(cmdLine.get(0));
         os = new FileOutputStream(outputFile);
         ps = new PrintWriter(os);
       } else {
@@ -627,7 +663,7 @@ public class SystemAdmin {
   }
 
   public static void showDiskStoreMetadata(ArrayList<String> args) {
-    String dsName = (String) args.get(0);
+    String dsName = args.get(0);
     File[] dirs = argsToFile(args.subList(1, args.size()));
 
     try {
@@ -643,7 +679,7 @@ public class SystemAdmin {
       out.mkdirs();
     }
 
-    String dsName = (String) args.get(0);
+    String dsName = args.get(0);
     File[] dirs = argsToFile(args.subList(1, args.size()));
 
     try {
@@ -757,7 +793,7 @@ public class SystemAdmin {
       } catch (FileNotFoundException ex) {
         throw new GemFireIOException(
             LocalizedStrings.SystemAdmin_COULD_NOT_CREATE_FILE_0_FOR_OUTPUT_BECAUSE_1
-                .toLocalizedString(new Object[] {outOption, getExceptionMessage(ex)}));
+                .toLocalizedString(outOption, getExceptionMessage(ex)));
       }
     } else {
       ps = System.out;
@@ -778,7 +814,7 @@ public class SystemAdmin {
       } catch (FileNotFoundException ex) {
         throw new GemFireIOException(
             LocalizedStrings.SystemAdmin_COULD_NOT_OPEN_TO_0_FOR_READING_BECAUSE_1
-                .toLocalizedString(new Object[] {fileName, getExceptionMessage(ex)}));
+                .toLocalizedString(fileName, getExceptionMessage(ex)));
       }
       if (!quiet) {
         ps.println("  " + fileName);
@@ -1312,7 +1348,7 @@ public class SystemAdmin {
               .toLocalizedString(
                   new Object[] {"commands", "options", "usage", "configuration", "-h"}));
     } else if (topic.equalsIgnoreCase("commands")) {
-      pw.println((String) usageMap.get("gemfire") + " <command> ...");
+      pw.println(usageMap.get("gemfire") + " <command> ...");
       format(pw, (String) helpMap.get("gemfire"), "  ", 0);
       for (int i = 0; i < validCommands.length; i++) {
         pw.println((String) usageMap.get(validCommands[i]));
@@ -1385,10 +1421,10 @@ public class SystemAdmin {
 
   private final static String[] validCommands = new String[] {"version", "stats", START_LOCATOR,
       "stop-locator", "status-locator", "info-locator", "tail-locator-log", "merge-logs",
-      "encrypt-password", "revoke-missing-disk-store", "list-missing-disk-stores",
-      "validate-disk-store", "upgrade-disk-store", "compact-disk-store", "compact-all-disk-stores",
-      "modify-disk-store", "show-disk-store-metadata", "export-disk-store", "shut-down-all",
-      "backup", "print-stacks", "help"};
+      "revoke-missing-disk-store", "list-missing-disk-stores", "validate-disk-store",
+      "upgrade-disk-store", "compact-disk-store", "compact-all-disk-stores", "modify-disk-store",
+      "show-disk-store-metadata", "export-disk-store", "shut-down-all", "backup", "print-stacks",
+      "help"};
 
   protected static String[] getValidCommands() {
     return validCommands.clone();
@@ -1475,28 +1511,23 @@ public class SystemAdmin {
   protected final Map helpMap = new HashMap();
 
   protected void initHelpMap() {
-    helpMap.put("gemfire", LocalizedStrings.SystemAdmin_GEMFIRE_HELP.toLocalizedString(
-        new Object[] {join(validCommands), "-h", "-debug", "-help", "-q", "-J<vmOpt>"}));
+    helpMap.put("gemfire", LocalizedStrings.SystemAdmin_GEMFIRE_HELP
+        .toLocalizedString(join(validCommands), "-h", "-debug", "-help", "-q", "-J<vmOpt>"));
     helpMap.put("version", LocalizedStrings.SystemAdmin_VERSION_HELP.toLocalizedString());
     helpMap.put("help", LocalizedStrings.SystemAdmin_HELP_HELP.toLocalizedString());
     helpMap.put("stats",
-        LocalizedStrings.SystemAdmin_STATS_HELP_PART_A.toLocalizedString(new Object[] {"+",
-            "++", ":", ".", "-details", "-nofilter", "-archive=", "-persec", "-persample",
-            "-prunezeros"}) + "\n"
-            + LocalizedStrings.SystemAdmin_STATS_HELP_PART_B.toLocalizedString(new Object[] {
-                "-starttime", "-archive=", DateFormatter.FORMAT_STRING, "-endtime",}));
-    helpMap.put("encrypt-password",
-        LocalizedStrings.SystemAdmin_ENCRYPTS_A_PASSWORD_FOR_USE_IN_CACHE_XML_DATA_SOURCE_CONFIGURATION
-            .toLocalizedString());
+        LocalizedStrings.SystemAdmin_STATS_HELP_PART_A.toLocalizedString("+", "++", ":", ".",
+            "-details", "-nofilter", "-archive=", "-persec", "-persample", "-prunezeros") + "\n"
+            + LocalizedStrings.SystemAdmin_STATS_HELP_PART_B.toLocalizedString("-starttime",
+                "-archive=", DateFormatter.FORMAT_STRING, "-endtime"));
     helpMap.put(START_LOCATOR,
-        LocalizedStrings.SystemAdmin_START_LOCATOR_HELP.toLocalizedString(new Object[] {"-port=",
+        LocalizedStrings.SystemAdmin_START_LOCATOR_HELP.toLocalizedString("-port=",
             Integer.valueOf(DistributionLocator.DEFAULT_LOCATOR_PORT), "-address=", "-dir=",
-            "-properties=", "-peer=", "-server=", "-hostname-for-clients=", "-D", "-X"}));
-    helpMap.put("stop-locator",
-        LocalizedStrings.SystemAdmin_STOP_LOCATOR_HELP.toLocalizedString(new Object[] {"-port=",
-            Integer.valueOf(DistributionLocator.DEFAULT_LOCATOR_PORT), "-address=", "-dir="}));
+            "-properties=", "-peer=", "-server=", "-hostname-for-clients=", "-D", "-X"));
+    helpMap.put("stop-locator", LocalizedStrings.SystemAdmin_STOP_LOCATOR_HELP.toLocalizedString(
+        "-port=", Integer.valueOf(DistributionLocator.DEFAULT_LOCATOR_PORT), "-address=", "-dir="));
     helpMap.put("status-locator", LocalizedStrings.SystemAdmin_STATUS_LOCATOR_HELP
-        .toLocalizedString(new Object[] {join(ManagerInfo.statusNames), "-dir="}));
+        .toLocalizedString(join(ManagerInfo.statusNames), "-dir="));
     helpMap.put("info-locator",
         LocalizedStrings.SystemAdmin_INFO_LOCATOR_HELP.toLocalizedString("-dir="));
     helpMap.put("tail-locator-log",
@@ -1593,15 +1624,15 @@ public class SystemAdmin {
             .toLocalizedString());
     helpMap.put("-starttime=",
         LocalizedStrings.SystemAdmin_CAUSES_THE_0_COMMAND_TO_IGNORE_STATISTICS_SAMPLES_TAKEN_BEFORE_THIS_TIME_THE_ARGUMENT_FORMAT_MUST_MATCH_1
-            .toLocalizedString(new Object[] {"stats", DateFormatter.FORMAT_STRING}));
+            .toLocalizedString("stats", DateFormatter.FORMAT_STRING));
     helpMap.put("-endtime=",
         LocalizedStrings.SystemAdmin_CAUSES_THE_0_COMMAND_TO_IGNORE_STATISTICS_SAMPLES_TAKEN_AFTER_THIS_TIME_THE_ARGUMENT_FORMAT_MUST_MATCH_1
-            .toLocalizedString(new Object[] {"stats", DateFormatter.FORMAT_STRING}));
+            .toLocalizedString("stats", DateFormatter.FORMAT_STRING));
     helpMap.put("-dir=",
-        LocalizedStrings.SystemAdmin_DIR_ARGUMENT_HELP
-            .toLocalizedString(new Object[] {DistributionConfig.GEMFIRE_PREFIX + "properties",
-                DistributionConfig.GEMFIRE_PREFIX + "systemDirectory", "GEMFIRE", "defaultSystem",
-                "version"}));
+        LocalizedStrings.SystemAdmin_DIR_ARGUMENT_HELP.toLocalizedString(
+            DistributionConfig.GEMFIRE_PREFIX + "properties",
+            DistributionConfig.GEMFIRE_PREFIX + "systemDirectory", "GEMFIRE", "defaultSystem",
+            "version"));
     helpMap.put("-D",
         LocalizedStrings.SystemAdmin_SETS_A_JAVA_SYSTEM_PROPERTY_IN_THE_LOCATOR_VM_USED_MOST_OFTEN_FOR_CONFIGURING_SSL_COMMUNICATION
             .toLocalizedString());
@@ -1649,7 +1680,6 @@ public class SystemAdmin {
     usageMap.put("info-locator", "info-locator [-dir=<locatorDir>]");
     usageMap.put("tail-locator-log", "tail-locator-log [-dir=<locatorDir>]");
     usageMap.put("merge-logs", "merge-logs <logFile>+ [-out=<outFile>]");
-    usageMap.put("encrypt-password", "encrypt-password <passwordString>");
     usageMap.put("validate-disk-store", "validate-disk-store <diskStoreName> <directory>+");
     usageMap.put("upgrade-disk-store",
         "upgrade-disk-store <diskStoreName> <directory>+ [-maxOplogSize=<int>]");
@@ -1733,7 +1763,6 @@ public class SystemAdmin {
     cmdOptionsMap.put("export-disk-store", new String[] {"-outputDir="});
     cmdOptionsMap.put("shut-down-all", new String[] {});
     cmdOptionsMap.put("backup", new String[] {"-baseline="});
-    cmdOptionsMap.put("encrypt-password", new String[] {});
     cmdOptionsMap.put("print-stacks", new String[] {"-all-threads"});
   }
 
@@ -1900,19 +1929,8 @@ public class SystemAdmin {
   }
 
   private static boolean needsSysDir(String cmd) {
-    if (cmd.equalsIgnoreCase("stats")) {
-      return false;
-    }
-    if (cmd.equalsIgnoreCase("merge-logs")) {
-      return false;
-    }
-    if (cmd.equalsIgnoreCase("version")) {
-      return false;
-    }
-    if (cmd.equalsIgnoreCase("help")) {
-      return false;
-    }
-    return true;
+    return !(cmd.equalsIgnoreCase("stats") || cmd.equalsIgnoreCase("merge-logs")
+        || cmd.equalsIgnoreCase("version") || cmd.equalsIgnoreCase("help"));
   }
 
   public static File getProductDir() {
@@ -2206,11 +2224,6 @@ public class SystemAdmin {
           usage(cmd);
         }
         backup((String) cmdLine.get(0));
-      } else if (cmd.equalsIgnoreCase("encrypt-password")) {
-        if (cmdLine.size() != 1) {
-          usage(cmd);
-        }
-        PasswordUtil.encrypt((String) cmdLine.get(0));
       } else if (cmd.equalsIgnoreCase("print-stacks")) {
         printStacks(cmdLine, printStacksOption != null);
       } else {

http://git-wip-us.apache.org/repos/asf/geode/blob/43b00bd8/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
index ae1c005..baad039 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/i18n/LocalizedStrings.java
@@ -4514,8 +4514,6 @@ public class LocalizedStrings {
       "Prints statistic values from a statistic archive\nBy default all statistics are printed.\nThe statSpec arguments can be used to print individual resources or a specific statistic.\nThe format of a statSpec is: an optional combine operator, followed by an optional instanceId, followed by an optional typeId, followed by an optional statId.\nA combine operator can be \"{0}\" to combine all matches in the same file, or \"{1}\" to combine all matches across all files.\nAn instanceId must be the name or id of a resource.\nA typeId is a \"{2}\" followed by the name of a resource type.\nA statId is a \"{3}\" followed by the name of a statistic.\nA typeId or instanceId with no statId prints out all the matching resources and all their statistics.\nA typeId or instanceId with a statId prints out just the named statistic on the matching resources.\nA statId with no typeId or instanceId matches all statistics with that name.\nThe \"{4}\" option causes statistic descriptions to also be pr
 inted.\nThe \"{5}\" option, in conjunction with \"{6}\", causes the printed statistics to all be raw, unfiltered, values.\nThe \"{7}\" option, in conjunction with \"{6}\", causes the printed statistics to be the rate of change, per second, of the raw values.\nThe \"{8}\" option, in conjunction with \"{6}\", causes the printed statistics to be the rate of change, per sample, of the raw values.\nThe \"{9}\" option, in conjunction with \"{6}\", causes statistics whose values are all zero to not be printed.");
   public static final StringId SystemAdmin_STATS_HELP_PART_B = new StringId(3751,
       "The \"{0}\" option, in conjunction with \"{1}\", causes statistics samples taken before this time to be ignored. The argument format must match \"{2}\".\nThe \"{3}\" option, in conjunction with \"{1}\", causes statistics samples taken after this time to be ignored. The argument format must match \"{2}\".\nThe \"{1}\" option causes the data to come from an archive file.");
-  public static final StringId SystemAdmin_ENCRYPTS_A_PASSWORD_FOR_USE_IN_CACHE_XML_DATA_SOURCE_CONFIGURATION =
-      new StringId(3752, "Encrypts a password for use in cache.xml data source configuration.");
   public static final StringId SystemAdmin_START_LOCATOR_HELP = new StringId(3753,
       "Starts a locator.\nThe \"{0}\" option specifies the port the locator will listen on. It defaults to \"{1}\"\nThe \"{2}\" option specifies the address the locator will listen on. It defaults to listening on all local addresses.\nThe \"{3}\" option can be used to specify the directory the locator will run in.\nThe \"{4}\" option can be used to specify the gemfire.properties file for configuring the locator''s distributed system.  The file''s path should be absolute, or relative to the locator''s directory ({3})\nThe \"{5}\" option can be used to specify whether peer locator service should be enabled. True (the default) will enable the service.\nThe \"{6}\" option can be used to specify whether server locator service should be enabled. True (the default) will enable the service.\nThe \"{7}\" option can be used to specify a host name or ip address that will be sent to clients so they can connect to this locator. The default is to use the address the locator is listening on.\nThe 
 \"{8}\" option can be used to set system properties for the locator VM\nThe \"{9}\" option can be used to set vendor-specific VM options and is usually used to increase the size of the locator VM when using multicast.\n");
   public static final StringId SystemAdmin_STOP_LOCATOR_HELP = new StringId(3754,
@@ -4960,7 +4958,6 @@ public class LocalizedStrings {
       new StringId(3979, "Pooled High Priority Message Processor ");
   public static final StringId DistributionManager_POOLED_WAITING_MESSAGE_PROCESSOR =
       new StringId(3980, "Pooled Waiting Message Processor ");
-  public static final StringId PasswordUtil_ENCRYPTED_TO_0 = new StringId(3981, "Encrypted to {0}");
   public static final StringId DistributionManager_SHUTDOWN_MESSAGE_THREAD_FOR_0 =
       new StringId(3982, "Shutdown Message Thread for {0}");
   public static final StringId HealthMonitorImpl_HEALTH_MONITOR_OWNED_BY_0 =

http://git-wip-us.apache.org/repos/asf/geode/blob/43b00bd8/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java b/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java
index fec81ca..844b484 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/net/SocketCreator.java
@@ -14,32 +14,6 @@
  */
 package org.apache.geode.internal.net;
 
-import org.apache.commons.lang.StringUtils;
-import org.apache.geode.GemFireConfigException;
-import org.apache.geode.SystemConnectException;
-import org.apache.geode.SystemFailure;
-import org.apache.geode.admin.internal.InetAddressUtil;
-import org.apache.geode.cache.wan.GatewaySender;
-import org.apache.geode.cache.wan.GatewayTransportFilter;
-import org.apache.geode.distributed.ClientSocketFactory;
-import org.apache.geode.distributed.internal.DistributionConfig;
-import org.apache.geode.distributed.internal.DistributionConfigImpl;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
-import org.apache.geode.internal.ClassPathLoader;
-import org.apache.geode.internal.ConnectionWatcher;
-import org.apache.geode.internal.GfeConsoleReaderFactory;
-import org.apache.geode.internal.GfeConsoleReaderFactory.GfeConsoleReader;
-import org.apache.geode.internal.admin.SSLConfig;
-import org.apache.geode.internal.cache.wan.TransportFilterServerSocket;
-import org.apache.geode.internal.cache.wan.TransportFilterSocketFactory;
-import org.apache.geode.internal.i18n.LocalizedStrings;
-import org.apache.geode.internal.logging.LogService;
-import org.apache.geode.internal.logging.log4j.LocalizedMessage;
-import org.apache.geode.internal.security.SecurableCommunicationChannel;
-import org.apache.geode.internal.util.ArgumentRedactor;
-import org.apache.geode.internal.util.PasswordUtil;
-import org.apache.logging.log4j.Logger;
-
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.net.BindException;
@@ -75,6 +49,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
+
 import javax.naming.Context;
 import javax.naming.NamingEnumeration;
 import javax.naming.directory.Attribute;
@@ -97,6 +72,33 @@ import javax.net.ssl.TrustManager;
 import javax.net.ssl.TrustManagerFactory;
 import javax.net.ssl.X509ExtendedKeyManager;
 
+import org.apache.commons.lang.StringUtils;
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.GemFireConfigException;
+import org.apache.geode.SystemConnectException;
+import org.apache.geode.SystemFailure;
+import org.apache.geode.admin.internal.InetAddressUtil;
+import org.apache.geode.cache.wan.GatewaySender;
+import org.apache.geode.cache.wan.GatewayTransportFilter;
+import org.apache.geode.distributed.ClientSocketFactory;
+import org.apache.geode.distributed.internal.DistributionConfig;
+import org.apache.geode.distributed.internal.DistributionConfigImpl;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.ClassPathLoader;
+import org.apache.geode.internal.ConnectionWatcher;
+import org.apache.geode.internal.GfeConsoleReaderFactory;
+import org.apache.geode.internal.GfeConsoleReaderFactory.GfeConsoleReader;
+import org.apache.geode.internal.admin.SSLConfig;
+import org.apache.geode.internal.cache.wan.TransportFilterServerSocket;
+import org.apache.geode.internal.cache.wan.TransportFilterSocketFactory;
+import org.apache.geode.internal.i18n.LocalizedStrings;
+import org.apache.geode.internal.logging.LogService;
+import org.apache.geode.internal.logging.log4j.LocalizedMessage;
+import org.apache.geode.internal.security.SecurableCommunicationChannel;
+import org.apache.geode.internal.util.ArgumentRedactor;
+import org.apache.geode.internal.util.PasswordUtil;
+
 /**
  * Analyze configuration data (gemfire.properties) and configure sockets accordingly for SSL.
  * <p>
@@ -294,7 +296,7 @@ public class SocketCreator {
    * hits and duplicate strings
    */
   public static synchronized String getHostName(InetAddress addr) {
-    String result = (String) hostNames.get(addr);
+    String result = hostNames.get(addr);
     if (result == null) {
       result = addr.getHostName();
       hostNames.put(addr, result);
@@ -307,7 +309,7 @@ public class SocketCreator {
    * hits and duplicate strings
    */
   public static synchronized String getCanonicalHostName(InetAddress addr, String hostName) {
-    String result = (String) hostNames.get(addr);
+    String result = hostNames.get(addr);
     if (result == null) {
       hostNames.put(addr, hostName);
       return hostName;
@@ -434,14 +436,8 @@ public class SocketCreator {
             throw new GemFireConfigException(
                 "SSL properties are empty, but a console is not available");
           }
-          if (key.toLowerCase().contains("password")) {
-            char[] password = consoleReader.readPassword("Please enter " + key + ": ");
-            env.put(key, PasswordUtil.encrypt(new String(password), false));
-          } else {
-            String val = consoleReader.readLine("Please enter " + key + ": ");
-            env.put(key, val);
-          }
-
+          String val = consoleReader.readLine("Please enter " + key + ": ");
+          env.put(key, val);
         }
       }
     }
@@ -727,7 +723,7 @@ public class SocketCreator {
       } catch (BindException e) {
         BindException throwMe =
             new BindException(LocalizedStrings.SocketCreator_FAILED_TO_CREATE_SERVER_SOCKET_ON_0_1
-                .toLocalizedString(new Object[] {bindAddr, Integer.valueOf(nport)}));
+                .toLocalizedString(bindAddr, Integer.valueOf(nport)));
         throwMe.initCause(e);
         throw throwMe;
       }
@@ -784,7 +780,7 @@ public class SocketCreator {
       } catch (BindException e) {
         BindException throwMe =
             new BindException(LocalizedStrings.SocketCreator_FAILED_TO_CREATE_SERVER_SOCKET_ON_0_1
-                .toLocalizedString(new Object[] {bindAddr, Integer.valueOf(nport)}));
+                .toLocalizedString(bindAddr, Integer.valueOf(nport)));
         throwMe.initCause(e);
         throw throwMe;
       }

http://git-wip-us.apache.org/repos/asf/geode/blob/43b00bd8/geode-core/src/main/java/org/apache/geode/internal/util/PasswordUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/util/PasswordUtil.java b/geode-core/src/main/java/org/apache/geode/internal/util/PasswordUtil.java
index b3472c7..5cc3bcd 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/util/PasswordUtil.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/util/PasswordUtil.java
@@ -14,8 +14,6 @@
  */
 package org.apache.geode.internal.util;
 
-import org.apache.geode.internal.i18n.LocalizedStrings;
-
 import javax.crypto.Cipher;
 import javax.crypto.spec.SecretKeySpec;
 
@@ -44,72 +42,30 @@ public class PasswordUtil {
   private static byte[] init = "string".getBytes();
 
   /**
-   * Encrypts a password string
-   * 
-   * @param password String to be encrypted.
-   * @return String encrypted String
-   */
-  public static String encrypt(String password) {
-    return encrypt(password, true);
-  }
-
-  /**
-   * 
-   * @param password String to be encrypted
-   * @param echo if true prints result to system.out
-   * @return String encrypted String
-   */
-  public static String encrypt(String password, boolean echo) {
-    String encryptedString = null;
-    try {
-      SecretKeySpec key = new SecretKeySpec(init, "Blowfish");
-      Cipher cipher = Cipher.getInstance("Blowfish");
-      cipher.init(Cipher.ENCRYPT_MODE, key);
-      byte[] encrypted = cipher.doFinal(password.getBytes());
-      encryptedString = byteArrayToHexString(encrypted);
-      if (echo) {
-        System.out.println(
-            LocalizedStrings.PasswordUtil_ENCRYPTED_TO_0.toLocalizedString(encryptedString));
-      }
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-    return encryptedString;
-  }
-
-  /**
    * Decrypts an encrypted password string.
-   * 
+   *
    * @param password String to be decrypted
    * @return String decrypted String
    */
+  @Deprecated
   public static String decrypt(String password) {
+    String toDecrypt;
     if (password.startsWith("encrypted(") && password.endsWith(")")) {
-      byte[] decrypted = null;
-      try {
-        String toDecrypt = password.substring(10, password.length() - 1);
-        SecretKeySpec key = new SecretKeySpec(init, "Blowfish");
-        Cipher cipher = Cipher.getInstance("Blowfish");
-        cipher.init(Cipher.DECRYPT_MODE, key);
-        decrypted = cipher.doFinal(hexStringToByteArray(toDecrypt));
-        return new String(decrypted);
-      } catch (Exception e) {
-        e.printStackTrace();
-      }
+      toDecrypt = password.substring(10, password.length() - 1);
+    } else {
+      toDecrypt = password;
     }
-    return password;
-  }
-
-  private static String byteArrayToHexString(byte[] b) {
-    StringBuilder sb = new StringBuilder(b.length * 2);
-    for (int i = 0; i < b.length; i++) {
-      int v = b[i] & 0xff;
-      if (v < 16) {
-        sb.append('0');
-      }
-      sb.append(Integer.toHexString(v));
+    byte[] decrypted;
+    try {
+      SecretKeySpec key = new SecretKeySpec(init, "Blowfish");
+      Cipher cipher = Cipher.getInstance("Blowfish");
+      cipher.init(Cipher.DECRYPT_MODE, key);
+      decrypted = cipher.doFinal(hexStringToByteArray(toDecrypt));
+      return new String(decrypted);
+    } catch (Exception e) {
+      e.printStackTrace();
     }
-    return sb.toString().toUpperCase();
+    return toDecrypt;
   }
 
   private static byte[] hexStringToByteArray(String s) {

http://git-wip-us.apache.org/repos/asf/geode/blob/43b00bd8/geode-core/src/main/java/org/apache/geode/management/internal/cli/Launcher.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/Launcher.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/Launcher.java
index e4e00da..a5986f5 100755
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/Launcher.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/Launcher.java
@@ -14,22 +14,23 @@
  */
 package org.apache.geode.management.internal.cli;
 
-import org.apache.geode.internal.GemFireVersion;
-import org.apache.geode.internal.PureJavaMode;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.shell.Gfsh;
-import org.apache.geode.management.internal.cli.shell.GfshConfig;
-import org.apache.geode.management.internal.cli.shell.jline.GfshHistory;
-import org.springframework.shell.core.ExitShellRequest;
-
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+
 import joptsimple.OptionException;
 import joptsimple.OptionParser;
 import joptsimple.OptionSet;
+import org.springframework.shell.core.ExitShellRequest;
+
+import org.apache.geode.internal.GemFireVersion;
+import org.apache.geode.internal.PureJavaMode;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.shell.Gfsh;
+import org.apache.geode.management.internal.cli.shell.GfshConfig;
+import org.apache.geode.management.internal.cli.shell.jline.GfshHistory;
 
 /**
  * Launcher class for :
@@ -93,8 +94,6 @@ public class Launcher {
   protected Launcher() {
     this.startupTimeLogHelper = new StartupTimeLogHelper();
     this.allowedCommandLineCommands = new HashSet<String>();
-
-    this.allowedCommandLineCommands.add(CliStrings.ENCRYPT);
     this.allowedCommandLineCommands.add(CliStrings.RUN);
     this.allowedCommandLineCommands.add(CliStrings.START_PULSE);
     this.allowedCommandLineCommands.add(CliStrings.START_JCONSOLE);

http://git-wip-us.apache.org/repos/asf/geode/blob/43b00bd8/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
index e37d1ba..2da95a7 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/ShellCommands.java
@@ -20,7 +20,38 @@ import static org.apache.geode.distributed.ConfigurationProperties.CLUSTER_SSL_P
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Writer;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.security.KeyStore;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+
+import javax.net.ssl.HttpsURLConnection;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManagerFactory;
+
 import org.apache.commons.lang.StringUtils;
+import org.springframework.shell.core.ExitShellRequest;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.ClassPathLoader;
 import org.apache.geode.internal.DSFIDFactory;
@@ -54,35 +85,6 @@ import org.apache.geode.management.internal.web.http.support.SimpleHttpRequester
 import org.apache.geode.management.internal.web.shell.HttpOperationInvoker;
 import org.apache.geode.management.internal.web.shell.RestHttpOperationInvoker;
 import org.apache.geode.security.AuthenticationFailedException;
-import org.springframework.shell.core.ExitShellRequest;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.Writer;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.security.KeyStore;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-import java.util.Set;
-import javax.net.ssl.HttpsURLConnection;
-import javax.net.ssl.KeyManagerFactory;
-import javax.net.ssl.SSLContext;
-import javax.net.ssl.TrustManagerFactory;
 
 /**
  *
@@ -985,13 +987,6 @@ public class ShellCommands implements GfshCommand {
     return result;
   }
 
-  @CliCommand(value = CliStrings.ENCRYPT, help = CliStrings.ENCRYPT__HELP)
-  @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GEODE_DEBUG_UTIL})
-  public Result encryptPassword(@CliOption(key = CliStrings.ENCRYPT_STRING,
-      help = CliStrings.ENCRYPT_STRING__HELP, mandatory = true) String stringToEncrypt) {
-    return ResultBuilder.createInfoResult(PasswordUtil.encrypt(stringToEncrypt, false/* echo */));
-  }
-
   @CliCommand(value = {CliStrings.VERSION}, help = CliStrings.VERSION__HELP)
   @CliMetaData(shellOnly = true, relatedTopic = {CliStrings.TOPIC_GFSH})
   public Result version(@CliOption(key = {CliStrings.VERSION__FULL}, specifiedDefaultValue = "true",

http://git-wip-us.apache.org/repos/asf/geode/blob/43b00bd8/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
index bdf5cff..0f74edd 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
@@ -36,6 +36,8 @@ import static org.apache.geode.distributed.ConfigurationProperties.STATISTIC_ARC
 import static org.apache.geode.distributed.ConfigurationProperties.STATISTIC_SAMPLE_RATE;
 import static org.apache.geode.distributed.ConfigurationProperties.USE_CLUSTER_CONFIGURATION;
 
+import java.text.MessageFormat;
+
 import org.apache.geode.cache.PartitionAttributesFactory;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.distributed.ConfigurationProperties;
@@ -44,8 +46,6 @@ import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.cache.xmlcache.CacheXml;
 import org.apache.geode.management.internal.cli.shell.Gfsh;
 
-import java.text.MessageFormat;
-
 /**
  * - * Contains 'String' constants used as key to the Localized strings to be used in classes under
  * <code>org.apache.geode.management.internal.cli</code> for Command Line Interface (CLI). NOTES: 1.
@@ -1265,13 +1265,6 @@ public class CliStrings {
       "String to be echoed. For example, \"SYS_USER variable is set to ${SYS_USER}\".";
   public static final String ECHO__MSG__NO_GFSH_INSTANCE = "Could not get GFSH Instance";
 
-  /* 'encrypt password' command */
-  public static final String ENCRYPT = "encrypt password";
-  public static final String ENCRYPT__HELP =
-      "Encrypt a password for use in data source configuration.";
-  public static final String ENCRYPT_STRING = "password";
-  public static final String ENCRYPT_STRING__HELP = "Password to be encrypted.";
-
   /* 'execute function' command */
   public static final String EXECUTE_FUNCTION = "execute function";
   public static final String EXECUTE_FUNCTION__HELP =

http://git-wip-us.apache.org/repos/asf/geode/blob/43b00bd8/geode-core/src/test/java/org/apache/geode/cache/util/PasswordUtilJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/util/PasswordUtilJUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/util/PasswordUtilJUnitTest.java
deleted file mode 100644
index 06e8666..0000000
--- a/geode-core/src/test/java/org/apache/geode/cache/util/PasswordUtilJUnitTest.java
+++ /dev/null
@@ -1,42 +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.cache.util;
-
-import static org.junit.Assert.*;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.internal.util.PasswordUtil;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.categories.UnitTest;
-
-@Category({UnitTest.class, SecurityTest.class})
-public class PasswordUtilJUnitTest {
-
-  @Test
-  public void testPasswordUtil() {
-    String x = "password";
-    String z = null;
-
-    // System.out.println(x);
-    String y = PasswordUtil.encrypt(x);
-    // System.out.println(y);
-    y = "encrypted(" + y + ")";
-    z = PasswordUtil.decrypt(y);
-    // System.out.println(z);
-    assertEquals(x, z);
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/43b00bd8/geode-core/src/test/java/org/apache/geode/internal/util/PasswordUtilJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/util/PasswordUtilJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/util/PasswordUtilJUnitTest.java
new file mode 100644
index 0000000..8051c56
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/util/PasswordUtilJUnitTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.util;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category({UnitTest.class, SecurityTest.class})
+public class PasswordUtilJUnitTest {
+
+  /**
+   * Tests PasswordUtil's decrypt method (which is now deprecated) and verifies that backwards
+   * compatibility has not been broken with the removal of PasswordUtil's encryption methods.
+   * Previously encrypted passwords may still be used and decrypted.
+   */
+  @Test
+  public void testPasswordUtil() {
+    String password = "password";
+    String encrypted = "encrypted(C3CDC3485F7FF64381841CD344CBDF8A)";
+    String decrypted = PasswordUtil.decrypt(encrypted);
+    assertEquals(password, decrypted);
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/43b00bd8/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
index 800d0da..7ce0438 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
@@ -224,7 +224,6 @@ public class TestCommand {
     createTestCommand("debug --state=on", null);
     createTestCommand("describe connection", null);
     createTestCommand("echo --string=\"Hello World!\"", null);
-    createTestCommand("encrypt password --password=value", null);
     createTestCommand("version", null);
     createTestCommand("sleep", null);
     createTestCommand("sh ls", null);
@@ -249,5 +248,5 @@ public class TestCommand {
 
     // Misc commands
     // createTestCommand("shutdown", clusterManage);
-  };
+  }
 }


[24/25] geode git commit: GEODE-3101: Release local locks held by the JTA beforeCompletion() in client when the JTA failed on the server.

Posted by kl...@apache.org.
GEODE-3101: Release local locks held by the JTA beforeCompletion() in client  when the JTA failed on the server.

          Also added a unit test would fail without the fix.


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: b7f5391d983cda4d95dc34a21aeb414a3c0a14c2
Parents: 822946b
Author: eshu <es...@pivotal.io>
Authored: Wed Jun 21 15:41:25 2017 -0700
Committer: eshu <es...@pivotal.io>
Committed: Wed Jun 21 15:41:25 2017 -0700

----------------------------------------------------------------------
 .../internal/cache/tx/ClientTXStateStub.java    |  10 +-
 .../internal/jta/ClientServerJTADUnitTest.java  | 143 +++++++++++++++++++
 2 files changed, 151 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/b7f5391d/geode-core/src/main/java/org/apache/geode/internal/cache/tx/ClientTXStateStub.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tx/ClientTXStateStub.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tx/ClientTXStateStub.java
index ded789e..ab5701c 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tx/ClientTXStateStub.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tx/ClientTXStateStub.java
@@ -21,7 +21,7 @@ import java.util.List;
 import javax.transaction.Status;
 
 import org.apache.logging.log4j.Logger;
-
+import org.apache.geode.GemFireException;
 import org.apache.geode.cache.CommitConflictException;
 import org.apache.geode.cache.TransactionDataNodeHasDepartedException;
 import org.apache.geode.cache.TransactionException;
@@ -237,7 +237,13 @@ public class ClientTXStateStub extends TXStateStub {
   @Override
   public void beforeCompletion() {
     obtainLocalLocks();
-    this.firstProxy.beforeCompletion(proxy.getTxId().getUniqId());
+    try {
+      this.firstProxy.beforeCompletion(proxy.getTxId().getUniqId());
+    } catch (GemFireException e) {
+      this.lockReq.releaseLocal();
+      this.firstProxy.getPool().releaseServerAffinity();
+      throw e;
+    }
   }
 
   public InternalDistributedMember getOriginatingMember() {

http://git-wip-us.apache.org/repos/asf/geode/blob/b7f5391d/geode-core/src/test/java/org/apache/geode/internal/jta/ClientServerJTADUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/jta/ClientServerJTADUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/jta/ClientServerJTADUnitTest.java
new file mode 100644
index 0000000..51ef44a
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/jta/ClientServerJTADUnitTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.jta;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import javax.transaction.Status;
+
+import org.apache.geode.GemFireException;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.client.ClientCacheFactory;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.cache.server.CacheServer;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.internal.cache.TXManagerImpl;
+import org.apache.geode.internal.cache.TXStateProxyImpl;
+import org.apache.geode.internal.cache.tx.ClientTXStateStub;
+import org.apache.geode.test.dunit.Assert;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.cache.internal.JUnit4CacheTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.awaitility.Awaitility;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({DistributedTest.class})
+public class ClientServerJTADUnitTest extends JUnit4CacheTestCase {
+  private String key = "key";
+  private String value = "value";
+  private String newValue = "newValue";
+
+  @Test
+  public void testClientTXStateStubBeforeCompletion() throws Exception {
+    final Host host = Host.getHost(0);
+    final VM server = host.getVM(0);
+    final VM client = host.getVM(1);
+    final String regionName = getUniqueName();
+    getBlackboard().initBlackboard();
+    final Properties properties = getDistributedSystemProperties();
+
+    final int port = server.invoke("create cache", () -> {
+      Cache cache = getCache(properties);
+      CacheServer cacheServer = createCacheServer(cache);
+      Region region = cache.createRegionFactory(RegionShortcut.REPLICATE).create(regionName);
+      region.put(key, value);
+
+      return cacheServer.getPort();
+    });
+
+    client.invoke(() -> createClientRegion(host, port, regionName));
+
+    createClientRegion(host, port, regionName);
+
+    Region region = getCache().getRegion(regionName);
+    assertTrue(region.get(key).equals(value));
+
+    String first = "one";
+    String second = "two";
+
+    client.invokeAsync(() -> commitTxWithBeforeCompletion(regionName, true, first, second));
+
+    getBlackboard().waitForGate(first, 30, TimeUnit.SECONDS);
+    TXManagerImpl mgr = (TXManagerImpl) getCache().getCacheTransactionManager();
+    mgr.begin();
+    region.put(key, newValue);
+    TXStateProxyImpl tx = (TXStateProxyImpl) mgr.internalSuspend();
+    ClientTXStateStub txStub = (ClientTXStateStub) tx.getRealDeal(null, null);
+    mgr.internalResume(tx);
+    try {
+      txStub.beforeCompletion();
+      fail("expected to get CommitConflictException");
+    } catch (GemFireException e) {
+      // expected commit conflict exception thrown from server
+      mgr.setTXState(null);
+      getBlackboard().signalGate(second);
+    }
+
+    Awaitility.await().pollInterval(10, TimeUnit.MILLISECONDS).pollDelay(10, TimeUnit.MILLISECONDS)
+        .atMost(30, TimeUnit.SECONDS).until(() -> region.get(key).equals(newValue));
+
+    try {
+      commitTxWithBeforeCompletion(regionName, false, first, second);
+    } catch (Exception e) {
+      Assert.fail("got unexpected exception", e);
+    }
+  }
+
+  private CacheServer createCacheServer(Cache cache) {
+    CacheServer server = cache.addCacheServer();
+    server.setPort(AvailablePortHelper.getRandomAvailableTCPPort());
+    try {
+      server.start();
+    } catch (IOException e) {
+      Assert.fail("got exception", e);
+    }
+    return server;
+  }
+
+  private void createClientRegion(final Host host, final int port0, String regionName) {
+    ClientCacheFactory cf = new ClientCacheFactory();
+    cf.addPoolServer(host.getHostName(), port0);
+    ClientCache cache = getClientCache(cf);
+    cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(regionName);
+  }
+
+  private void commitTxWithBeforeCompletion(String regionName, boolean withWait, String first,
+      String second) throws TimeoutException, InterruptedException {
+    Region region = getCache().getRegion(regionName);
+    TXManagerImpl mgr = (TXManagerImpl) getCache().getCacheTransactionManager();
+    mgr.begin();
+    region.put(key, newValue);
+    TXStateProxyImpl tx = (TXStateProxyImpl) mgr.internalSuspend();
+    ClientTXStateStub txStub = (ClientTXStateStub) tx.getRealDeal(null, null);
+    mgr.internalResume(tx);
+    txStub.beforeCompletion();
+    if (withWait) {
+      getBlackboard().signalGate(first);
+      getBlackboard().waitForGate(second, 30, TimeUnit.SECONDS);
+    }
+    txStub.afterCompletion(Status.STATUS_COMMITTED);
+  }
+}


[13/25] geode git commit: GEODE-3092: fix specifiedDefaultValue for cacheLoader and cacheWriter

Posted by kl...@apache.org.
GEODE-3092: fix specifiedDefaultValue for cacheLoader and cacheWriter


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: c6d88f748f32293edfb2c92e914a12b21969c061
Parents: 87c26be
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon Jun 19 09:08:35 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Jun 19 14:02:24 2017 -0700

----------------------------------------------------------------------
 .../CreateAlterDestroyRegionCommands.java       | 45 ++++++++++----------
 .../internal/cli/GfshParserParsingTest.java     | 19 +++++++--
 2 files changed, 38 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/c6d88f74/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
index 842802b..bf22854 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
@@ -14,7 +14,28 @@
  */
 package org.apache.geode.management.internal.cli.commands;
 
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
 import org.apache.commons.lang.StringUtils;
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
 import org.apache.geode.LogWriter;
 import org.apache.geode.cache.DataPolicy;
 import org.apache.geode.cache.ExpirationAttributes;
@@ -58,26 +79,6 @@ import org.apache.geode.management.internal.configuration.domain.XmlEntity;
 import org.apache.geode.management.internal.security.ResourceOperation;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
-import org.springframework.shell.core.annotation.CliAvailabilityIndicator;
-import org.springframework.shell.core.annotation.CliCommand;
-import org.springframework.shell.core.annotation.CliOption;
-
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Pattern;
-import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
-import javax.management.ObjectName;
 
 /**
  * @since GemFire 7.0
@@ -408,9 +409,9 @@ public class CreateAlterDestroyRegionCommands implements GfshCommand {
           help = CliStrings.ALTER_REGION__REGIONEXPIRATIONTTLACTION__HELP) String regionExpirationTTLAction,
       @CliOption(key = CliStrings.ALTER_REGION__CACHELISTENER, specifiedDefaultValue = "",
           help = CliStrings.ALTER_REGION__CACHELISTENER__HELP) String[] cacheListeners,
-      @CliOption(key = CliStrings.ALTER_REGION__CACHELOADER, specifiedDefaultValue = "null",
+      @CliOption(key = CliStrings.ALTER_REGION__CACHELOADER, specifiedDefaultValue = "",
           help = CliStrings.ALTER_REGION__CACHELOADER__HELP) String cacheLoader,
-      @CliOption(key = CliStrings.ALTER_REGION__CACHEWRITER, specifiedDefaultValue = "null",
+      @CliOption(key = CliStrings.ALTER_REGION__CACHEWRITER, specifiedDefaultValue = "",
           help = CliStrings.ALTER_REGION__CACHEWRITER__HELP) String cacheWriter,
       @CliOption(key = CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID, specifiedDefaultValue = "",
           help = CliStrings.ALTER_REGION__ASYNCEVENTQUEUEID__HELP) String[] asyncEventQueueIds,

http://git-wip-us.apache.org/repos/asf/geode/blob/c6d88f74/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserParsingTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserParsingTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserParsingTest.java
index 47d150d..961cdf8 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserParsingTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/GfshParserParsingTest.java
@@ -17,15 +17,16 @@ package org.apache.geode.management.internal.cli;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.Assert.assertEquals;
 
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.test.dunit.rules.GfshParserRule;
-import org.apache.geode.test.junit.categories.IntegrationTest;
+import java.util.Map;
+
 import org.junit.ClassRule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.springframework.shell.event.ParseResult;
 
-import java.util.Map;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.test.dunit.rules.GfshParserRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
 
 @Category(IntegrationTest.class)
 public class GfshParserParsingTest {
@@ -290,4 +291,14 @@ public class GfshParserParsingTest {
     GfshParseResult result = parser.parse(command);
     assertThat(result.getParamValue("name")).isEqualTo("\\u0005Name");
   }
+
+  @Test
+  public void testAlterRegion() throws Exception {
+    String command =
+        "alter region --name=/Person --cache-writer='' --cache-listener='' --cache-loader=''";
+    GfshParseResult result = parser.parse(command);
+    assertThat(result.getParamValue("cache-writer")).isNotNull().isEmpty();
+    assertThat(result.getParamValue("cache-listener")).isNotNull().isEmpty();
+    assertThat(result.getParamValue("cache-loader")).isNotNull().isEmpty();
+  }
 }


[02/25] geode git commit: Remove debug println

Posted by kl...@apache.org.
Remove debug println


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: ee88cd282be5fbe43a2d89cc3c1d4f548ba2e183
Parents: 588c3ed
Author: Jens Deppe <jd...@pivotal.io>
Authored: Thu Jun 15 08:01:46 2017 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Thu Jun 15 08:01:46 2017 -0700

----------------------------------------------------------------------
 gradle/docker.gradle | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/ee88cd28/gradle/docker.gradle
----------------------------------------------------------------------
diff --git a/gradle/docker.gradle b/gradle/docker.gradle
index 7a74c5d..5c2ef77 100644
--- a/gradle/docker.gradle
+++ b/gradle/docker.gradle
@@ -65,7 +65,7 @@ def dockerConfig = {
 
       args[3] = args[3] + matcher[0][1]
       def workdir = new File(args[3])
-      println "dockerize: making ${workdir}"
+      // println "dockerize: making ${workdir}"
       workdir.mkdirs()
       // println args
 


[05/25] geode git commit: GEODE-3072: Ignore dunit test

Posted by kl...@apache.org.
GEODE-3072: Ignore dunit test


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 4bf80a6a9236f98070ce63a1d77dce3732c35b4f
Parents: 42350f1
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Fri Jun 16 14:10:27 2017 -0700
Committer: Barry Oglesby <bo...@pivotal.io>
Committed: Fri Jun 16 14:10:27 2017 -0700

----------------------------------------------------------------------
 .../internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/4bf80a6a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java
index c732662..fa68781 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java
@@ -31,6 +31,7 @@ import org.apache.geode.test.junit.categories.ClientServerTest;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
 import org.awaitility.Awaitility;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
@@ -95,7 +96,8 @@ public class ClientServerMiscBCDUnitTest extends ClientServerMiscDUnitTest {
     });
   }
 
-  @Test
+  //@Test
+  @Ignore
   public void testDistributedMemberBytesWithCurrentServerAndOldClient() throws Exception {
     // Start current version server
     int serverPort = initServerCache(true);


[11/25] geode git commit: GEODE-2601: Fixing banner being logged twice during locator startup.

Posted by kl...@apache.org.
GEODE-2601: Fixing banner being logged twice during locator startup.


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 2d3e05ecedf724af8d413f50567afc7f9a87e343
Parents: 3154424
Author: YehEmily <em...@gmail.com>
Authored: Mon Jun 12 13:55:24 2017 -0700
Committer: Ken Howe <kh...@pivotal.io>
Committed: Mon Jun 19 09:52:13 2017 -0700

----------------------------------------------------------------------
 .../internal/InternalDistributedSystem.java     | 24 ++++++++++++
 .../distributed/internal/InternalLocator.java   | 41 ++++++++++----------
 .../internal/logging/LogWriterFactory.java      | 20 ++++------
 3 files changed, 53 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/2d3e05ec/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index d77e8b9..4ed58af 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -18,6 +18,30 @@ package org.apache.geode.distributed.internal;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 
+import java.io.File;
+import java.io.IOException;
+import java.io.Reader;
+import java.lang.reflect.Array;
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.StringTokenizer;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.CancelException;
 import org.apache.geode.ForcedDisconnectException;

http://git-wip-us.apache.org/repos/asf/geode/blob/2d3e05ec/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
index ae548c4..3ff27ea 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
@@ -18,7 +18,24 @@ import static org.apache.geode.distributed.ConfigurationProperties.BIND_ADDRESS;
 import static org.apache.geode.distributed.ConfigurationProperties.CACHE_XML_FILE;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 
+import java.io.File;
+import java.io.IOException;
+import java.net.ConnectException;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.commons.lang.StringUtils;
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.CancelException;
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.GemFireCache;
@@ -65,22 +82,6 @@ import org.apache.geode.management.internal.configuration.handlers.SharedConfigu
 import org.apache.geode.management.internal.configuration.messages.ConfigurationRequest;
 import org.apache.geode.management.internal.configuration.messages.SharedConfigurationStatusRequest;
 import org.apache.geode.management.internal.configuration.messages.SharedConfigurationStatusResponse;
-import org.apache.logging.log4j.Logger;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.ConnectException;
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Properties;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 /**
  * Provides the implementation of a distribution {@code Locator} as well as internal-only
@@ -469,17 +470,17 @@ public class InternalLocator extends Locator implements ConnectListener {
         LogWriterAppenders.getOrCreateAppender(LogWriterAppenders.Identifier.SECURITY, true, false,
             this.config, false);
 
-      } else {
-        // do not create a LogWriterAppender for security -- let it go through to logWriterAppender
       }
+        // do not create a LogWriterAppender for security -- let it go through to logWriterAppender
     }
 
     // LOG: create LogWriters for GemFireTracer (or use whatever was passed in)
     if (logWriter == null) {
       logWriter = LogWriterFactory.createLogWriterLogger(false, false, this.config,
-          !startDistributedSystem);
-      if (logger.isDebugEnabled())
+          false);
+      if (logger.isDebugEnabled()) {
         logger.debug("LogWriter for locator is created.");
+      }
     }
 
     if (securityLogWriter == null) {

http://git-wip-us.apache.org/repos/asf/geode/blob/2d3e05ec/geode-core/src/main/java/org/apache/geode/internal/logging/LogWriterFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/LogWriterFactory.java b/geode-core/src/main/java/org/apache/geode/internal/logging/LogWriterFactory.java
index c4fecfd..4202afd 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/logging/LogWriterFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/LogWriterFactory.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.internal.logging;
 
-import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.apache.geode.distributed.ConfigurationProperties.LOG_LEVEL;
 
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
@@ -79,16 +79,13 @@ public class LogWriterFactory {
     }
 
     // log the banner
-    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));
-      }
-      System.setProperty(InternalLocator.INHIBIT_DM_BANNER, "true"); // Ensure no more banners will
-                                                                     // be logged
+    if (!Boolean.getBoolean(InternalLocator.INHIBIT_DM_BANNER)
+        && InternalDistributedSystem.getReconnectAttemptCounter() == 0 // avoid filling up logs
+                                                                       // during auto-reconnect
+        && !isSecure // && !isLoner /* do this on a loner to fix bug 35602 */
+        && logConfig) {
+      // LOG:CONFIG:
+      logger.info(LogMarker.CONFIG, Banner.getString(null));
     } else {
       logger.debug("skipping banner - " + InternalLocator.INHIBIT_DM_BANNER + " is set to true");
     }
@@ -101,7 +98,6 @@ public class LogWriterFactory {
               LocalizedStrings.InternalDistributedSystem_STARTUP_CONFIGURATIONN_0,
               config.toLoggerString()));
     }
-
     return logger;
   }
 }


[08/25] geode git commit: GEODE-3071: Pull config of docker volumes into top level script so that they can be overridden

Posted by kl...@apache.org.
GEODE-3071: Pull config of docker volumes into top level script so that they can be overridden


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 07e8986e3deb722c0beea380eac00a18c86844aa
Parents: b26fa51
Author: Jens Deppe <jd...@pivotal.io>
Authored: Mon Jun 19 07:42:57 2017 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Jun 19 07:48:27 2017 -0700

----------------------------------------------------------------------
 build.gradle         | 5 +++++
 gradle/docker.gradle | 4 ++--
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/07e8986e/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 57372b3..4feb879 100755
--- a/build.gradle
+++ b/build.gradle
@@ -74,6 +74,11 @@ if (name == 'geode') {
   ext.scriptDir = 'gradle'
 }
 
+if (project.hasProperty('parallelDunit')) {
+  def pwd = System.getenv('PWD')
+  ext.dunitDockerVolumes = ["${pwd}":pwd]
+}
+
 apply from: "${scriptDir}/utilities.gradle"
 apply from: "${scriptDir}/java.gradle"
 apply from: "${scriptDir}/dependency-resolution.gradle"

http://git-wip-us.apache.org/repos/asf/geode/blob/07e8986e/gradle/docker.gradle
----------------------------------------------------------------------
diff --git a/gradle/docker.gradle b/gradle/docker.gradle
index b67c073..7971974 100644
--- a/gradle/docker.gradle
+++ b/gradle/docker.gradle
@@ -47,11 +47,11 @@ def dockerConfig = {
 
     // volumes mounted to the containers
     // in a form: host_dir : container_dir
-    def pwd = System.getenv('PWD')
     def gradleHome = System.getenv('GRADLE_USER_HOME') ?: "${System.getenv('HOME')}/.gradle"
     volumes = ["${gradleHome}":gradleHome]
 
-    volumes << ["${pwd}":pwd]
+    // Add volumes configured by top-level build script
+    volumes << project.dunitDockerVolumes
 
     // specify the user for starting Gradle test worker within the container.
     user = dunitDockerUser


[09/25] geode git commit: Merge branch 'develop' into feature/GEODE-3071

Posted by kl...@apache.org.
Merge branch 'develop' into feature/GEODE-3071


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: dba7dcd3042ae7cf9776822bb4e69f47291fc296
Parents: 07e8986 78f08e8
Author: Jens Deppe <jd...@pivotal.io>
Authored: Mon Jun 19 08:23:24 2017 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Jun 19 08:23:24 2017 -0700

----------------------------------------------------------------------
 .../cli/commands/StatusLocatorRealGfshTest.java |  50 +++
 ...erConfigurationServiceEndToEndDUnitTest.java |   8 +-
 .../geode/test/dunit/rules/gfsh/GfshRule.java   | 116 ++++++
 .../geode/test/dunit/rules/gfsh/GfshScript.java | 124 ++++++
 .../source/subnavs/geode-subnav.erb             |   8 +-
 geode-core/build.gradle                         |   4 -
 .../internal/InternalDistributedSystem.java     |  62 +--
 .../apache/geode/internal/cache/EventID.java    |   2 +-
 .../geode/internal/cache/GemFireCacheImpl.java  |   2 -
 .../geode/internal/jta/GlobalTransaction.java   |   4 +
 .../geode/internal/jta/TransactionImpl.java     |   3 +
 .../internal/jta/TransactionManagerImpl.java    |  15 +
 .../geode/internal/jta/UserTransactionImpl.java |   4 +
 .../org/apache/geode/internal/jta/XidImpl.java  |   4 +
 .../internal/security/CallbackInstantiator.java |  28 ++
 .../security/CustomSecurityService.java         | 346 ----------------
 .../security/DisabledSecurityService.java       | 221 -----------
 .../security/EnabledSecurityService.java        | 393 -------------------
 .../security/IntegratedSecurityService.java     | 374 ++++++++++++++++++
 .../security/LegacySecurityService.java         | 186 +--------
 .../internal/security/SecurityService.java      |  99 +++--
 .../security/SecurityServiceFactory.java        | 188 +++------
 .../internal/security/SecurityServiceType.java  |  28 --
 .../security/shiro/ConfigInitializer.java       |  43 --
 .../security/shiro/CustomAuthRealm.java         |  19 +-
 .../security/shiro/RealmInitializer.java        |  54 ---
 .../security/shiro/SecurityManagerProvider.java |  83 ++++
 .../internal/cli/commands/ConfigCommands.java   |  17 +-
 .../CreateAlterDestroyRegionCommands.java       |  24 +-
 .../internal/cli/commands/DataCommands.java     |   4 +-
 .../internal/cli/commands/DeployCommands.java   |  57 +--
 .../cli/commands/DiskStoreCommands.java         |  12 +-
 .../cli/commands/DurableClientCommands.java     |  22 +-
 .../cli/commands/ExportLogsCommand.java         |   4 +-
 .../internal/cli/commands/FunctionCommands.java |  20 +-
 .../internal/cli/commands/IndexCommands.java    |  14 +-
 .../cli/commands/LauncherLifecycleCommands.java |   4 +-
 .../internal/cli/commands/MemberCommands.java   |   4 +-
 .../cli/commands/MiscellaneousCommands.java     |  34 +-
 .../internal/cli/commands/QueueCommands.java    |   2 +-
 .../internal/cli/commands/RegionCommands.java   |   6 +-
 .../internal/cli/commands/WanCommands.java      | 116 +++---
 .../internal/cli/functions/DeployFunction.java  |   3 -
 .../cli/functions/UndeployFunction.java         |   9 +-
 .../internal/cli/i18n/CliStrings.java           |  99 +----
 .../controllers/ConfigCommandsController.java   |  24 +-
 .../web/controllers/DataCommandsController.java |   4 +-
 .../controllers/DeployCommandsController.java   |  26 +-
 .../DiskStoreCommandsController.java            |  22 +-
 .../DurableClientCommandsController.java        |  50 +--
 .../web/controllers/ExportLogController.java    |  10 +-
 .../controllers/FunctionCommandsController.java |  32 +-
 .../controllers/IndexCommandsController.java    |  36 +-
 .../controllers/MemberCommandsController.java   |   6 +-
 .../MiscellaneousCommandsController.java        |  36 +-
 .../controllers/QueueCommandsController.java    |   6 +-
 .../controllers/RegionCommandsController.java   |  20 +-
 .../web/controllers/WanCommandsController.java  | 143 +++----
 .../internal/index/HashIndexSetJUnitTest.java   |   2 +-
 .../membership/MembershipJUnitTest.java         |   3 -
 .../messenger/JGroupsMessengerJUnitTest.java    |   6 +-
 .../gms/mgr/GMSMembershipManagerJUnitTest.java  |   8 +-
 .../DeposePrimaryBucketMessageTest.java         |   1 -
 .../partitioned/FetchEntryMessageTest.java      |   1 -
 .../FetchPartitionDetailsMessageTest.java       |   1 -
 .../partitioned/MoveBucketMessageTest.java      |   1 -
 .../partitioned/RemoveBucketMessageTest.java    |   1 -
 .../sockets/ClientServerMiscBCDUnitTest.java    |  42 ++
 .../tier/sockets/ClientServerMiscDUnitTest.java |   4 +-
 .../tier/sockets/command/ContainsKey66Test.java |   2 +-
 .../tier/sockets/command/CreateRegionTest.java  |   2 +-
 .../tier/sockets/command/Destroy65Test.java     |   2 +-
 .../tier/sockets/command/DestroyRegionTest.java |   2 +-
 .../cache/tier/sockets/command/DestroyTest.java |   2 +-
 .../sockets/command/ExecuteFunction65Test.java  |   2 +-
 .../sockets/command/ExecuteFunction66Test.java  |   2 +-
 .../sockets/command/ExecuteFunctionTest.java    |   2 +-
 .../cache/tier/sockets/command/Get70Test.java   |   2 +-
 .../tier/sockets/command/GetAll651Test.java     |   2 +-
 .../tier/sockets/command/GetAll70Test.java      |   2 +-
 .../cache/tier/sockets/command/GetAllTest.java  |   2 +-
 .../sockets/command/GetAllWithCallbackTest.java |   2 +-
 .../tier/sockets/command/InvalidateTest.java    |   2 +-
 .../cache/tier/sockets/command/KeySetTest.java  |   2 +-
 .../cache/tier/sockets/command/Put61Test.java   |  30 +-
 .../cache/tier/sockets/command/Put65Test.java   |  28 +-
 .../cache/tier/sockets/command/PutTest.java     |  34 +-
 .../sockets/command/RegisterInterest61Test.java |   2 +-
 .../command/RegisterInterestList61Test.java     |   2 +-
 .../command/RegisterInterestList66Test.java     |   2 +-
 .../command/RegisterInterestListTest.java       |   2 +-
 .../sockets/command/RegisterInterestTest.java   |   2 +-
 .../tier/sockets/command/RemoveAllTest.java     |   2 +-
 .../cache/tier/sockets/command/RequestTest.java |   2 +-
 .../sockets/command/UnregisterInterestTest.java |   2 +-
 .../cache/wan/serial/DestroyMessageTest.java    |   1 -
 .../xmlcache/DefaultEntityResolver2Test.java    |   1 -
 .../logging/log4j/FastLoggerJUnitTest.java      |   3 +-
 .../security/DisabledSecurityServiceTest.java   | 164 --------
 .../security/EnabledSecurityServiceTest.java    | 184 ---------
 .../internal/security/FakePostProcessor.java    | 103 -----
 .../internal/security/FakeSecurityManager.java  | 103 -----
 ...ntegratedSecurityServiceConstructorTest.java |  93 +++++
 .../security/IntegratedSecurityServiceTest.java | 166 ++++++++
 .../security/LegacySecurityServiceTest.java     |  58 +++
 ...urityServiceFactoryShiroIntegrationTest.java |  38 +-
 .../security/SecurityServiceFactoryTest.java    | 314 ++++++---------
 .../internal/security/SecurityServiceTest.java  |  21 +-
 .../shiro/ConfigInitializerIntegrationTest.java |  91 -----
 .../internal/statistics/StatisticsImplTest.java |  61 +--
 .../geode/internal/tcp/ConnectionJUnitTest.java |   2 +-
 .../internal/CompositeBuilderViaFromTest.java   |   1 -
 .../internal/CompositeBuilderViaProxyTest.java  |   1 -
 .../beans/DistributedSystemBridgeJUnitTest.java |   2 +-
 .../cli/GfshParserAutoCompletionTest.java       |   6 +-
 .../cli/commands/ConfigCommandsDUnitTest.java   |   4 +-
 ...eateAlterDestroyRegionCommandsDUnitTest.java |  26 +-
 .../cli/commands/DeployCommandsDUnitTest.java   |  36 +-
 .../commands/DiskStoreCommandsDUnitTest.java    |  16 +-
 .../commands/GemfireDataCommandsDUnitTest.java  |  60 ++-
 .../cli/commands/IndexCommandsDUnitTest.java    |  16 +-
 .../ListAndDescribeRegionDUnitTest.java         |   8 +-
 .../cli/commands/MemberCommandsDUnitTest.java   |   2 +-
 .../MiscellaneousCommandsDUnitTest.java         |  17 +-
 .../cli/commands/QueueCommandsDUnitTest.java    |   6 +-
 .../cli/commands/ShowMetricsDUnitTest.java      |   6 +-
 .../cli/commands/ShowStackTraceDUnitTest.java   |   8 +-
 .../cli/shell/GfshMultilineCommandTest.java     |   7 +-
 .../ClusterConfigDistributionDUnitTest.java     |   2 +-
 ...tyServiceWithCustomRealmIntegrationTest.java |   2 +-
 ...urityServiceWithShiroIniIntegrationTest.java |   2 +-
 .../WanCommandsControllerJUnitTest.java         |  14 +-
 .../executor/AbstractScanExecutorTest.java      |   1 -
 .../CacheFactoryWithSecurityObjectTest.java     |  68 +++-
 ...SecurityManagerLifecycleDistributedTest.java |   4 +-
 .../transactions/JTA_transactions.html.md.erb   |  22 +-
 geode-docs/rest_apps/setup_config.html.md.erb   | 269 ++++++++-----
 .../tools_modules/gfsh/tour_of_gfsh.html.md.erb |  86 ++--
 .../lucene/internal/LuceneIndexFactorySpy.java  |   3 +-
 .../PartitionedRepositoryManagerJUnitTest.java  |   2 +-
 .../LuceneClusterConfigurationDUnitTest.java    |   4 +-
 .../LuceneQueryFunctionJUnitTest.java           |  12 +-
 .../TopEntriesFunctionCollectorJUnitTest.java   |   4 +-
 .../cache/lucene/test/IndexRepositorySpy.java   |   2 +-
 geode-pulse/build.gradle                        |   3 -
 .../apache/geode/tools/pulse/tests/Server.java  |  19 +-
 .../tools/pulse/tests/rules/ServerRule.java     |   6 +-
 ...mandCreateDestroyGatewaySenderDUnitTest.java |  56 +--
 ...anCommandCreateGatewayReceiverDUnitTest.java |  57 ++-
 ...WanCommandGatewayReceiverStartDUnitTest.java |  39 +-
 .../WanCommandGatewayReceiverStopDUnitTest.java |  39 +-
 .../WanCommandGatewaySenderStartDUnitTest.java  |  40 +-
 .../WanCommandGatewaySenderStopDUnitTest.java   |  34 +-
 .../wan/wancommand/WanCommandListDUnitTest.java |  32 +-
 .../WanCommandPauseResumeDUnitTest.java         |  68 ++--
 .../wancommand/WanCommandStatusDUnitTest.java   |  56 ++-
 .../ClusterConfigurationDUnitTest.java          |  10 +-
 ...nfigurationIndexWithFromClauseDUnitTest.java |   4 +-
 gradle/dependency-versions.properties           |   4 +-
 gradle/test.gradle                              |   7 +-
 160 files changed, 2604 insertions(+), 3534 deletions(-)
----------------------------------------------------------------------



[17/25] geode git commit: GEODE-3056: fix the message for invalid partition-resolver

Posted by kl...@apache.org.
GEODE-3056: fix the message for invalid partition-resolver


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: d8a11d1e4a4ccc78ddc3f86db172edf0dddf7215
Parents: d8160d6
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon Jun 19 09:53:39 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Jun 19 17:09:30 2017 -0700

----------------------------------------------------------------------
 .../CreateAlterDestroyRegionCommands.java       | 13 ++--
 .../CreateAlterDestroyRegionCommandsTest.java   | 65 ++++++++++++++++++++
 2 files changed, 73 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/d8a11d1e/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
index bf22854..b762396 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
@@ -612,14 +612,17 @@ public class CreateAlterDestroyRegionCommands implements GfshCommand {
     }
   }
 
-  private void validateRegionFunctionArgs(InternalCache cache,
-      RegionFunctionArgs regionFunctionArgs) {
+  DistributedSystemMXBean getDSMBean(InternalCache cache) {
+    ManagementService managementService = ManagementService.getExistingManagementService(cache);
+    return managementService.getDistributedSystemMXBean();
+  }
+
+  void validateRegionFunctionArgs(InternalCache cache, RegionFunctionArgs regionFunctionArgs) {
     if (regionFunctionArgs.getRegionPath() == null) {
       throw new IllegalArgumentException(CliStrings.CREATE_REGION__MSG__SPECIFY_VALID_REGION_PATH);
     }
 
-    ManagementService managementService = ManagementService.getExistingManagementService(cache);
-    DistributedSystemMXBean dsMBean = managementService.getDistributedSystemMXBean();
+    DistributedSystemMXBean dsMBean = getDSMBean(cache);
 
     String useAttributesFrom = regionFunctionArgs.getUseAttributesFrom();
     if (useAttributesFrom != null && !useAttributesFrom.isEmpty()
@@ -840,7 +843,7 @@ public class CreateAlterDestroyRegionCommands implements GfshCommand {
         } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
           throw new IllegalArgumentException(
               CliStrings.format(CliStrings.CREATE_REGION__MSG__INVALID_PARTITION_RESOLVER,
-                  new Object[] {regionFunctionArgs.getCompressor()}),
+                  new Object[] {regionFunctionArgs.getPartitionResolver()}),
               e);
         }
       }

http://git-wip-us.apache.org/repos/asf/geode/blob/d8a11d1e/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsTest.java b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsTest.java
new file mode 100644
index 0000000..155b1ad
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/cli/commands/CreateAlterDestroyRegionCommandsTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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.management.internal.cli.commands;
+
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.management.DistributedSystemMXBean;
+import org.apache.geode.management.internal.cli.functions.RegionFunctionArgs;
+import org.apache.geode.test.dunit.rules.GfshParserRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class CreateAlterDestroyRegionCommandsTest {
+
+  @Rule
+  public GfshParserRule parser = new GfshParserRule();
+
+  @Test
+  public void testCreateRegionWithInvalidPartitionResolver() throws Exception {
+    InternalCache cache = mock(InternalCache.class);
+    DistributedSystemMXBean dsMBean = mock(DistributedSystemMXBean.class);
+    CreateAlterDestroyRegionCommands spy =
+        parser.spyCommand("create region --name=region3 --type=PARTITION --partition-resolver=Foo");
+    doReturn(cache).when(spy).getCache();
+    doReturn(dsMBean).when(spy).getDSMBean(cache);
+
+    parser.executeLastCommandWithInstance(spy);
+
+    ArgumentCaptor<RegionFunctionArgs> argsCaptor =
+        ArgumentCaptor.forClass(RegionFunctionArgs.class);
+
+    verify(spy).validateRegionFunctionArgs(any(), argsCaptor.capture());
+
+    RegionFunctionArgs args = argsCaptor.getValue();
+    assertThat(args.getPartitionResolver()).isEqualTo("Foo");
+
+    assertThatThrownBy(() -> spy.validateRegionFunctionArgs(cache, args))
+        .hasMessageContaining("Foo is an invalid Partition Resolver");
+  }
+}


[20/25] geode git commit: GEODE-3104 : correct help string to use --include-locators

Posted by kl...@apache.org.
GEODE-3104 : correct help string to use --include-locators

* this closes #595


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: b517ef8ddb4613729157c4e4ba6c3e83ae2dfc2c
Parents: 7df5af3
Author: Amey Barve <ab...@apache.org>
Authored: Wed Jun 21 18:27:01 2017 +0530
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Wed Jun 21 08:16:04 2017 -0700

----------------------------------------------------------------------
 .../org/apache/geode/management/internal/cli/i18n/CliStrings.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/b517ef8d/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
index 0829495..bdf5cff 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/cli/i18n/CliStrings.java
@@ -2063,7 +2063,7 @@ public class CliStrings {
   public static final String SHUTDOWN__MSG__CAN_NOT_SHUTDOWN_WITHIN_TIMEOUT =
       "Could not shutdown within timeout. Shutdown will continue in background";
   public static final String SHUTDOWN__MSG__NO_DATA_NODE_FOUND =
-      "No data node found for stopping. Please specify --shutdown-locators option if you want locators to be stopped";
+      "No data node found for stopping. Please specify --include-locators option if you want locators to be stopped";
 
   public static final String SHUTDOWN_TIMEDOUT =
       "Shutdown command timedout. Please manually check node status";


[10/25] geode git commit: GEODE-2601: Fixing banner being logged twice during locator startup (now only logs once if both locator and distributed systems are started)

Posted by kl...@apache.org.
GEODE-2601: Fixing banner being logged twice during locator startup (now only logs once if both locator and distributed systems are started)


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 31544243fd94aa8c1966e29670a4aa12363557c6
Parents: 78f08e8
Author: YehEmily <em...@gmail.com>
Authored: Thu Jun 8 14:40:39 2017 -0700
Committer: Ken Howe <kh...@pivotal.io>
Committed: Mon Jun 19 09:04:11 2017 -0700

----------------------------------------------------------------------
 .../internal/InternalDistributedSystem.java     | 32 +++++++++-----------
 .../distributed/internal/InternalLocator.java   |  3 +-
 .../internal/logging/LogWriterFactory.java      | 16 +++++-----
 3 files changed, 23 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/31544243/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
index 85f9146..d77e8b9 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalDistributedSystem.java
@@ -1797,7 +1797,7 @@ public class InternalDistributedSystem extends DistributedSystem
   @Override
   public Statistics[] getStatistics() {
     List<Statistics> statsList = this.statsList;
-    return (Statistics[]) statsList.toArray(new Statistics[0]);
+    return statsList.toArray(new Statistics[0]);
   }
 
   // StatisticsFactory methods
@@ -1829,7 +1829,7 @@ public class InternalDistributedSystem extends DistributedSystem
   }
 
   public FunctionStats getFunctionStats(String textId) {
-    FunctionStats stats = (FunctionStats) functionExecutionStatsMap.get(textId);
+    FunctionStats stats = functionExecutionStatsMap.get(textId);
     if (stats == null) {
       stats = new FunctionStats(this, textId);
       FunctionStats oldStats = functionExecutionStatsMap.putIfAbsent(textId, stats);
@@ -1870,7 +1870,7 @@ public class InternalDistributedSystem extends DistributedSystem
    */
   public interface StatisticsVisitor {
 
-    public void visit(Statistics stat);
+    void visit(Statistics stat);
   }
 
   public Set<String> getAllFunctionExecutionIds() {
@@ -2167,7 +2167,7 @@ public class InternalDistributedSystem extends DistributedSystem
   private void notifyResourceEventListeners(ResourceEvent event, Object resource) {
     for (Iterator<ResourceEventsListener> iter = resourceListeners.iterator(); iter.hasNext();) {
       try {
-        ResourceEventsListener listener = (ResourceEventsListener) iter.next();
+        ResourceEventsListener listener = iter.next();
         listener.handleEvent(event, resource);
       } catch (CancelException e) {
         // ignore
@@ -2302,7 +2302,7 @@ public class InternalDistributedSystem extends DistributedSystem
               boolean isDurableClient = false;
 
               if (dca != null) {
-                isDurableClient = ((dca.getId() == null || dca.getId().isEmpty()) ? false : true);
+                isDurableClient = (!(dca.getId() == null || dca.getId().isEmpty()));
               }
 
               ((InternalDistributedSystem) ds).disconnect(false,
@@ -2337,7 +2337,7 @@ public class InternalDistributedSystem extends DistributedSystem
      * 
      * @param sys the the system we are disconnecting from process should take before returning.
      */
-    public void onDisconnect(InternalDistributedSystem sys);
+    void onDisconnect(InternalDistributedSystem sys);
 
   }
 
@@ -2352,7 +2352,7 @@ public class InternalDistributedSystem extends DistributedSystem
      * @param oldSystem the old DS, which is in a partially disconnected state and cannot be used
      *        for messaging
      */
-    public void reconnecting(InternalDistributedSystem oldSystem);
+    void reconnecting(InternalDistributedSystem oldSystem);
 
     /**
      * Invoked after a reconnect to the distributed system
@@ -2360,8 +2360,7 @@ public class InternalDistributedSystem extends DistributedSystem
      * @param oldSystem the old DS
      * @param newSystem the new DS
      */
-    public void onReconnect(InternalDistributedSystem oldSystem,
-        InternalDistributedSystem newSystem);
+    void onReconnect(InternalDistributedSystem oldSystem, InternalDistributedSystem newSystem);
   }
 
   /**
@@ -2374,7 +2373,7 @@ public class InternalDistributedSystem extends DistributedSystem
      * 
      * @param sys
      */
-    public void onShutdown(InternalDistributedSystem sys);
+    void onShutdown(InternalDistributedSystem sys);
   }
 
   /**
@@ -2427,10 +2426,7 @@ public class InternalDistributedSystem extends DistributedSystem
       return false;
     }
     boolean newDsConnected = (rds == null || !rds.isConnected());
-    if (!newDsConnected) {
-      return false;
-    }
-    return true;
+    return newDsConnected;
   }
 
 
@@ -2754,7 +2750,7 @@ public class InternalDistributedSystem extends DistributedSystem
         if (newDM instanceof DistributionManager) {
           // Admin systems don't carry a cache, but for others we can now create
           // a cache
-          if (((DistributionManager) newDM).getDMType() != DistributionManager.ADMIN_ONLY_DM_TYPE) {
+          if (newDM.getDMType() != DistributionManager.ADMIN_ONLY_DM_TYPE) {
             try {
               CacheConfig config = new CacheConfig();
               if (cacheXML != null) {
@@ -2962,7 +2958,7 @@ public class InternalDistributedSystem extends DistributedSystem
     /**
      * Invoked after a connection to the distributed system is created
      */
-    public void onConnect(InternalDistributedSystem sys);
+    void onConnect(InternalDistributedSystem sys);
   }
 
   public String forceStop() {
@@ -3064,8 +3060,8 @@ public class InternalDistributedSystem extends DistributedSystem
    * Privacy Violations that Fortify will complain about.
    * </p>
    */
-  public static interface CreationStackGenerator {
+  public interface CreationStackGenerator {
 
-    public Throwable generateCreationStack(final DistributionConfig config);
+    Throwable generateCreationStack(final DistributionConfig config);
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/31544243/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
index c299dd0..ae548c4 100644
--- a/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
+++ b/geode-core/src/main/java/org/apache/geode/distributed/internal/InternalLocator.java
@@ -478,9 +478,8 @@ public class InternalLocator extends Locator implements ConnectListener {
     if (logWriter == null) {
       logWriter = LogWriterFactory.createLogWriterLogger(false, false, this.config,
           !startDistributedSystem);
-      if (logger.isDebugEnabled()) {
+      if (logger.isDebugEnabled())
         logger.debug("LogWriter for locator is created.");
-      }
     }
 
     if (securityLogWriter == null) {

http://git-wip-us.apache.org/repos/asf/geode/blob/31544243/geode-core/src/main/java/org/apache/geode/internal/logging/LogWriterFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/logging/LogWriterFactory.java b/geode-core/src/main/java/org/apache/geode/internal/logging/LogWriterFactory.java
index 999bca9..c4fecfd 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/logging/LogWriterFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/logging/LogWriterFactory.java
@@ -87,19 +87,19 @@ public class LogWriterFactory {
         // LOG:CONFIG:
         logger.info(LogMarker.CONFIG, Banner.getString(null));
       }
+      System.setProperty(InternalLocator.INHIBIT_DM_BANNER, "true"); // Ensure no more banners will
+                                                                     // be logged
     } else {
       logger.debug("skipping banner - " + InternalLocator.INHIBIT_DM_BANNER + " is set to true");
     }
 
     // log the config
-    if (logConfig) {
-      if (!isLoner) {
-        // LOG:CONFIG: changed from config to info
-        logger.info(LogMarker.CONFIG,
-            LocalizedMessage.create(
-                LocalizedStrings.InternalDistributedSystem_STARTUP_CONFIGURATIONN_0,
-                config.toLoggerString()));
-      }
+    if (logConfig && !isLoner) {
+      // LOG:CONFIG: changed from config to info
+      logger.info(LogMarker.CONFIG,
+          LocalizedMessage.create(
+              LocalizedStrings.InternalDistributedSystem_STARTUP_CONFIGURATIONN_0,
+              config.toLoggerString()));
     }
 
     return logger;


[16/25] geode git commit: GEODE-3091: remove empty method

Posted by kl...@apache.org.
GEODE-3091: remove empty method

This closes #585


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: d8160d6e644976e88fa32b08993c3cc6574143c8
Parents: 80667f3
Author: Nick Reich <nr...@pivotal.io>
Authored: Thu Jun 15 15:59:50 2017 -0700
Committer: Anil <ag...@pivotal.io>
Committed: Mon Jun 19 15:00:23 2017 -0700

----------------------------------------------------------------------
 .../geode/internal/cache/DistributedRegion.java      |  1 -
 .../org/apache/geode/internal/cache/LocalRegion.java | 15 ---------------
 .../geode/internal/cache/PartitionedRegion.java      |  1 -
 3 files changed, 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/d8160d6e/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
index 9df64d0..21f911d 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/DistributedRegion.java
@@ -935,7 +935,6 @@ public class DistributedRegion extends LocalRegion implements CacheDistributionA
   public void invalidate(Object key, Object aCallbackArgument)
       throws TimeoutException, EntryNotFoundException {
     validateKey(key);
-    validateCallbackArg(aCallbackArgument);
     checkReadiness();
     checkForLimitedOrNoAccess();
     Lock dlock = this.getDistributedLockIfGlobal(key);

http://git-wip-us.apache.org/repos/asf/geode/blob/d8160d6e/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
----------------------------------------------------------------------
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 2b45a54..02625ee 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
@@ -1120,7 +1120,6 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
   @Retained
   EntryEventImpl newDestroyEntryEvent(Object key, Object aCallbackArgument) {
     validateKey(key);
-    validateCallbackArg(aCallbackArgument);
     checkReadiness();
     checkForLimitedOrNoAccess();
 
@@ -1349,7 +1348,6 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
       boolean retainResult) throws TimeoutException, CacheLoaderException {
     assert !retainResult || preferCD;
     validateKey(key);
-    validateCallbackArg(aCallbackArgument);
     checkReadiness();
     checkForNoAccess();
     discoverJTA();
@@ -1563,7 +1561,6 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
   @Override
   public void invalidateRegion(Object aCallbackArgument) throws TimeoutException {
     getDataView().checkSupportsRegionInvalidate();
-    validateCallbackArg(aCallbackArgument);
     checkReadiness();
     checkForLimitedOrNoAccess();
     RegionEventImpl event = new RegionEventImpl(this, Operation.REGION_INVALIDATE,
@@ -3118,8 +3115,6 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
   protected void validateArguments(Object key, Object value, Object aCallbackArgument) {
     validateKey(key);
     validateValue(value);
-    validateCallbackArg(aCallbackArgument);
-
   }
 
   void validateKey(Object key) {
@@ -3142,15 +3137,6 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
   }
 
   /**
-   * Starting in 3.5, we don't check to see if the callback argument is {@code Serializable}. We
-   * instead rely on the actual serialization (which happens in-thread with the put) to tell us if
-   * there are any problems. TODO: delete method validateCallbackArg
-   */
-  void validateCallbackArg(Object aCallbackArgument) {
-    // do nothing
-  }
-
-  /**
    * @since GemFire 5.0.2
    */
   private final boolean doExpensiveValidations =
@@ -11633,7 +11619,6 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
   public boolean remove(Object key, Object value, Object callbackArg) {
     checkIfConcurrentMapOpsAllowed();
     validateKey(key);
-    validateCallbackArg(callbackArg);
     checkReadiness();
     checkForLimitedOrNoAccess();
 

http://git-wip-us.apache.org/repos/asf/geode/blob/d8160d6e/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
index 288f4a9..cf3ca76 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
@@ -3082,7 +3082,6 @@ public class PartitionedRegion extends LocalRegion
       EntryEventImpl clientEvent, boolean returnTombstones)
       throws TimeoutException, CacheLoaderException {
     validateKey(key);
-    validateCallbackArg(aCallbackArgument);
     checkReadiness();
     checkForNoAccess();
     discoverJTA();


[07/25] geode git commit: GEODE-2854: GEODE REST API Docs Missing Authentication This closes #583

Posted by kl...@apache.org.
GEODE-2854: GEODE REST API Docs Missing Authentication
This closes #583


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 78f08e8a224cf99ff9981acacbf88452911f9069
Parents: dbc3197
Author: Dave Barnes <db...@pivotal.io>
Authored: Thu Jun 15 13:43:41 2017 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Fri Jun 16 15:47:05 2017 -0700

----------------------------------------------------------------------
 .../source/subnavs/geode-subnav.erb             |   8 +-
 geode-docs/rest_apps/setup_config.html.md.erb   | 269 ++++++++++++-------
 2 files changed, 176 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/78f08e8a/geode-book/master_middleman/source/subnavs/geode-subnav.erb
----------------------------------------------------------------------
diff --git a/geode-book/master_middleman/source/subnavs/geode-subnav.erb b/geode-book/master_middleman/source/subnavs/geode-subnav.erb
index aa0faf4..37c0b83 100644
--- a/geode-book/master_middleman/source/subnavs/geode-subnav.erb
+++ b/geode-book/master_middleman/source/subnavs/geode-subnav.erb
@@ -1576,8 +1576,14 @@ limitations under the License.
                     <li>
                         <a href="/docs/guide/12/rest_apps/rest_prereqs.html">Prerequisites and Limitations for Writing REST Applications</a>
                     </li>
-                    <li>
+                    <li class="has_submenu">
                         <a href="/docs/guide/12/rest_apps/setup_config.html">Setup and Configuration</a>
+                        <ul>
+                        <li><a href="/docs/guide/12/rest_apps/setup_config.html#setup_config_enabling_rest">Enabling the REST API</a></li>
+                        <li><a href="/docs/guide/12/rest_apps/setup_config.html#setup_config_starting_rest">Starting the REST API Service</a></li>
+                        <li><a href="/docs/guide/12/rest_apps/setup_config.html#setup_config_implementing_auth">Implementing Authentication</a></li>
+                        <li><a href="/docs/guide/12/rest_apps/setup_config.html#setup_config_implementing_auth">Programmatic Startup</a></li>
+                        </ul>
                     </li>
                     <li>
                         <a href="/docs/guide/12/rest_apps/using_swagger.html">Using the Swagger UI to Browse REST APIs</a>

http://git-wip-us.apache.org/repos/asf/geode/blob/78f08e8a/geode-docs/rest_apps/setup_config.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/rest_apps/setup_config.html.md.erb b/geode-docs/rest_apps/setup_config.html.md.erb
index 566b1a2..c557f4f 100644
--- a/geode-docs/rest_apps/setup_config.html.md.erb
+++ b/geode-docs/rest_apps/setup_config.html.md.erb
@@ -19,7 +19,10 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 
-The Apache Geode developer REST interface runs as an embedded HTTP or HTTPS service (Jetty server) within a Geode data node.
+The Apache Geode Developer REST interface runs as an embedded HTTP or HTTPS service (Jetty server) within one
+or more Geode servers.
+
+# REST API Libraries
 
 All Geode REST interface classes and required JAR files are distributed as a WAR file with the Geode product distribution. You can find the file in the following location:
 
@@ -29,162 +32,228 @@ All Geode REST interface classes and required JAR files are distributed as a WAR
 
 where _install-dir_ is the server installation directory and _n.n.n_ is a version number.
 
-To enable the developer REST API service in Apache Geode, set the `start-dev-rest-api` Geode property to `true` when starting a data node using either `gfsh` or the ServerLauncher API. Setting this property to true on a data node will start up an embedded Jetty server and deploy the REST developer API WAR file.
+# <a id="setup_config_enabling_rest" class="no-quick-link"></a>Enabling the REST API
+
+The REST API service for application development runs only on servers; you cannot run the service on a locator.
 
-**Note:**
-The REST API service for application development runs only on servers; you cannot use locators to host the developer Geode REST API services.
+To enable the Developer REST API service on a given server, set the `start-dev-rest-api` property
+to `true` for the server, using either the `gfsh start server` command or the ServerLauncher API.
+This starts an embedded Jetty server and deploys the Developer REST API WAR file on that server.
 
-You can have multiple REST enabled data nodes in a single distributed system. Each data node should
-have a separate host name and unique end point. To ensure that the data node is reachable on a
-machine with multiple NIC addresses, you can use `http-service-bind-address` to bind an address to
-the REST API service (as well as the other embedded web services such as Pulse).
+## Enabling the REST API on Multiple Servers
 
-You can also configure the Developer REST API service to run over
-HTTPS by enabling ssl for the `http` component in `gemfire.properties`
-or `gfsecurity.properties` or on server startup:
-See [SSL](../managing/security/ssl_overview.html) for details on configuring SSL parameters.
-These SSL parameters apply to all HTTP services hosted on the configured server, which can include the following:
+You can configure multiple REST-enabled servers in a single distributed system. Each server should
+have a separate host name and unique end point. To ensure that the server is reachable on a
+machine with multiple NIC addresses, use `http-service-bind-address` to bind an address to
+the REST API service (as well as the other embedded web services, such as Pulse).
+
+You can configure the Developer REST API service to run over HTTPS by enabling SSL for the `http`
+component in `gemfire.properties` or `gfsecurity.properties`, or on server startup.  See
+[SSL](../managing/security/ssl_overview.html) for details on configuring SSL parameters.  These SSL
+parameters apply to all HTTP services hosted on the configured server, which can include the
+following:
 
 -   Developer REST API service
 -   Management REST API service (for remote cluster management)
 -   Pulse monitoring tool
 
-The following procedure starts up a REST API service-enabled Geode deployment:
+# <a id="setup_config_starting_rest" class="no-quick-link"></a> Starting the REST API Service
 
-1.  Configure PDX for your cluster. You must configure PDX if either or both of the following conditions apply: 
-    -   Application peer member caches will access REST-accessible Regions (resources) with the `Region.get(key)`.
-    -   Your deployment has persistent regions that must be available as resources to the REST API.  To configure PDX in your cluster, perform the following steps:
-        1.  Start up a locator running the [cluster configuration service](../configuring/cluster_config/gfsh_persist.html) (enabled by default). For example: 
+To start a REST API service-enabled Geode deployment, configure PDX serialization for your
+cluster, then start the service on one or more server nodes.
 
-            ``` pre
-            gfsh>start locator --name=locator1
-            ```
-        2.  If your deployment has application peer member caches (for example, Java clients) that must also access REST-accessible Regions (resources), use the following gfsh command:
+## Configure PDX for your cluster
 
-            ``` pre
-            gfsh>configure pdx --read-serialized=true
-            ```
-        **Note:**
-        You do not need to configure `--read-serialized=true` if no application peer member caches are accessing the REST-accessible regions (resources) in your deployment.
-        3.  If your deployment contains **persistent regions** that must be REST-accessible, use the following gfsh command:
+You must configure PDX if either or both of the following conditions apply: 
 
-            ``` pre
-            gfsh>configure pdx --disk-store
-            ```
-        This command sets `pdx` `persistent` equal to true and sets the disk-store-name to DEFAULT. If desired, specify an existing disk store name as the value for `--disk-store`.
-        4.  If both of the above cases apply to your deployment, then configure PDX with the following single command:
+-   Application peer member caches will access REST-accessible regions (resources) with `Region.get(key)`.
+-   Your deployment has persistent regions that must be available as resources to the REST API.  
 
-            ``` pre
-            gfsh>configure pdx --read-serialized=true --disk-store
-            ```
+To configure PDX in your cluster, perform the following steps:
 
-    After you have configured PDX for your caches, then proceed with starting up your REST-enabled servers and other data nodes.
+1.  Start a locator running the [cluster configuration service](../configuring/cluster_config/gfsh_persist.html) (enabled by default). For example: 
 
-2.  Start a server node with the Geode property `start-dev-rest-api` set to `true`. 
-    Optionally, you can also configure a `http-service-bind-address` and `http-service-port` to
-    identify the cache server and specific port that will host REST services. If you do not specify
-    the `http-service-port`, the default port is 7070. If you do not specify
-    `http-service-bind-address`, the HTTP service will bind to all local addresses by default.
-    **Note:** If your application will be running in a VM (as when running in the cloud, for example), it's good practice to specify `http-service-bind-address` and `http-service-port`
-    so they will be publicly visible. The default values may not be visible outside the VM in which the application is running.
+    ``` pre
+    gfsh>start locator --name=locator1
+    ```
 
-    For example:
+2.  If your deployment has application peer member caches (for example, Java clients) that must also access REST-accessible Regions (resources), use the following gfsh command:
 
     ``` pre
-    gfsh>start server --name=server1 --start-rest-api=true \
-    --http-service-port=8080 --http-service-bind-address=localhost
+    gfsh>configure pdx --read-serialized=true
     ```
 
-    Any server that hosts data, even a server acting as a JMX manager, can start the developer REST API service. For example, to start the service on a server that is also a JMX manager, you would run:
+    **Note:**
+    You do not need to configure `--read-serialized=true` if no application peer member caches are accessing the REST-accessible regions (resources) in your deployment.
+
+3.  If your deployment contains **persistent regions** that must be REST-accessible, use the following gfsh command:
 
     ``` pre
-    gfsh>start server --name=server1  --start-rest-api=true \
-    --http-service-port=8080 --http-service-bind-address=localhost \
-    --J=-Dgemfire.jmx-manager=true --J=-Dgemfire.jmx-manager-start=true
+    gfsh>configure pdx --disk-store
     ```
+    This command sets `pdx` `persistent` equal to true and sets the disk-store-name to DEFAULT. If desired, specify an existing disk store name as the value for `--disk-store`.
+
+4.  If both of the above cases apply to your deployment, then configure PDX with the following single command:
+
+    ``` pre
+    gfsh>configure pdx --read-serialized=true --disk-store
+    ```
+
+    After you have configured PDX for your caches, then proceed with starting your REST-enabled servers and other servers.
+
+## Start the REST API Service on One or More Servers
+
+As described above, you can start the REST API service on a server by setting the Geode property `start-dev-rest-api` set to `true`. 
+If you wish to start the service on multiple servers, use a `http-service-bind-address` and `http-service-port` to
+identify the cache server and specific port that will host REST services. If you do not specify
+the `http-service-port`, the default port is 7070, which may collide with other locators and servers.
+If you do not specify `http-service-bind-address`, the HTTP service will bind to all local addresses by default.
 
-    Note that when started as a JMX Manager, the server will also host the Pulse web application in the same HTTP service.
+**Note:** If your application will be running in a VM (as when running in the cloud, for example),
+it is good practice to specify `http-service-bind-address` and `http-service-port` so they will be
+publicly visible. The default values may not be visible outside the VM in which the application is
+running.
 
-3.  You may also need to specify a CLASSPATH to load any functions that need to be made available to your REST services. For example:
+For example:
+
+```
+gfsh>start server --name=server1 --start-rest-api=true \
+--http-service-port=8080 --http-service-bind-address=localhost
+```
+
+Any server that hosts data, even a server acting as a JMX manager, can start the Developer REST API service. For example, to start the service on a server that is also a JMX manager, you would run:
+
+```
+gfsh>start server --name=server1  --start-rest-api=true \
+--http-service-port=8080 --http-service-bind-address=localhost \
+--J=-Dgemfire.jmx-manager=true --J=-Dgemfire.jmx-manager-start=true
+```
+
+Note that when started as a JMX Manager, the server will also host the Pulse web application in the same HTTP service.
+
+You may need to specify a CLASSPATH to load any functions that need to be made available to your REST services. For example:
+
+```
+gfsh>start server --name=server1 --start-rest-api=true \
+--http-service-port=8080 --http-service-bind-address=localhost \
+--classpath=/myapps/testfunctions.jar
+```
+
+You can specify these properties either upon server startup or in the server’s gemfire.properties configuration file. For example:
+
+```
+gfsh>start server --name=serverX --server-port=40405 --cache-xml-file=cache-config.xml \
+--properties-file=gemfire.properties --classpath=/myapps/testfunctions.jar
+```
+
+where gemfire.properties contains:
+
+```
+http-service-port=8080
+http-service-bind-address=localhost
+start-dev-rest-api=true
+```
+
+## Verify That The Service is Running
+
+Verify that the Geode REST API service is up and running. To validate this, you can perform the following checks:
+
+1.  Test the list resources endpoint (this step assumes that you have regions defined on your cluster):
+
+    ``` pre
+    curl -i http://localhost:8080/geode/v1
+    ```
+
+2.  Examine the server logs for the following messages:
 
     ``` pre
-    gfsh>start server --name=server1 --start-rest-api=true \
-    --http-service-port=8080 --http-service-bind-address=localhost \
-    --classpath=/myapps/testfunctions.jar
+    [info 2017/06/13 13:48:14.090 PDT gfsec-server1 <main> tid=0x1] Initializing Spring FrameworkServlet 'geode-mgmt'
+    [info 2017/06/13 13:48:14.091 PDT gfsec-server1 <main> tid=0x1] FrameworkServlet 'geode-mgmt': initialization started
     ```
 
-4.  You can also specify these properties either upon server startup or in the server’s gemfire.properties configuration file.
+3.  Open a browser and enter the following URL to browse the Swagger-enabled REST APIs:
 
     ``` pre
-    gfsh>start server --name=serverX --server-port=40405 --cache-xml-file=cache-config.xml \
-    --properties-file=gemfire.properties --classpath=/myapps/testfunctions.jar
+    http://<http-service-bind-address>:<http-service-port>/geode/docs/index.html
     ```
 
-    where gemfire.properties contains:
+    where *http-service-bind-address* is the address and *http-service-port* is the port number that you specified when starting the Development REST API service on the server. For example, based on the server started in an earlier example, you would enter:
 
     ``` pre
-    http-service-port=8080
-    http-service-bind-address=localhost
-    start-dev-rest-api=true
+    http://localhost:8080/geode/docs/index.html
     ```
 
-5.  Verify that the Geode REST API service is up and running. To validate this, you can perform the following checks:
-    1.  Test the list resources endpoint (this step assumes that you have regions defined on your cluster):
+If you did not specify these properties upon server startup or in `gemfire.properties`, then use the
+default of localhost and port 7070. See [Using the Swagger UI to Browse REST
+APIs](using_swagger.html#concept_rlr_y3c_54) for more information.
+
+# <a id="setup_config_implementing_auth" class="no-quick-link"></a>Implementing Authentication
+
+To turn on integrated security, start your servers and locators with the security-manager property
+set in your gemfire.properties file or on the gfsh command-line.
+The following example uses the sample implementation that is included in the Geode source,
+`org.apache.geode.examples.security.ExampleSecurityManager`.
+
+This implementation requires a JSON security configuration file which defines the allowed users and their corresponding
+permissions. (See the javadocs for `ExampleSecurityManager` for details on how to compose the JSON file.)
+Place a copy of the JSON security configuration file in the execution directory of each security-enabled member, then
+specify `--classpath=.` in the start command for each of those members.
 
-        ``` pre
-        curl -i http://localhost:8080/gemfire-api/v1
-        ```
+To start a server using a username and password that are defined in that server's security configuration, include the
+`--user=username` and `--password=password` options in the server's start command:
 
-    2.  Examine the server logs for the following messages:
+For example, suppose the JSON config file defines user "super-user" with password "1234567":
 
-        ``` pre
-        [info 2014/06/12 14:56:52.431 PDT rest-test <localhost-startStop-1> tid=0x4d] 
-        (tid=11 msgId=8) Initializing Spring FrameworkServlet 'gemfire-api'[info 2014/06/12 
-        14:56:52.432 PDT rest-test <localhost-startStop-1> tid=0x4d] (tid=11 msgId=9) 
-        FrameworkServlet 'gemfire-api': initialization started
-        ```
+```
+gfsh>start server --name=server1 --start-rest-api=true \
+--http-service-port=8080 --http-service-bind-address=localhost \
+--J=-Dgemfire.security-manager=org.apache.geode.examples.security.ExampleSecurityManager \
+--classpath=. --user=super-user --password=1234567
+```
 
-    3.  Open a browser and enter the following URL to browse the Swagger-enabled REST APIs:
+To contact the server through the REST interface, you must provide the username and password. Various REST GUI interfaces
+provide different ways of accomplishing this. The `curl` command offers the `--user` (or `-u`) option for this purpose,
+where username and password are specified as a colon-separated pair:
 
-        ``` pre
-        http://<http-service-bind-address>:<http-service-port>/gemfire-api/docs/index.html
-        ```
+```
+curl -i --user super-user:1234567 http://localhost:8080/geode/v1
+```
 
-        where *http-service-bind-address* is the address and *http-service-port* is the port number that you specified when starting the Development REST API service on the server. For example, based on the server started in step 2, you would enter:
+In a simple URL, such as in a browser address bar, the credentials can be given as a prefix to the host name
+in the form `username:password@`:
 
-        ``` pre
-        http://localhost:8080/gemfire-api/docs/index.html
-        ```
+```
+http://super-user:1234567@localhost:8080/geode/v1
+```
 
-        If you did not specify these properties upon server startup or in `gemfire.properties`, then use the default of localhost and port 7070. See [Using the Swagger UI to Browse REST APIs](using_swagger.html#concept_rlr_y3c_54) for more information.
 
-## Programmatic Startup
+# <a id="setup_config_implementing_auth" class="no-quick-link"></a>Programmatic Startup
 
-You can also start up and configure Geode REST services programmatically. For example:
+You can also start and configure Geode REST services programmatically. For example:
 
 ``` pre
-    import org.apache.geode.distributed.ServerLauncher;
+import org.apache.geode.distributed.ServerLauncher;
 
-     public class MyEmbeddedRestServer {
+public class MyEmbeddedRestServer {
 
-     public static void main(String[] args){
-         ServerLauncher serverLauncher  = new ServerLauncher.Builder()
-           .set("start-dev-rest-api", "true")
-           .set("http-service-port", "8080")
-           .set("http-service-bind-address", "localhost")
-           .setPdxReadSerialized(true)
-           .build();
+public static void main(String[] args){
+     ServerLauncher serverLauncher  = new ServerLauncher.Builder()
+       .set("start-dev-rest-api", "true")
+       .set("http-service-port", "8080")
+       .set("http-service-bind-address", "localhost")
+       .setPdxReadSerialized(true)
+       .build();
 
-          serverLauncher.start();  
+      serverLauncher.start();  
 
-          System.out.println("REST server successfully started");
-        }
+      System.out.println("REST server successfully started");
     }
+}
 ```
-You can then verify that the developer REST API service has been started programmatically by visiting the following URL:
+You can then verify that the Developer REST API service has been started programmatically by visiting the following URL:
 
 ``` pre
-http://localhost:8080/gemfire-api/docs/index.html
+http://localhost:8080/geode/docs/index.html
 ```
 
 


[23/25] geode git commit: GEODE-2820: Added awaitlity clause to wait for the index size to be correct

Posted by kl...@apache.org.
GEODE-2820: Added awaitlity clause to wait for the index size to be correct

	* Added awaitility clause to wait for all the indexes to have all the entries
	* This is done to make sure that the queries are executed after the indexes are ready.

	This closes #588


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 822946b9ddc426b45857c6b3919cd0b8b5a98406
Parents: 43b00bd
Author: nabarun <nn...@pivotal.io>
Authored: Thu Jun 15 16:32:46 2017 -0700
Committer: nabarun <nn...@pivotal.io>
Committed: Wed Jun 21 15:12:49 2017 -0700

----------------------------------------------------------------------
 .../dunit/QueryIndexUsingXMLDUnitTest.java      | 30 +++++++++++++++++---
 1 file changed, 26 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/822946b9/geode-core/src/test/java/org/apache/geode/cache/query/dunit/QueryIndexUsingXMLDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/cache/query/dunit/QueryIndexUsingXMLDUnitTest.java b/geode-core/src/test/java/org/apache/geode/cache/query/dunit/QueryIndexUsingXMLDUnitTest.java
index 83f1da3..e08b019 100644
--- a/geode-core/src/test/java/org/apache/geode/cache/query/dunit/QueryIndexUsingXMLDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/cache/query/dunit/QueryIndexUsingXMLDUnitTest.java
@@ -21,6 +21,7 @@ import static org.apache.geode.test.dunit.Invoke.*;
 import static org.apache.geode.test.dunit.LogWriterUtils.*;
 import static org.assertj.core.api.Assertions.*;
 import static org.awaitility.Awaitility.*;
+import static org.junit.Assert.assertEquals;
 
 import java.io.File;
 import java.net.URL;
@@ -28,8 +29,10 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.io.FileUtils;
+import org.awaitility.Awaitility;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Ignore;
@@ -473,11 +476,7 @@ public class QueryIndexUsingXMLDUnitTest extends JUnit4CacheTestCase {
 
   /**
    * Creates async indexes and compares the results between index and non-index results.
-   * <p>
-   * DISABLED. This test is disabled due to a high rate of throw new AssertionError. See ticket
-   * #52167
    */
-  @Ignore("TODO: test is disabled because of #52167")
   @Test
   public void testCreateAsyncIndexWhileDoingGIIAndCompareQueryResults() throws Exception {
     Host host = Host.getHost(0);
@@ -510,11 +509,34 @@ public class QueryIndexUsingXMLDUnitTest extends JUnit4CacheTestCase {
     vm1.invoke(prIndexCreationCheck(PERSISTENT_REG_NAME, "secIndex", 50));
     vm1.invoke(indexCreationCheck(REP_REG_NAME, "secIndex"));
 
+    vm0.invoke(() -> validateIndexSize());
+    vm1.invoke(() -> validateIndexSize());
+
+
     // Execute query and verify index usage
     vm0.invoke(executeQueryAndCompareResult(false));
     vm1.invoke(executeQueryAndCompareResult(false));
   }
 
+  public void validateIndexSize() {
+    Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
+      boolean indexSizeCheck_NAME = validateIndexSizeForRegion(NAME);
+      boolean indexSizeCheck_REP_REG_NAME = validateIndexSizeForRegion(REP_REG_NAME);
+      boolean indexSizeCheck_PERSISTENT_REG_NAME = validateIndexSizeForRegion(PERSISTENT_REG_NAME);
+      assertEquals("Index does not contain all the entries after 60 seconds have elapsed ", true,
+          (indexSizeCheck_NAME && indexSizeCheck_REP_REG_NAME
+              && indexSizeCheck_PERSISTENT_REG_NAME));
+    });
+  }
+
+  private boolean validateIndexSizeForRegion(final String regionName) {
+    Region region = getCache().getRegion(regionName);
+    QueryService queryService = getCache().getQueryService();
+    return queryService.getIndex(region, "statusIndex").getStatistics().getNumberOfValues() == 500
+        && queryService.getIndex(region, "idIndex").getStatistics().getNumberOfValues() == 500
+        && queryService.getIndex(region, "statusIndex").getStatistics().getNumberOfValues() == 500;
+  }
+
   @Test
   public void testIndexCreationForReplicatedPersistentOverFlowRegionOnRestart() throws Exception {
     Host host = Host.getHost(0);


[15/25] geode git commit: Merge branch 'feature/GEODE-3071' into develop

Posted by kl...@apache.org.
Merge branch 'feature/GEODE-3071' into develop


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 80667f30ef66d52987235374f3c95f6a39e0cd78
Parents: 8db7862 dba7dcd
Author: Jens Deppe <jd...@pivotal.io>
Authored: Mon Jun 19 14:22:22 2017 -0700
Committer: Jens Deppe <jd...@pivotal.io>
Committed: Mon Jun 19 14:22:22 2017 -0700

----------------------------------------------------------------------
 build.gradle         |   7 ++++
 gradle.properties    |   7 ++++
 gradle/docker.gradle | 100 ++++++++++++++++++++++++++++++++++++++++++++++
 3 files changed, 114 insertions(+)
----------------------------------------------------------------------



[06/25] geode git commit: GEODE-3072: Ignore dunit test

Posted by kl...@apache.org.
GEODE-3072: Ignore dunit test


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: dbc3197f3645f6beab058ad1bfe82e12860617c6
Parents: 4bf80a6
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Fri Jun 16 14:17:23 2017 -0700
Committer: Barry Oglesby <bo...@pivotal.io>
Committed: Fri Jun 16 14:17:23 2017 -0700

----------------------------------------------------------------------
 .../internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/dbc3197f/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java
index fa68781..5fb8fa2 100755
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/tier/sockets/ClientServerMiscBCDUnitTest.java
@@ -96,7 +96,7 @@ public class ClientServerMiscBCDUnitTest extends ClientServerMiscDUnitTest {
     });
   }
 
-  //@Test
+  // @Test
   @Ignore
   public void testDistributedMemberBytesWithCurrentServerAndOldClient() throws Exception {
     // Start current version server


[19/25] geode git commit: (no ticket) docs code cleanup - gfsh start command reference

Posted by kl...@apache.org.
(no ticket) docs code cleanup - gfsh start command reference


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 7df5af35ac885c007e2287316213360a12241c21
Parents: ee5f60c
Author: Dave Barnes <db...@pivotal.io>
Authored: Tue Jun 20 13:08:33 2017 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Tue Jun 20 13:08:33 2017 -0700

----------------------------------------------------------------------
 .../gfsh/command-pages/start.html.md.erb        | 21 +++++---------------
 1 file changed, 5 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/7df5af35/geode-docs/tools_modules/gfsh/command-pages/start.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/tools_modules/gfsh/command-pages/start.html.md.erb b/geode-docs/tools_modules/gfsh/command-pages/start.html.md.erb
index 0bec322..0044b9e 100644
--- a/geode-docs/tools_modules/gfsh/command-pages/start.html.md.erb
+++ b/geode-docs/tools_modules/gfsh/command-pages/start.html.md.erb
@@ -65,8 +65,6 @@ Note that you can only have one gateway receiver on each member, and unlike a ga
 start gateway-receiver [--group=value(,value)*] [--member=value]
 ```
 
-<a id="topic_67738A5B68E84DEE95D1C92DAB2E26E5__table_eyh_3vg_2w"></a>
-
 | Name                                           | Description                                                      |
 |------------------------------------------------|------------------------------------------------------------------|
 | <span class="keyword parmname">\\-\\-member</span> | Name or ID of the member on which to start the Gateway Receiver. |
@@ -117,8 +115,6 @@ For information on how to configure a gateway sender, see [Configure Gateway Sen
 start gateway-sender --id=value [--group=value(,value)*] [--member=value]
 ```
 
-<a id="topic_AB8BA3F42B9645A8BE9BD97CE2F839A8__table_gxc_lvg_2w"></a>
-
 | Name                                           | Description                                               |
 |------------------------------------------------|-----------------------------------------------------------|
 | <span class="keyword parmname">\\-\\-id</span>     | *Required.* ID of the GatewaySender.                      |
@@ -175,8 +171,6 @@ start jconsole [--interval=<seconds>] [--notile] [--version]
 [--J<jconsole JVM options>]
 ```
 
-<a id="topic_D00507416F3944DFAB48D2FA2B9E4A31__table_uck_4vg_2w"></a>
-
 | Name                                               | Description                                                                                                                                                                                                                                                                                                                                                                                                | Default Value |
 |----------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------|
 | <span class="keyword parmname">\\-\\-interval</span>   | Set the update interval to n seconds (default is 4 seconds). (Equivalent to JConsole's `-interval=n`)                                                                                                                                                                                                                                                                                                      | 4             |
@@ -238,10 +232,8 @@ Start the JDK's Java VisualVM monitoring application in a separate process.
 start jvisualvm [--J=value(,value)*]
 ```
 
-<a id="topic_5B5BF8BEE905463D8B7762B89E2D65E7__table_ssp_svg_2w"></a>
-
-| Name                                      | Description                                                                                                    |
-|-------------------------------------------|----------------------------------------------------------------------------------------------------------------|
+| Name                                      | Description |
+|-------------------------------------------|------------------------------------------------------------------------------------------------|
 | <span class="keyword parmname">\\-\\-J</span> | VM-option passed to the spawned CacheServer VM. For example: `-J-Dfoo.bar=true` for setting foo.bar to 'true'. |
 
 <span class="tablecap">Table 4. Start JVisalVM Parameters</span>
@@ -280,10 +272,7 @@ start locator --name=value [--bind-address=value] [--force(=value)] [--group=val
  [--J=value(,value)*]
 ```
 
-<a id="topic_591260CF25D64562A0EDD7260D2AC6D4__table_hly_vvg_2w"></a>
-
 <table>
-<caption><span class="tablecap">Table 5. Parameters</span></caption>
 <colgroup>
 <col width="33%" />
 <col width="33%" />
@@ -468,8 +457,6 @@ For more information on Geode Pulse, see [Geode Pulse](../../pulse/pulse-overvie
 start pulse [--url=value]
 ```
 
-<a id="topic_E906BA7D9E7F4C5890FEFA7ECD40DD77__table_cv3_w54_k4eee"></a>
-
 | Name                                        | Description                      | Default                       |
 |---------------------------------------------|----------------------------------|-------------------------------|
 | <span class="keyword parmname">\\-\\-url</span> | URL of the Pulse Web application | `http://localhost:7070/pulse` |
@@ -520,7 +507,6 @@ start server --name=value [--assign-buckets(=value)] [--bind-address=value]
 <a id="topic_3764EE2DB18B4AE4A625E0354471738A__table_at5_bwg_2w"></a>
 
 <table>
-<caption><span class="tablecap">Table 7. Parameters</span></caption>
 <colgroup>
 <col width="33%" />
 <col width="33%" />
@@ -795,6 +781,9 @@ the cluster.
 </tbody>
 </table>
 
+<span class="tablecap">Table 7. Start Server Parameters</span>
+
+
 ### <a id="topic_3764EE2DB18B4AE4A625E0354471738A__section_A50120613D3C4A848463236C4EFF18C7" class="no-quick-link"></a>Examples
 
 ``` pre


[04/25] geode git commit: GEODE-2301 Doc note to deprecate Geode JTA trans mgr

Posted by kl...@apache.org.
GEODE-2301 Doc note to deprecate Geode JTA trans mgr

    This closes #581


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: 42350f1ac3a5cac92f60284863a9d704097b5288
Parents: 5546a87
Author: Karen Miller <km...@pivotal.io>
Authored: Thu Jun 15 10:39:44 2017 -0700
Committer: Karen Miller <km...@pivotal.io>
Committed: Fri Jun 16 09:42:35 2017 -0700

----------------------------------------------------------------------
 geode-docs/developing/transactions/JTA_transactions.html.md.erb | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/42350f1a/geode-docs/developing/transactions/JTA_transactions.html.md.erb
----------------------------------------------------------------------
diff --git a/geode-docs/developing/transactions/JTA_transactions.html.md.erb b/geode-docs/developing/transactions/JTA_transactions.html.md.erb
index 3164dce..ffb6082 100644
--- a/geode-docs/developing/transactions/JTA_transactions.html.md.erb
+++ b/geode-docs/developing/transactions/JTA_transactions.html.md.erb
@@ -190,6 +190,7 @@ See [JCA Resource Adapter Example](jca_adapter_example.html#concept_swv_z2p_wk)
 ## <a id="concept_8567sdkbigige" class="no-quick-link"></a>Using Geode as the JTA Transaction Manager
 
 You can also use Geode as the JTA transaction manager.
+As of Geode 1.2, Geode's JTA transaction manager is deprecated.
 
 Geode ships with its own implementation of a JTA transaction manager. However, note that this implementation is not XA-compliant; therefore, it does not persist any state, which could lead to an inconsistent state after recovering a crashed member.
 


[18/25] geode git commit: GEODE-3095: fix parameter type mismatch between the diskstore command and controller

Posted by kl...@apache.org.
GEODE-3095: fix parameter type mismatch between the diskstore command and controller


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

Branch: refs/heads/feature/GEODE-3062-2
Commit: ee5f60c7fdac7227983a7e3bfac7094593e15ed0
Parents: d8a11d1
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon Jun 19 11:02:20 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Jun 19 21:18:59 2017 -0700

----------------------------------------------------------------------
 .../web/controllers/DiskStoreCommandsController.java | 11 ++++++-----
 .../management/internal/security/TestCommand.java    | 15 +++++++++------
 2 files changed, 15 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/ee5f60c7/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DiskStoreCommandsController.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DiskStoreCommandsController.java b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DiskStoreCommandsController.java
index c613a8a..2e06811 100644
--- a/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DiskStoreCommandsController.java
+++ b/geode-core/src/main/java/org/apache/geode/management/internal/web/controllers/DiskStoreCommandsController.java
@@ -14,9 +14,8 @@
  */
 package org.apache.geode.management.internal.web.controllers;
 
-import org.apache.geode.internal.lang.StringUtils;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import java.util.concurrent.Callable;
+
 import org.springframework.http.ResponseEntity;
 import org.springframework.stereotype.Controller;
 import org.springframework.web.bind.annotation.PathVariable;
@@ -25,7 +24,9 @@ import org.springframework.web.bind.annotation.RequestMethod;
 import org.springframework.web.bind.annotation.RequestParam;
 import org.springframework.web.bind.annotation.ResponseBody;
 
-import java.util.concurrent.Callable;
+import org.apache.geode.internal.lang.StringUtils;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
 
 /**
  * The DiskStoreCommandsController class implements GemFire Management REST API web service
@@ -107,7 +108,7 @@ public class DiskStoreCommandsController extends AbstractCommandsController {
       @RequestParam(value = CliStrings.CREATE_DISK_STORE__DISK_USAGE_WARNING_PCT,
           defaultValue = "90") final Float diskUsageWarningPercentage,
       @RequestParam(value = CliStrings.CREATE_DISK_STORE__DISK_USAGE_CRITICAL_PCT,
-          defaultValue = "99") final Integer diskUsageCriticalPercentage,
+          defaultValue = "99") final Float diskUsageCriticalPercentage,
       @RequestParam(value = CliStrings.GROUP, required = false) final String[] groups) {
     CommandStringBuilder command = new CommandStringBuilder(CliStrings.CREATE_DISK_STORE);
 

http://git-wip-us.apache.org/repos/asf/geode/blob/ee5f60c7/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
index ea4b60d..800d0da 100644
--- a/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
+++ b/geode-core/src/test/java/org/apache/geode/management/internal/security/TestCommand.java
@@ -14,14 +14,16 @@
  */
 package org.apache.geode.management.internal.security;
 
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.shiro.authz.Permission;
+
 import org.apache.geode.security.ResourcePermission;
-import org.apache.geode.security.ResourcePermission.Target;
 import org.apache.geode.security.ResourcePermission.Operation;
 import org.apache.geode.security.ResourcePermission.Resource;
-import org.apache.shiro.authz.Permission;
-
-import java.util.ArrayList;
-import java.util.List;
+import org.apache.geode.security.ResourcePermission.Target;
 
 public class TestCommand {
 
@@ -77,7 +79,8 @@ public class TestCommand {
   }
 
   public static List<TestCommand> getCommands() {
-    return testCommands;
+    // returns a copy of the list every time
+    return testCommands.stream().collect(Collectors.toList());
   }
 
   public static List<TestCommand> getPermittedCommands(Permission permission) {