You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2016/10/10 17:24:35 UTC

incubator-geode git commit: GEODE-1973: add more tests to cover GMSAuthenticator and SimpleSecurityManager

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 33cdf6886 -> 68aef5b7f


GEODE-1973: add more tests to cover GMSAuthenticator and SimpleSecurityManager

* adding more tests


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

Branch: refs/heads/develop
Commit: 68aef5b7fbdb7ff740d5fbbc66c7848f14b834ff
Parents: 33cdf68
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Sep 30 13:13:56 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Oct 10 10:24:03 2016 -0700

----------------------------------------------------------------------
 .../templates/SimpleSecurityManager.java        | 71 +++++++++++++++
 .../security/PeerAuthenticatorDUnitTest.java    | 93 +++++++++++++++++++
 ...henticatorWithCachelessLocatorDUnitTest.java | 94 ++++++++++++++++++++
 .../templates/SimpleSecurityManagerTest.java    | 80 +++++++++++++++++
 .../geode/security/CQClientAuthDunitTest.java   | 88 ++++++++++++++++++
 5 files changed, 426 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/68aef5b7/geode-core/src/main/java/org/apache/geode/security/templates/SimpleSecurityManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/security/templates/SimpleSecurityManager.java b/geode-core/src/main/java/org/apache/geode/security/templates/SimpleSecurityManager.java
