You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2016/05/04 22:57:19 UTC

[19/63] [abbrv] incubator-geode git commit: GEODE-1258: Added tests for WAN authentication.

GEODE-1258: Added tests for WAN authentication.

* Created tests for checking WAN authentication using valid and invalid credentials.

This closes #131


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

Branch: refs/heads/feature/GEODE-1276
Commit: 6b4cdb1c46cd5c72a074fb908c9ed791e7219677
Parents: 46535f2
Author: nabarun <nn...@pivotal.io>
Authored: Tue Apr 19 14:15:10 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Wed Apr 27 15:57:50 2016 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/wan/WANTestBase.java |  17 +
 .../wan/misc/NewWanAuthenticationDUnitTest.java | 309 +++++++++++++++++++
 2 files changed, 326 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6b4cdb1c/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/WANTestBase.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/WANTestBase.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/WANTestBase.java
index 39154b2..09ec3e1 100644
--- a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/WANTestBase.java
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/WANTestBase.java
@@ -356,6 +356,23 @@ public class WANTestBase extends DistributedTestCase{
     return port;
   }
 
+  public static int createReceiverInSecuredCache() {
+    GatewayReceiverFactory fact = WANTestBase.cache.createGatewayReceiverFactory();
+    int port = AvailablePortHelper.getRandomAvailablePortForDUnitSite();
+    fact.setStartPort(port);
+    fact.setEndPort(port);
+    fact.setManualStart(true);
+    GatewayReceiver receiver = fact.create();
+    try {
+      receiver.start();
+    }
+    catch (IOException e) {
+      e.printStackTrace();
+      com.gemstone.gemfire.test.dunit.Assert.fail("Failed to start GatewayRecevier on port " + port, e);
+    }
+    return port;
+  }
+
   public static void createReplicatedRegion(String regionName, String senderIds, Boolean offHeap){
     IgnoredException exp = IgnoredException.addIgnoredException(ForceReattemptException.class
         .getName());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6b4cdb1c/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/NewWanAuthenticationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/NewWanAuthenticationDUnitTest.java b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/NewWanAuthenticationDUnitTest.java
new file mode 100644
index 0000000..f0303b9
--- /dev/null
+++ b/geode-wan/src/test/java/com/gemstone/gemfire/internal/cache/wan/misc/NewWanAuthenticationDUnitTest.java
@@ -0,0 +1,309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.internal.cache.wan.misc;
+
+import java.util.Properties;
+
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.security.AuthInitialize;
+import com.gemstone.gemfire.security.AuthenticationFailedException;
+import com.gemstone.gemfire.security.SecurityTestUtils;
+import com.gemstone.gemfire.security.generator.CredentialGenerator;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.DistributedSystem;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.cache.wan.WANTestBase;
+
+import com.gemstone.gemfire.security.generator.DummyCredentialGenerator;
+import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
+
+import org.apache.logging.log4j.Logger;
+
+public class NewWanAuthenticationDUnitTest extends WANTestBase {
+
+  private static final long serialVersionUID = 1L;
+
+  public static final Logger logger = LogService.getLogger();
+
+  public NewWanAuthenticationDUnitTest(String name) {
+    super(name);
+  }
+
+  /**
+   * Authentication test for new WAN with valid credentials. Although, nothing
+   * related to authentication has been changed in new WAN, this test case is
+   * added on request from QA for defect 44650.
+   */
+  public void testWanAuthValidCredentials() {
+    Integer lnPort = (Integer)vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId( 1 ));
+    logger.info("Created locator on local site");
+
+    Integer nyPort = (Integer)vm1.invoke(() -> WANTestBase.createFirstRemoteLocator( 2, lnPort ));
+    logger.info("Created locator on remote site");
+
+
+    CredentialGenerator gen = new DummyCredentialGenerator();
+    Properties extraProps = gen.getSystemProperties();
+
+    String clientauthenticator = gen.getAuthenticator();
+    String clientauthInit = gen.getAuthInit();
+
+    Properties credentials1 = gen.getValidCredentials(1);
+    if (extraProps != null) {
+      credentials1.putAll(extraProps);
+    }
+    Properties javaProps1 = gen.getJavaProperties();
+
+    Properties credentials2 = gen.getValidCredentials(2);
+    if (extraProps != null) {
+      credentials2.putAll(extraProps);
+    }
+    Properties javaProps2 = gen.getJavaProperties();
+
+    Properties props1 = buildProperties(clientauthenticator, clientauthInit,
+      null, credentials1, null);
+    Properties props2 = buildProperties(clientauthenticator, clientauthInit,
+      null, credentials2, null);
+
+    vm2.invoke(() -> NewWanAuthenticationDUnitTest.createSecuredCache(
+      props1, javaProps1, lnPort ));
+    logger.info("Created secured cache in vm2");
+
+    vm3.invoke(() -> NewWanAuthenticationDUnitTest.createSecuredCache(
+      props2, javaProps2, nyPort ));
+    logger.info("Created secured cache in vm3");
+
+    vm2.invoke(() -> WANTestBase.createSender( "ln", 2,
+      false, 100, 10, false, false, null, true ));
+    logger.info("Created sender in vm2");
+
+    vm3.invoke(() -> createReceiverInSecuredCache());
+    logger.info("Created receiver in vm3");
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(
+      getTestMethodName() + "_RR", "ln", isOffHeap()  ));
+    logger.info("Created RR in vm2");
+    vm3.invoke(() -> WANTestBase.createReplicatedRegion(
+      getTestMethodName() + "_RR", null, isOffHeap()  ));
+    logger.info("Created RR in vm3");
+
+    vm2.invoke(() -> WANTestBase.startSender( "ln" ));
+    vm2.invoke(() -> WANTestBase.waitForSenderRunningState( "ln" ));
+    logger.info("Done successfully.");
+
+  }
+
+  /**
+   * Test authentication with new WAN with invalid credentials. Although,
+   * nothing related to authentication has been changed in new WAN, this test
+   * case is added on request from QA for defect 44650.
+   */
+  public void testWanAuthInvalidCredentials() {
+    Integer lnPort = (Integer)vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId( 1 ));
+    logger.info("Created locator on local site");
+
+    Integer nyPort = (Integer)vm1.invoke(() -> WANTestBase.createFirstRemoteLocator( 2, lnPort ));
+    logger.info("Created locator on remote site");
+
+
+    CredentialGenerator gen = new DummyCredentialGenerator();
+    logger.info("Picked up credential: " + gen);
+
+    Properties extraProps = gen.getSystemProperties();
+
+    String clientauthenticator = gen.getAuthenticator();
+    String clientauthInit = gen.getAuthInit();
+
+    Properties credentials1 = gen.getInvalidCredentials(1);
+    if (extraProps != null) {
+      credentials1.putAll(extraProps);
+    }
+    Properties javaProps1 = gen.getJavaProperties();
+    Properties credentials2 = gen.getInvalidCredentials(2);
+    if (extraProps != null) {
+      credentials2.putAll(extraProps);
+    }
+    Properties javaProps2 = gen.getJavaProperties();
+
+    Properties props1 = buildProperties(clientauthenticator, clientauthInit,
+      null, credentials1, null);
+    Properties props2 = buildProperties(clientauthenticator, clientauthInit,
+      null, credentials2, null);
+
+    logger.info("Done building auth properties");
+
+    vm2.invoke(() -> NewWanAuthenticationDUnitTest.createSecuredCache(
+      props1, javaProps1, lnPort ));
+    logger.info("Created secured cache in vm2");
+
+    vm3.invoke(() -> NewWanAuthenticationDUnitTest.createSecuredCache(
+      props2, javaProps2, nyPort ));
+    logger.info("Created secured cache in vm3");
+
+    vm2.invoke(() -> WANTestBase.createSender( "ln", 2,
+      false, 100, 10, false, false, null, true ));
+    logger.info("Created sender in vm2");
+
+    vm3.invoke(() -> createReceiverInSecuredCache());
+    logger.info("Created receiver in vm3");
+
+    vm2.invoke(() -> WANTestBase.createReplicatedRegion(
+      getTestMethodName() + "_RR", "ln", isOffHeap()  ));
+    logger.info("Created RR in vm2");
+    vm3.invoke(() -> WANTestBase.createReplicatedRegion(
+      getTestMethodName() + "_RR", null, isOffHeap()  ));
+    logger.info("Created RR in vm3");
+
+    try {
+      vm2.invoke(() -> WANTestBase.startSender( "ln" ));
+      fail("Authentication Failed: While starting the sender, an exception should have been thrown");
+    } catch (Exception e) {
+      if (!(e.getCause().getCause() instanceof AuthenticationFailedException)) {
+        fail("Authentication is not working as expected");
+      }
+    }
+  }
+
+  private static Properties buildProperties(String clientauthenticator,
+                                            String clientAuthInit, String accessor, Properties extraAuthProps,
+                                            Properties extraAuthzProps) {
+
+    Properties authProps = new Properties();
+    if (clientauthenticator != null) {
+      authProps.setProperty(
+        DistributionConfig.SECURITY_CLIENT_AUTHENTICATOR_NAME,
+        clientauthenticator);
+    }
+    if (accessor != null) {
+      authProps.setProperty(DistributionConfig.SECURITY_CLIENT_ACCESSOR_NAME,
+        accessor);
+    }
+    if (clientAuthInit != null) {
+      authProps.setProperty(DistributionConfig.SECURITY_CLIENT_AUTH_INIT_NAME,
+        clientAuthInit);
+    }
+    if (extraAuthProps != null) {
+      authProps.putAll(extraAuthProps);
+    }
+    if (extraAuthzProps != null) {
+      authProps.putAll(extraAuthzProps);
+    }
+    return authProps;
+  }
+
+  public static void createSecuredCache(Properties authProps, Object javaProps, Integer locPort) {
+    authProps.setProperty(DistributionConfig.MCAST_PORT_NAME, "0");
+    authProps.setProperty(DistributionConfig.LOCATORS_NAME, "localhost[" + locPort + "]");
+
+    logger.info("Set the server properties to: " + authProps);
+    logger.info("Set the java properties to: " + javaProps);
+
+    SecurityTestUtils tmpInstance = new SecurityTestUtils("temp");
+    DistributedSystem ds = tmpInstance.createSystem(authProps, (Properties)javaProps);
+    assertNotNull(ds);
+    assertTrue(ds.isConnected());
+    cache = CacheFactory.create(ds);
+    assertNotNull(cache);
+  }
+
+  public static boolean isDifferentServerInGetCredentialCall = false;
+  public static class UserPasswdAI extends UserPasswordAuthInit {
+    public static AuthInitialize createAI() {
+      return new UserPasswdAI();
+    }
+    @Override
+    public Properties getCredentials(Properties props,
+                                     DistributedMember server, boolean isPeer)
+      throws AuthenticationFailedException {
+      boolean val = ( CacheFactory.getAnyInstance().getDistributedSystem().getDistributedMember().getProcessId() != server.getProcessId());
+      Assert.assertTrue(val, "getCredentials: Server should be different");
+      Properties p = super.getCredentials(props, server, isPeer);
+      if(val) {
+        isDifferentServerInGetCredentialCall = true;
+        CacheFactory.getAnyInstance().getLoggerI18n().convertToLogWriter().config("setting  isDifferentServerInGetCredentialCall " + isDifferentServerInGetCredentialCall);
+      } else {
+        CacheFactory.getAnyInstance().getLoggerI18n().convertToLogWriter().config("setting22  isDifferentServerInGetCredentialCall " + isDifferentServerInGetCredentialCall);
+      }
+      return p;
+    }
+  }
+
+  public static void verifyDifferentServerInGetCredentialCall(){
+    Assert.assertTrue(isDifferentServerInGetCredentialCall, "verifyDifferentServerInGetCredentialCall: Server should be different");
+    isDifferentServerInGetCredentialCall = false;
+  }
+
+  public void testWanAuthValidCredentialsWithServer() {
+    disconnectAllFromDS();
+    {
+      Integer lnPort = (Integer)vm0.invoke(() -> WANTestBase.createFirstLocatorWithDSId( 1 ));
+      logger.info("Created locator on local site");
+
+      Integer nyPort = (Integer)vm1.invoke(() -> WANTestBase.createFirstRemoteLocator( 2, lnPort ));
+      logger.info("Created locator on remote site");
+
+      DummyCredentialGenerator gen = new DummyCredentialGenerator();
+      gen.init();
+      Properties extraProps = gen.getSystemProperties();
+
+      String clientauthenticator = gen.getAuthenticator();
+      String clientauthInit = UserPasswdAI.class.getName() + ".createAI";
+
+      Properties credentials1 = gen.getValidCredentials(1);
+      if (extraProps != null) {
+        credentials1.putAll(extraProps);
+      }
+      Properties javaProps1 = gen.getJavaProperties();
+
+      Properties credentials2 = gen.getValidCredentials(2);
+      if (extraProps != null) {
+        credentials2.putAll(extraProps);
+      }
+      Properties javaProps2 = gen.getJavaProperties();
+
+      Properties props1 = buildProperties(clientauthenticator, clientauthInit,
+        null, credentials1, null);
+      Properties props2 = buildProperties(clientauthenticator, clientauthInit,
+        null, credentials2, null);
+
+      vm2.invoke(() -> NewWanAuthenticationDUnitTest.createSecuredCache(
+        props1, javaProps1, lnPort ));
+      logger.info("Created secured cache in vm2");
+
+      vm3.invoke(() -> NewWanAuthenticationDUnitTest.createSecuredCache(
+        props2, javaProps2, nyPort ));
+      logger.info("Created secured cache in vm3");
+
+      vm2.invoke(() -> WANTestBase.createSender( "ln", 2,
+        false, 100, 10, false, false, null, true ));
+      logger.info("Created sender in vm2");
+
+      vm3.invoke(() -> createReceiverInSecuredCache());
+      logger.info("Created receiver in vm3");
+
+      vm2.invoke(() -> WANTestBase.startSender( "ln" ));
+      vm2.invoke(() -> WANTestBase.waitForSenderRunningState( "ln" ));
+
+      vm2.invoke(() -> verifyDifferentServerInGetCredentialCall());
+      vm3.invoke(() -> verifyDifferentServerInGetCredentialCall());
+
+    }
+  }
+}