You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2016/03/18 20:48:35 UTC

[1/4] incubator-geode git commit: GEODE-934: CI failure: DurableClientSimpleDUnitTest.testReadyForEventsNotCalledImplicitlyWithCacheXML.

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-1050 636951e98 -> 072f66946


GEODE-934: CI failure: DurableClientSimpleDUnitTest.testReadyForEventsNotCalledImplicitlyWithCacheXML.

In this test the client sees previously delivered events, when it disconnects and reconnects immediately.
This happens when the message dispatcher is in the process of removing the messages while its in the pause state; and client reconnects at the same time. When client reconnects, it clears out the acked-event list, this causes the message dispatcher thread not to remove dispatched event from the queue. The fix is to take the pause lock and start removing the dispatched event, this blocks the client reconnect process to wait tille the messages are removed.

Testing:
The test used to fail once in 4 runs. With the change it run for 50 times.
precheckin


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

Branch: refs/heads/feature/GEODE-1050
Commit: be74e3a1e9ea0699743dd7ec9e63774402d3542e
Parents: 8c690ac
Author: agingade <ag...@pivotal.io>
Authored: Tue Mar 15 15:31:37 2016 -0700
Committer: agingade <ag...@pivotal.io>
Committed: Fri Mar 18 11:23:55 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/CacheClientProxy.java    | 25 ++++++++++----------
 1 file changed, 12 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/be74e3a1/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