new file mode 100644
index 0000000..d4855fe
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/security/templates/SimpleSecurityManager.java
@@ -0,0 +1,71 @@
+/*
+ * 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.security.templates;
+
+import java.util.Properties;
+
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.ResourcePermission;
+import org.apache.geode.security.SecurityManager;
+
+/**
+ * This class provides a simple implementation of {@link SecurityManager} for
+ * authentication and authorization solely based on the username and password provided.
+ *
+ * It is meant for demo purpose, not for production.
+ *
+ * Authentiation: All users whose password matches the username are authenticated.
+ * e.g. username/password = test/test, user/user, admin/admin
+ *
+ * Authorization: users whose username is a substring (case insensitive) of the permission
+ * required are authorized.
+ * e.g. username = data: is authorized for all data operations:
+ *        data;
+ *        data:manage
+ *        data:read
+ *        data:write
+ *      username = dataWrite: is authorized for data writes on all regions:
+ *        data:write
+ *        data:write:regionA
+ *      username = cluster: authorized for all cluster operations
+ *      username = cluserRead: authorzed for all cluster read operations
+ */
+public class SimpleSecurityManager implements SecurityManager {
+  @Override
+  public void init(final Properties securityProps) {}
+
+  @Override
+  public Object authenticate(final Properties credentials) throws AuthenticationFailedException {
+    String username = credentials.getProperty("security-username");
+    String password = credentials.getProperty("security-password");
+    if(username!=null && username.equals(password)){
+      return username;
+    }
+    throw new AuthenticationFailedException("invalid username/password");
+  }
+
+  @Override
+  public boolean authorize(final Object principal, final ResourcePermission permission) {
+    String permissionString = permission.toString().replace(":","").toLowerCase();
+    String principle = principal.toString().toLowerCase();
+    return permissionString.startsWith(principle);
+  }
+
+  @Override
+  public void close() {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/68aef5b7/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorDUnitTest.java
new file mode 100644
index 0000000..bb147c7
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorDUnitTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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.security;
+
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.File;
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.distributed.Locator;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.security.templates.DummyAuthenticator;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+
+@Category({ DistributedTest.class, SecurityTest.class })
+public class PeerAuthenticatorDUnitTest extends JUnit4DistributedTestCase {
+  protected VM locator = null;
+  protected VM server = null;
+  protected VM server1 = null;
+
+  @Before
+  public void before() throws Exception {
+    final Host host = Host.getHost(0);
+    this.locator = host.getVM(0);
+    this.server = host.getVM(1);
+    this.server1 = host.getVM(2);
+  }
+
+  @Test
+  public void testPeerAuthenticator() throws Exception{
+    int locatorPort = locator.invoke(()->{
+      Properties props = new Properties();
+      props.setProperty(SECURITY_PEER_AUTHENTICATOR, DummyAuthenticator.class.getName());
+      props.setProperty(MCAST_PORT, "0");
+      props.put(JMX_MANAGER, "true");
+      props.put(JMX_MANAGER_START, "true");
+      props.put(JMX_MANAGER_PORT, "0");
+      Locator locatorObj = Locator.startLocatorAndDS(0, new File("locator.log"), props);
+      return locatorObj.getPort();
+    });
+
+    // set up server with security
+    String locators = "localhost[" + locatorPort + "]";
+    server.invoke(()->{
+      Properties props = new Properties();
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(LOCATORS, locators);
+
+      // the following are needed for peer-to-peer authentication
+      props.setProperty("security-username", "user");
+      props.setProperty("security-password", "user");
+      // this should execute without exception
+      InternalDistributedSystem ds = getSystem(props);
+    });
+
+    server1.invoke(()->{
+      Properties props = new Properties();
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(LOCATORS, locators);
+
+      // the following are needed for peer-to-peer authentication
+      props.setProperty("security-username", "bogus");
+      props.setProperty("security-password", "user");
+
+      assertThatThrownBy(()->getSystem(props)).isInstanceOf(GemFireSecurityException.class);
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/68aef5b7/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorWithCachelessLocatorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorWithCachelessLocatorDUnitTest.java b/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorWithCachelessLocatorDUnitTest.java
new file mode 100644
index 0000000..eecce50
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/PeerAuthenticatorWithCachelessLocatorDUnitTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.security;
+
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+import static org.assertj.core.api.Assertions.*;
+
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.distributed.DistributedSystem;
+import org.apache.geode.distributed.internal.InternalDistributedSystem;
+import org.apache.geode.internal.AvailablePortHelper;
+import org.apache.geode.security.templates.DummyAuthenticator;
+import org.apache.geode.test.dunit.Host;
+import org.apache.geode.test.dunit.VM;
+import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+
+@Category({ DistributedTest.class, SecurityTest.class })
+public class PeerAuthenticatorWithCachelessLocatorDUnitTest extends JUnit4DistributedTestCase {
+  protected VM locator = null;
+  protected VM server = null;
+  protected VM server1 = null;
+
+  @Before
+  public void before() throws Exception {
+    final Host host = Host.getHost(0);
+    this.locator = host.getVM(0);
+    this.server = host.getVM(1);
+    this.server1 = host.getVM(2);
+  }
+
+  @Test
+  public void testPeerAuthenticator() throws Exception{
+    int locatorPort = AvailablePortHelper.getRandomAvailableTCPPort();
+    locator.invoke(()->{
+      Properties props = new Properties();
+      props.setProperty(SECURITY_PEER_AUTHENTICATOR, DummyAuthenticator.class.getName());
+      props.setProperty(MCAST_PORT, "0");
+      props.put(JMX_MANAGER, "true");
+      props.put(JMX_MANAGER_START, "true");
+      props.put(JMX_MANAGER_PORT, "0");
+      props.setProperty("start-locator", "localhost["+locatorPort+"]");
+      DistributedSystem.connect(props);
+    });
+
+    // set up server with security
+    String locators = "localhost[" + locatorPort + "]";
+    server.invoke(()->{
+      Properties props = new Properties();
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(LOCATORS, locators);
+
+      // the following are needed for peer-to-peer authentication
+      props.setProperty("security-username", "user");
+      props.setProperty("security-password", "user");
+      // this should execute without exception
+      InternalDistributedSystem ds = getSystem(props);
+    });
+
+    server1.invoke(()->{
+      Properties props = new Properties();
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(LOCATORS, locators);
+
+      // the following are needed for peer-to-peer authentication
+      props.setProperty("security-username", "bogus");
+      props.setProperty("security-password", "user");
+
+      assertThatThrownBy(()->getSystem(props)).isInstanceOf(GemFireSecurityException.class);
+    });
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/68aef5b7/geode-core/src/test/java/org/apache/geode/security/templates/SimpleSecurityManagerTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/org/apache/geode/security/templates/SimpleSecurityManagerTest.java b/geode-core/src/test/java/org/apache/geode/security/templates/SimpleSecurityManagerTest.java
new file mode 100644
index 0000000..a44af00
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/security/templates/SimpleSecurityManagerTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.security.templates;
+
+import static org.apache.geode.internal.Assert.assertTrue;
+import static org.assertj.core.api.Assertions.*;
+import static org.junit.Assert.*;
+
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.ResourcePermission;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category({ UnitTest.class, SecurityTest.class })
+public class SimpleSecurityManagerTest {
+  private SimpleSecurityManager manager;
+  private Properties credentials;
+
+  @Before
+  public void before(){
+    manager = new SimpleSecurityManager();
+    credentials = new Properties();
+  }
+
+  @Test
+  public void testAuthenticateSuccess(){
+    credentials.put("security-username", "user");
+    credentials.put("security-password", "user");
+    assertEquals("user", manager.authenticate(credentials));
+  }
+
+  @Test
+  public void testAuthenticateFail() {
+    credentials.put("security-username", "user1");
+    credentials.put("security-password", "user2");
+    assertThatThrownBy(() -> manager.authenticate(credentials)).isInstanceOf(AuthenticationFailedException.class);
+  }
+
+  @Test
+  public void testAuthenticateFailNull(){
+    assertThatThrownBy(()->manager.authenticate(credentials)).isInstanceOf(AuthenticationFailedException.class);
+  }
+
+  @Test
+  public void testAuthorization(){
+    ResourcePermission permission = new ResourcePermission("CLUSTER", "READ");
+    assertTrue(manager.authorize("clusterRead", permission));
+    assertTrue(manager.authorize("cluster", permission));
+    assertFalse(manager.authorize("data", permission));
+
+    permission = new ResourcePermission("DATA", "WRITE", "regionA", "key1");
+    assertTrue(manager.authorize("data", permission));
+    assertTrue(manager.authorize("dataWrite", permission));
+    assertTrue(manager.authorize("dataWriteRegionA", permission));
+    assertTrue(manager.authorize("dataWriteRegionAKey1", permission));
+    assertFalse(manager.authorize("dataRead", permission));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/68aef5b7/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDunitTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDunitTest.java b/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDunitTest.java
new file mode 100644
index 0000000..2386af1
--- /dev/null
+++ b/geode-cq/src/test/java/org/apache/geode/security/CQClientAuthDunitTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.security;
+
+import static org.apache.geode.distributed.ConfigurationProperties.*;
+
+import java.util.Properties;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.Region;
+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.client.Pool;
+import org.apache.geode.cache.client.PoolManager;
+import org.apache.geode.cache.client.internal.ProxyCache;
+import org.apache.geode.cache.query.CqAttributes;
+import org.apache.geode.cache.query.CqAttributesFactory;
+import org.apache.geode.cache.query.CqQuery;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.security.templates.SamplePostProcessor;
+import org.apache.geode.security.templates.UserPasswordAuthInit;
+import org.apache.geode.test.junit.categories.DistributedTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+
+@Category({ DistributedTest.class, SecurityTest.class })
+public class CQClientAuthDunitTest extends AbstractSecureServerDUnitTest {
+
+  public CQClientAuthDunitTest(){
+    this.postProcessor = SamplePostProcessor.class;
+  }
+
+  @Test
+  public void testPostProcess(){
+    String query = "select * from /AuthRegion";
+    client1.invoke(()-> {
+      Properties props = new Properties();
+      props.setProperty(LOCATORS, "");
+      props.setProperty(MCAST_PORT, "0");
+      props.setProperty(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName() + ".create");
+      ClientCacheFactory factory = new ClientCacheFactory(props);
+
+      factory.addPoolServer("localhost", this.serverPort);
+      factory.setPoolThreadLocalConnections(false);
+      factory.setPoolMinConnections(5);
+      factory.setPoolSubscriptionEnabled(true);
+      factory.setPoolMultiuserAuthentication(true);
+
+
+      ClientCache clientCache = factory.create();
+      Region region = clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      Pool pool = PoolManager.find(region);
+
+      Properties userProps = new Properties();
+      userProps.setProperty("security-username", "super-user");
+      userProps.setProperty("security-password", "1234567");
+      ProxyCache cache = (ProxyCache)clientCache.createAuthenticatedView(userProps, pool.getName());
+
+      QueryService qs = cache.getQueryService();
+
+      CqAttributesFactory cqAttributesFactory = new CqAttributesFactory();
+
+      CqAttributes cqa = cqAttributesFactory.create();
+
+      // Create the CqQuery
+      CqQuery cq = qs.newCq("CQ1", query, cqa, true);
+      cq.execute();
+    });
+  }
+
+}