index 6edb570..d6226b5 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientProxy.java
@@ -2582,21 +2582,20 @@ public class CacheClientProxy implements ClientSession {
         try {
           // If paused, wait to be told to resume (or interrupted if stopped)
           if (getProxy().isPaused()) {
-            try {
-              // ARB: Before waiting for resumption, process acks from client. 
-              // This will reduce the number of duplicates that a client receives after
-              // reconnecting.
-              if (this._messageQueue.size() > 0) {
-                Thread.sleep(50);
-              }
-              logger.info("available ids = " + this._messageQueue.size()+ " , isEmptyAckList =" + this._messageQueue.isEmptyAckList() 
+            // ARB: Before waiting for resumption, process acks from client. 
+            // This will reduce the number of duplicates that a client receives after
+            // reconnecting.
+            synchronized (_pausedLock) {
+              try {
+                logger.info("available ids = " + this._messageQueue.size()+ " , isEmptyAckList =" + this._messageQueue.isEmptyAckList() 
                             + ", peekInitialized = " + this._messageQueue.isPeekInitialized());
-              while (!this._messageQueue.isEmptyAckList()&& this._messageQueue.isPeekInitialized()) {
-                this._messageQueue.remove();
+                while (!this._messageQueue.isEmptyAckList()&& this._messageQueue.isPeekInitialized()) {
+                  this._messageQueue.remove();
+                }
+              }
+              catch (InterruptedException ex) {
+                logger.warn(LocalizedMessage.create(LocalizedStrings.CacheClientProxy_0_SLEEP_INTERRUPTED, this));
               }
-            }
-            catch (InterruptedException ex) {
-              logger.warn(LocalizedMessage.create(LocalizedStrings.CacheClientProxy_0_SLEEP_INTERRUPTED, this));
             }
             waitForResumption();
           }


[2/4] incubator-geode git commit: GEODE-1112: Fix templates.security to com.gemstone.gemfire.security.templates

Posted by kl...@apache.org.
GEODE-1112: Fix templates.security to com.gemstone.gemfire.security.templates

* change hardcoded string reflection for security templates to use class.getName()
* fixup javadocs
* fixup spark md with updated package


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

Branch: refs/heads/feature/GEODE-1050
Commit: 8c74d600bf9ec5c777ce25c1e993821fbb3f6d0f
Parents: be74e3a
Author: Kirk Lund <kl...@pivotal.io>
Authored: Fri Mar 18 12:28:57 2016 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Fri Mar 18 12:42:03 2016 -0700

----------------------------------------------------------------------
 .../QueryParamsAuthorizationDUnitTest.java      | 14 ++++++------
 ...ributedRegionFunctionExecutionDUnitTest.java |  8 +++----
 .../security/P2PAuthenticationDUnitTest.java    | 24 ++++++++------------
 .../security/templates/PKCSAuthInit.java        |  2 +-
 .../templates/UserPasswordAuthInit.java         |  4 ++--
 .../cache/query/cq/dunit/CqStateDUnitTest.java  | 16 ++++++-------
 geode-spark-connector/doc/3_connecting.md       |  6 ++---
 7 files changed, 33 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c74d600/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryParamsAuthorizationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryParamsAuthorizationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryParamsAuthorizationDUnitTest.java
index 5984576..5748e17 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryParamsAuthorizationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/QueryParamsAuthorizationDUnitTest.java
@@ -16,8 +16,6 @@
  */
 package com.gemstone.gemfire.cache.query.dunit;
 
-import org.junit.Ignore;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.Region;
@@ -31,10 +29,13 @@ import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache30.CacheTestCase;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.security.templates.DummyAuthenticator;
+import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.dunit.SerializableCallable;
 import com.gemstone.gemfire.test.dunit.VM;
+import org.junit.Ignore;
 
 /**
  * Test for accessing query bind parameters from authorization callbacks
@@ -67,8 +68,7 @@ public class QueryParamsAuthorizationDUnitTest extends CacheTestCase {
             .set("mcast-port", "0")
             .set("security-client-accessor",
                 "com.gemstone.gemfire.cache.query.dunit.QueryAuthorization.create")
-            .set("security-client-authenticator",
-                "templates.security.DummyAuthenticator.create");
+            .set("security-client-authenticator", DummyAuthenticator.class.getName() + ".create");
         Cache cache = getCache(cf);
         cache.createRegionFactory(RegionShortcut.REPLICATE).create(regName);
         CacheServer server = cache.addCacheServer();
@@ -85,9 +85,9 @@ public class QueryParamsAuthorizationDUnitTest extends CacheTestCase {
       public Object call() throws Exception {
         ClientCacheFactory ccf = new ClientCacheFactory()
             .addPoolServer(NetworkUtils.getServerHostName(server1.getHost()), port)
-            .set("security-client-auth-init",
-                "templates.security.UserPasswordAuthInit.create")
-            .set("security-username", "root").set("security-password", "root");
+            .set("security-client-auth-init", UserPasswordAuthInit.class.getName() + ".create")
+            .set("security-username", "root")
+            .set("security-password", "root");
 
         ClientCache cache = getClientCache(ccf);
         Region r1 = cache.createClientRegionFactory(

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c74d600/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/DistributedRegionFunctionExecutionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/DistributedRegionFunctionExecutionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/DistributedRegionFunctionExecutionDUnitTest.java
index d1437f5..56d94e0 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/DistributedRegionFunctionExecutionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/DistributedRegionFunctionExecutionDUnitTest.java
@@ -52,6 +52,8 @@ import com.gemstone.gemfire.internal.cache.functions.DistribuedRegionFunctionFun
 import com.gemstone.gemfire.internal.cache.functions.DistributedRegionFunction;
 import com.gemstone.gemfire.internal.cache.functions.TestFunction;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
+import com.gemstone.gemfire.security.templates.DummyAuthenticator;
+import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
 import com.gemstone.gemfire.test.dunit.Assert;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
@@ -909,8 +911,7 @@ public class DistributedRegionFunctionExecutionDUnitTest extends
   public static void createCacheInVm_41367() {
     Properties props = new Properties();
     props.put(DistributionConfig.NAME_NAME, "SecurityServer");
-    props.put("security-client-authenticator",
-        "templates.security.DummyAuthenticator.create");
+    props.put("security-client-authenticator", DummyAuthenticator.class.getName() + ".create");
     new DistributedRegionFunctionExecutionDUnitTest("temp").createCache(props);
   }
 
@@ -927,8 +928,7 @@ public class DistributedRegionFunctionExecutionDUnitTest extends
     props.put("mcast-port", "0");
     props.put("locators", "");
     props.put(DistributionConfig.NAME_NAME, "SecurityClient");
-    props.put("security-client-auth-init",
-        "templates.security.UserPasswordAuthInit.create");
+    props.put("security-client-auth-init", UserPasswordAuthInit.class.getName() + ".create");
     props.put("security-username", "reader1");
     props.put("security-password", "reader1");
     new DistributedRegionFunctionExecutionDUnitTest("temp").createCache(props);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c74d600/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
index 3af4e14..560b9e4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
@@ -1,6 +1,3 @@
-
-package com.gemstone.gemfire.security;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -9,9 +6,9 @@ package com.gemstone.gemfire.security;
  * 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
@@ -19,7 +16,7 @@ package com.gemstone.gemfire.security;
  * specific language governing permissions and limitations
  * under the License.
  */
-
+package com.gemstone.gemfire.security;
 
 import java.io.File;
 import java.util.Properties;
@@ -38,6 +35,8 @@ import com.gemstone.gemfire.security.generator.CredentialGenerator;
 import com.gemstone.gemfire.security.generator.DummyCredentialGenerator;
 import com.gemstone.gemfire.security.generator.LdapUserCredentialGenerator;
 import com.gemstone.gemfire.security.generator.UserPasswordWithExtraPropsAuthInit;
+import com.gemstone.gemfire.security.templates.LdapUserAuthenticator;
+import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
 import com.gemstone.gemfire.test.dunit.DistributedTestCase;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
@@ -46,7 +45,6 @@ import com.gemstone.gemfire.test.dunit.NetworkUtils;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 
-
 /**
  * Tests peer to peer authentication in Gemfire
  * 
@@ -100,8 +98,7 @@ public class P2PAuthenticationDUnitTest extends DistributedTestCase {
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "26753");
     props.setProperty(DistributionConfig.LOCATORS_NAME, 
                       NetworkUtils.getIPLiteral() + "[" + port + "]");
-    props.setProperty(DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
-        "templates.security.UserPasswordAuthInit.create");
+    props.setProperty(DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME, UserPasswordAuthInit.class.getName() + ".create");
     props.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
 
     try {
@@ -117,8 +114,7 @@ public class P2PAuthenticationDUnitTest extends DistributedTestCase {
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "26753");
     props.setProperty(DistributionConfig.LOCATORS_NAME, 
                       NetworkUtils.getIPLiteral() +"[" + port + "]");
-    props.setProperty(DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
-        "templates.security.LdapUserAuthenticator.create");
+    props.setProperty(DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME, LdapUserAuthenticator.class.getName() + ".create");
     props.setProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false");
     try {
       Locator.startLocatorAndDS(port, logFile, null, props);
@@ -130,8 +126,7 @@ public class P2PAuthenticationDUnitTest extends DistributedTestCase {
 
     props = new Properties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "26753");
-    props.setProperty(DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME,
-        "templates.security.UserPasswordAuthInit.create");
+    props.setProperty(DistributionConfig.SECURITY_PEER_AUTH_INIT_NAME, UserPasswordAuthInit.class.getName() + ".create");
     try {
       getSystem(props);
       fail("Expected an IllegalArgumentException while connection to DS");
@@ -143,8 +138,7 @@ public class P2PAuthenticationDUnitTest extends DistributedTestCase {
     // Also try setting the authenticator
     props = new Properties();
     props.setProperty(DistributionConfig.MCAST_PORT_NAME, "26753");
-    props.setProperty(DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME,
-        "templates.security.LdapUserAuthenticator.create");
+    props.setProperty(DistributionConfig.SECURITY_PEER_AUTHENTICATOR_NAME, LdapUserAuthenticator.class.getName() + ".create");
     try {
       getSystem(props);
       fail("Expected an IllegalArgumentException while connection to DS");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c74d600/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthInit.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthInit.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthInit.java
index a60c8fb..0b38a7b 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthInit.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/PKCSAuthInit.java
@@ -37,7 +37,7 @@ import org.apache.logging.log4j.Logger;
  * 
  * To use this class the {@code security-client-auth-init} property should be
  * set to the fully qualified name the static {@code create} function
- * viz. <{@code templates.security.PKCSAuthInit.create}
+ * viz. {@code com.gemstone.gemfire.security.templates.PKCSAuthInit.create}
  * 
  * @since 5.5
  */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c74d600/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UserPasswordAuthInit.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UserPasswordAuthInit.java b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UserPasswordAuthInit.java
index 312f18b..34161ac 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UserPasswordAuthInit.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/templates/UserPasswordAuthInit.java
@@ -28,8 +28,8 @@ import java.util.Properties;
  * password as the credentials from the given set of properties.
  * 
  * To use this class the {@code security-client-auth-init} property should be
- * set to the fully qualified name the static {@code create} function
- * viz. {@code templates.security.UserPasswordAuthInit.create}
+ * set to the fully qualified name the static {@code create} method
+ * viz. {@code com.gemstone.gemfire.security.templates.UserPasswordAuthInit.create}
  * 
  * @since 5.5
  */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c74d600/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStateDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStateDUnitTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStateDUnitTest.java
index 7c1e8f2..0165fbf 100644
--- a/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStateDUnitTest.java
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/cache/query/cq/dunit/CqStateDUnitTest.java
@@ -19,8 +19,11 @@ package com.gemstone.gemfire.cache.query.cq.dunit;
 import java.util.Properties;
 
 import com.gemstone.gemfire.cache.query.CqQuery;
+import com.gemstone.gemfire.cache.query.dunit.CloseCacheAuthorization;
 import com.gemstone.gemfire.cache.query.dunit.HelperTestCase;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.security.templates.DummyAuthenticator;
+import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.IgnoredException;
@@ -109,12 +112,9 @@ public class CqStateDUnitTest extends HelperTestCase {
   public Properties getAuthenticatedServerProperties() {
     Properties props = new Properties();
     props.put("mcast-port", "0");
-    props.put("security-client-accessor",
-        "com.gemstone.gemfire.cache.query.dunit.CloseCacheAuthorization.create");
-    props.put("security-client-accessor-pp",
-        "com.gemstone.gemfire.cache.query.dunit.CloseCacheAuthorization.create");
-    props.put("security-client-authenticator",
-        "templates.security.DummyAuthenticator.create");
+    props.put("security-client-accessor", CloseCacheAuthorization.class.getName() + ".create");
+    props.put("security-client-accessor-pp", CloseCacheAuthorization.class.getName() + ".create");
+    props.put("security-client-authenticator", DummyAuthenticator.class.getName() + ".create");
     return props;
   }
   
@@ -126,9 +126,7 @@ public class CqStateDUnitTest extends HelperTestCase {
   
   public Properties getClientProperties() {
     Properties props = new Properties();
-    props.put("security-client-auth-init",
-        "templates.security.UserPasswordAuthInit.create");
-    
+    props.put("security-client-auth-init", UserPasswordAuthInit.class.getName() + ".create");
     props.put("security-username", "root");
     props.put("security-password", "root");
     return props;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8c74d600/geode-spark-connector/doc/3_connecting.md
----------------------------------------------------------------------
diff --git a/geode-spark-connector/doc/3_connecting.md b/geode-spark-connector/doc/3_connecting.md
index 8428657..8972be9 100644
--- a/geode-spark-connector/doc/3_connecting.md
+++ b/geode-spark-connector/doc/3_connecting.md
@@ -14,7 +14,7 @@ application code. In the following examples, we assume you want to provide
 In `<spark dir>/conf/spark-defaults.com`
 ```
 spark.gemfire.locators=192.168.1.47[10334]
-spark.gemfire.security-client-auth-init=templates.security.UserPasswordAuthInit.create
+spark.gemfire.security-client-auth-init=com.gemstone.gemfire.security.templates.UserPasswordAuthInit.create
 spark.gemfire.security-username=scott
 spark.gemfire.security-password=tiger
 ```
@@ -24,7 +24,7 @@ Or in the Spark application code:
 import io.pivotal.gemfire.spark.connector._
 val sparkConf = new SparkConf()
   .set(GemFireLocatorPropKey, "192.168.1.47[10334]")
-  .set("spark.gemfire.security-client-auth-init", "templates.security.UserPasswordAuthInit.create")
+  .set("spark.gemfire.security-client-auth-init", "com.gemstone.gemfire.security.templates.UserPasswordAuthInit.create")
   .set("spark.gemfire.security-username", "scott")
   .set("spark.gemfire.security-password", "tiger")
 ```
@@ -35,7 +35,7 @@ After this, you can use all connector APIs without providing `GemfireConnectionC
 Here's the code that creates `GemFireConnectionConf` with the same set of 
 properties as the examples above:
 ```
-val props = Map("security-client-auth-init" -> "templates.security.UserPasswordAuthInit.create",
+val props = Map("security-client-auth-init" -> "com.gemstone.gemfire.security.templates.UserPasswordAuthInit.create",
                 "security-username" -> "scott",
                 "security-password" -> "tiger")
 val connConf = GemFireConnectionConf("192.168.1.47[10334]", props)


[3/4] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-1050

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


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

Branch: refs/heads/feature/GEODE-1050
Commit: 9b8cfd505246541f42b23a898616933ce0a878c8
Parents: 636951e 8c74d60
Author: Kirk Lund <kl...@apache.org>
Authored: Fri Mar 18 12:44:40 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Mar 18 12:44:40 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/CacheClientProxy.java    | 25 ++++++++++----------
 .../QueryParamsAuthorizationDUnitTest.java      | 14 +++++------
 ...ributedRegionFunctionExecutionDUnitTest.java |  8 +++----
 .../security/P2PAuthenticationDUnitTest.java    | 24 +++++++------------
 .../security/templates/PKCSAuthInit.java        |  2 +-
 .../templates/UserPasswordAuthInit.java         |  4 ++--
 .../cache/query/cq/dunit/CqStateDUnitTest.java  | 16 ++++++-------
 geode-spark-connector/doc/3_connecting.md       |  6 ++---
 8 files changed, 45 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9b8cfd50/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/execute/DistributedRegionFunctionExecutionDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9b8cfd50/geode-core/src/test/java/com/gemstone/gemfire/security/P2PAuthenticationDUnitTest.java
----------------------------------------------------------------------


[4/4] incubator-geode git commit: Mark overridden method with @Override. Remove unused @After.

Posted by kl...@apache.org.
Mark overridden method with @Override. Remove unused @After.


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

Branch: refs/heads/feature/GEODE-1050
Commit: 072f66946a9fb6a3df480c54256dbb368bfb236e
Parents: 9b8cfd5
Author: Kirk Lund <kl...@apache.org>
Authored: Fri Mar 18 12:48:06 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Mar 18 12:48:06 2016 -0700

----------------------------------------------------------------------
 .../internal/BackwardCompatibilitySerializationDUnitTest.java      | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/072f6694/geode-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
index d85b5df..3197dc0 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/BackwardCompatibilitySerializationDUnitTest.java
@@ -69,7 +69,7 @@ public class BackwardCompatibilitySerializationDUnitTest extends CacheTestCase {
         TestMessage.class);
   }
 
-  @After
+  @Override
   public final void preTearDownCacheTestCase() {
     resetFlags();
     // reset the class mapped to the dsfid