You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2021/09/06 20:51:42 UTC

[GitHub] [geode] jdeppe-pivotal opened a new pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

jdeppe-pivotal opened a new pull request #6844:
URL: https://github.com/apache/geode/pull/6844


   Thank you for submitting a contribution to Apache Geode.
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced in the commit message?
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `develop`)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   - [ ] Does `gradlew build` run cleanly?
   
   - [ ] Have you written or updated unit tests to verify your changes?
   
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   
   ### Note:
   Please ensure that once the PR is submitted, check Concourse for build issues and
   submit an update to your PR as soon as possible. If you need help, please send an
   email to dev@geode.apache.org.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705595851



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/connection/AuthExecutor.java
##########
@@ -15,35 +15,54 @@
  */
 package org.apache.geode.redis.internal.executor.connection;
 
-import java.util.Arrays;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_AUTH_CALLED_WITHOUT_SECURITY_CONFIGURED;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_INVALID_USERNAME_OR_PASSWORD;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToString;
+
 import java.util.List;
+import java.util.Properties;
+
+import org.apache.shiro.subject.Subject;
 
-import org.apache.geode.redis.internal.RedisConstants;
+import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.redis.internal.executor.Executor;
 import org.apache.geode.redis.internal.executor.RedisResponse;
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.SecurityManager;
 
 public class AuthExecutor implements Executor {
 
   @Override
-  public RedisResponse executeCommand(Command command,
-      ExecutionHandlerContext context) {
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    byte[] password = context.getAuthPassword();
-    if (password == null) {
-      return RedisResponse.error(RedisConstants.ERROR_NO_PASS);
-    }
+    SecurityService securityService = context.getSecurityService();
 
-    boolean correct = Arrays.equals(commandElems.get(1), password);
+    // We're deviating from Redis here in that any AUTH requests, without security explicitly
+    // set up, will fail.
+    if (!securityService.isIntegratedSecurity()) {
+      return RedisResponse.error(ERROR_AUTH_CALLED_WITHOUT_SECURITY_CONFIGURED);
+    }
 
-    if (correct) {
-      context.setAuthenticationVerified();
-      return RedisResponse.ok();
+    Properties props = new Properties();
+    if (commandElems.size() == 2) {
+      props.setProperty(SecurityManager.USER_NAME, context.getRedisUsername());
+      props.setProperty(SecurityManager.PASSWORD, bytesToString(commandElems.get(1)));
     } else {
-      return RedisResponse.error(RedisConstants.ERROR_INVALID_PWD);
+      props.setProperty(SecurityManager.USER_NAME, bytesToString(commandElems.get(1)));
+      props.setProperty(SecurityManager.PASSWORD, bytesToString(commandElems.get(2)));
     }
+
+    try {
+      Subject subject = securityService.login(props);

Review comment:
       Ah - thanks that's a good point.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705555999



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/connection/AuthExecutor.java
##########
@@ -15,35 +15,55 @@
  */
 package org.apache.geode.redis.internal.executor.connection;
 
-import java.util.Arrays;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_INVALID_USERNAME_OR_PASSWORD;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToString;
+
 import java.util.List;
+import java.util.Properties;
 
-import org.apache.geode.redis.internal.RedisConstants;
 import org.apache.geode.redis.internal.executor.Executor;
 import org.apache.geode.redis.internal.executor.RedisResponse;
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.SecurityManager;
 
 public class AuthExecutor implements Executor {
 
   @Override
-  public RedisResponse executeCommand(Command command,
-      ExecutionHandlerContext context) {
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    byte[] password = context.getAuthPassword();
-    if (password == null) {
-      return RedisResponse.error(RedisConstants.ERROR_NO_PASS);
+    SecurityManager securityManager = context.getSecurityManager();
+    Properties props = new Properties();
+    if (commandElems.size() == 2) {

Review comment:
       Changed a lot since we're returning an error now when there is no SecurityManager.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705559816



##########
File path: geode-apis-compatible-with-redis/README.md
##########
@@ -91,68 +94,139 @@ To confirm that everything shut down correctly, if you execute a Redis command i
 Could not connect to Redis at 127.0.0.1:6379: Connection refused 
 not connected>
 ```
-### <a name="redis-commands"></a>Redis Commands
+
+## <a name="security"></a>Security
+
+Security is implemented slightly differently to OSS Redis. Redis stores password information in plain text in the redis.conf file.     
+
+When using Apache Geode, to enable security, a Security Manager needs to be configured on the server(s). This Security Manager will authenticate `AUTH <password>` commands and `AUTH <username> <password>` commands. Users can set a custom `default` username using the `geode-compatible-with-redis-username` parameter. This username will be used when `AUTH <password>` commands are sent without a `<username>`. 
+
+The following gfsh command will configure a `SimpleSecurityManager`:

Review comment:
       We can add that in the next bit of work which is for authorization.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r704547871



##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/connection/AbstractAuthIntegrationTest.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.redis.internal.executor.connection;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.RedisURI;
+import org.junit.Test;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.Protocol;
+
+
+public abstract class AbstractAuthIntegrationTest {
+
+  static final String USERNAME = "default";

Review comment:
       This is also used to authenticate against Redis' default username of `default`. Applying the suggestion would still cause tests to fail, in the native acceptance tests, if the Geode default changes.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jinmeiliao commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r704582597



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/connection/AuthExecutor.java
##########
@@ -15,35 +15,55 @@
  */
 package org.apache.geode.redis.internal.executor.connection;
 
-import java.util.Arrays;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_INVALID_USERNAME_OR_PASSWORD;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToString;
+
 import java.util.List;
+import java.util.Properties;
 
-import org.apache.geode.redis.internal.RedisConstants;
 import org.apache.geode.redis.internal.executor.Executor;
 import org.apache.geode.redis.internal.executor.RedisResponse;
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.SecurityManager;
 
 public class AuthExecutor implements Executor {
 
   @Override
-  public RedisResponse executeCommand(Command command,
-      ExecutionHandlerContext context) {
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    byte[] password = context.getAuthPassword();
-    if (password == null) {
-      return RedisResponse.error(RedisConstants.ERROR_NO_PASS);
+    SecurityManager securityManager = context.getSecurityManager();
+    Properties props = new Properties();
+    if (commandElems.size() == 2) {

Review comment:
       nitpick: probably do early returns to avoid nested if/elses.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/connection/AuthExecutor.java
##########
@@ -15,35 +15,55 @@
  */
 package org.apache.geode.redis.internal.executor.connection;
 
-import java.util.Arrays;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_INVALID_USERNAME_OR_PASSWORD;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToString;
+
 import java.util.List;
+import java.util.Properties;
 
-import org.apache.geode.redis.internal.RedisConstants;
 import org.apache.geode.redis.internal.executor.Executor;
 import org.apache.geode.redis.internal.executor.RedisResponse;
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.SecurityManager;
 
 public class AuthExecutor implements Executor {
 
   @Override
-  public RedisResponse executeCommand(Command command,
-      ExecutionHandlerContext context) {
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    byte[] password = context.getAuthPassword();
-    if (password == null) {
-      return RedisResponse.error(RedisConstants.ERROR_NO_PASS);
+    SecurityManager securityManager = context.getSecurityManager();

Review comment:
       Is it possible to get the securtyService out of context? SecurityManager is supposed to be an internal component used by the SecurityService, the product should interact only with SecurityService. 

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/NettyRedisServer.java
##########
@@ -86,18 +85,20 @@
   private final Channel serverChannel;
   private final int serverPort;
   private final DistributedMember member;
+  private final SecurityManager securityManager;
 
   public NettyRedisServer(Supplier<DistributionConfig> configSupplier,
       RegionProvider regionProvider, PubSub pubsub, Supplier<Boolean> allowUnsupportedSupplier,
       Runnable shutdownInvoker, int port, String requestedAddress, RedisStats redisStats,
-      DistributedMember member) {
+      DistributedMember member, SecurityManager securityManager) {

Review comment:
       Maybe we can pass in the SecurityService here

##########
File path: geode-apis-compatible-with-redis/README.md
##########
@@ -56,14 +53,40 @@ Your Geode instance should now be up and running (1 locator and 1 server) and re
 To confirm the server is listening, in a separate terminal run:
 
 ```console
-redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> -a <compatibleWithRedisPassword> ping
+redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> ping
 ```
 
 - Replace `<compatibleWithRedisBindAddress>`, `<compatibleWithRedisPort>`, and `<compatibleWithRedisPassword>` with the same values as the server.
 
 If the server is functioning properly, you should see a response of `PONG`.
 
-### <a name="adding-a-server"></a>Optional - Adding an additional Geode server with compatible with Redis APIS
+## <a name="security"></a>Security
+
+Security is implemented slightly differently to Redis. To enable security, a Security Manager needs to be configured. This Security Manager will authenticate `AUTH username password` commands. Instead of a default, system-wide password, a default username can be set using the `geode-compatible-with-redis-username` parameter. This username is used when `AUTH` commands are sent with only a password.
+
+For example, the following gfsh command will configure a `SimpleSecurityManager`:
+
+```console
+start server \
+  --name=<serverName> \
+  --locators=<locatorPort> \
+  --compatible-with-redis-port=<compatibleWithRedisPort> \
+  --compatible-with-redis-bind-address=<compatibleWithRedisBindAddress> \
+  --compatible-with-redis-username=<compatibleWithRedisUsername> \
+  --J=-Dgemfire.security-manager=org.apache.geode.examples.SimpleSecurityManager
+```
+
+To confirm that the server is working, in a separate terminal run:
+
+```console
+redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> --user <compatibleWithRedisUsername> -a <compatibleWithRedisUsername> ping

Review comment:
       --user and -a both takes "compatibleWithRedisUsername"?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal merged pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal merged pull request #6844:
URL: https://github.com/apache/geode/pull/6844


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on pull request #6844:
URL: https://github.com/apache/geode/pull/6844#issuecomment-914450081


   @jomartin-999 Please would you review the README changes.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r704605950



##########
File path: geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
##########
@@ -1317,7 +1317,7 @@ static Class _getAttributeType(String attName) {
         "Specifies the address on which the Redis API for Geode is listening. If set to the empty string or this property is not specified, localhost is requested from the operating system.");
     m.put(REDIS_ENABLED,
         "When the default value of false, the Redis API for Geode is not available.  Set to true to enable the Redis API for Geode.");
-    m.put(REDIS_PASSWORD,
+    m.put(REDIS_USERNAME,
         "Specifies the password that the server uses when a client attempts to authenticate. The default is none and no authentication will be required.");

Review comment:
       this help string needs to be updated. It is still describing the old password property

##########
File path: geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
##########
@@ -3515,24 +3515,24 @@ static InetAddress _getDefaultMcastAddress() {
   boolean DEFAULT_REDIS_ENABLED = false;
 
   /**
-   * Returns the value of the {@link ConfigurationProperties#REDIS_PASSWORD} property
+   * Returns the value of the {@link ConfigurationProperties#REDIS_USERNAME} property
    * <p>
    * Returns the value of the
-   * {@link ConfigurationProperties#REDIS_PASSWORD} property
+   * {@link ConfigurationProperties#REDIS_USERNAME} property
    *
    * @return the authentication password for GeodeRedisServer

Review comment:
       update comment to refer to username

##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java
##########
@@ -290,7 +290,7 @@ ResultModel doStartServer(String memberName, Boolean assignBuckets, String bindA
         redisPort);
     StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
         ConfigurationProperties.REDIS_BIND_ADDRESS, redisBindAddress);
-    StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.REDIS_PASSWORD,
+    StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.REDIS_USERNAME,
         redisPassword);

Review comment:
       change "redisPassword" to "redisUsername"




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jinmeiliao commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705561065



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/connection/AuthExecutor.java
##########
@@ -15,35 +15,54 @@
  */
 package org.apache.geode.redis.internal.executor.connection;
 
-import java.util.Arrays;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_AUTH_CALLED_WITHOUT_SECURITY_CONFIGURED;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_INVALID_USERNAME_OR_PASSWORD;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToString;
+
 import java.util.List;
+import java.util.Properties;
+
+import org.apache.shiro.subject.Subject;
 
-import org.apache.geode.redis.internal.RedisConstants;
+import org.apache.geode.internal.security.SecurityService;
 import org.apache.geode.redis.internal.executor.Executor;
 import org.apache.geode.redis.internal.executor.RedisResponse;
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.SecurityManager;
 
 public class AuthExecutor implements Executor {
 
   @Override
-  public RedisResponse executeCommand(Command command,
-      ExecutionHandlerContext context) {
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    byte[] password = context.getAuthPassword();
-    if (password == null) {
-      return RedisResponse.error(RedisConstants.ERROR_NO_PASS);
-    }
+    SecurityService securityService = context.getSecurityService();
 
-    boolean correct = Arrays.equals(commandElems.get(1), password);
+    // We're deviating from Redis here in that any AUTH requests, without security explicitly
+    // set up, will fail.
+    if (!securityService.isIntegratedSecurity()) {
+      return RedisResponse.error(ERROR_AUTH_CALLED_WITHOUT_SECURITY_CONFIGURED);
+    }
 
-    if (correct) {
-      context.setAuthenticationVerified();
-      return RedisResponse.ok();
+    Properties props = new Properties();
+    if (commandElems.size() == 2) {
+      props.setProperty(SecurityManager.USER_NAME, context.getRedisUsername());
+      props.setProperty(SecurityManager.PASSWORD, bytesToString(commandElems.get(1)));
     } else {
-      return RedisResponse.error(RedisConstants.ERROR_INVALID_PWD);
+      props.setProperty(SecurityManager.USER_NAME, bytesToString(commandElems.get(1)));
+      props.setProperty(SecurityManager.PASSWORD, bytesToString(commandElems.get(2)));
     }
+
+    try {
+      Subject subject = securityService.login(props);

Review comment:
       to avoid resource leak, a login should have a corresponding logout when we know the subject has come to an end. we should either call `securiityService.logout()` if the subject is bound to the thread, or `subject.logout()` sometime. So probably it's best to save the subject to the context instead of just the principal, so that we can call logout in the end.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] upthewaterspout commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
upthewaterspout commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r704654640



##########
File path: geode-apis-compatible-with-redis/README.md
##########
@@ -91,68 +94,139 @@ To confirm that everything shut down correctly, if you execute a Redis command i
 Could not connect to Redis at 127.0.0.1:6379: Connection refused 
 not connected>
 ```
-### <a name="redis-commands"></a>Redis Commands
+
+## <a name="security"></a>Security
+
+Security is implemented slightly differently to OSS Redis. Redis stores password information in plain text in the redis.conf file.     
+
+When using Apache Geode, to enable security, a Security Manager needs to be configured on the server(s). This Security Manager will authenticate `AUTH <password>` commands and `AUTH <username> <password>` commands. Users can set a custom `default` username using the `geode-compatible-with-redis-username` parameter. This username will be used when `AUTH <password>` commands are sent without a `<username>`. 
+
+The following gfsh command will configure a `SimpleSecurityManager`:

Review comment:
       Maybe we should demo the `ExampleSecurityManager` instead - which at least lets you set the password?
   
   We should also mention what permission the user needs to have given to them - eg DATA:WRITE:REDIS_DATA.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/connection/AuthExecutor.java
##########
@@ -15,35 +15,55 @@
  */
 package org.apache.geode.redis.internal.executor.connection;
 
-import java.util.Arrays;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_INVALID_USERNAME_OR_PASSWORD;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToString;
+
 import java.util.List;
+import java.util.Properties;
 
-import org.apache.geode.redis.internal.RedisConstants;
 import org.apache.geode.redis.internal.executor.Executor;
 import org.apache.geode.redis.internal.executor.RedisResponse;
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.SecurityManager;
 
 public class AuthExecutor implements Executor {
 
   @Override
-  public RedisResponse executeCommand(Command command,
-      ExecutionHandlerContext context) {
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    byte[] password = context.getAuthPassword();
-    if (password == null) {
-      return RedisResponse.error(RedisConstants.ERROR_NO_PASS);
+    SecurityManager securityManager = context.getSecurityManager();
+    Properties props = new Properties();
+    if (commandElems.size() == 2) {
+      if (securityManager == null) {
+        return RedisResponse.error(ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED);
+      }
+      props.setProperty("security-username", context.getRedisUsername());
+      props.setProperty("security-password", bytesToString(commandElems.get(1)));
+    } else {
+      if (securityManager == null) {
+        String receivedUsername = new String(commandElems.get(1));
+        if (receivedUsername.equalsIgnoreCase(context.getRedisUsername())) {
+          return RedisResponse.ok();

Review comment:
       Yesterday we discussed that we think we ought to deviate for the native redis behavior here. If there is no security manager, *any* attempt to call auth should fail. Hopefully that can make the logic in this method a bit simpler.

##########
File path: geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
##########
@@ -1317,7 +1317,7 @@ static Class _getAttributeType(String attName) {
         "Specifies the address on which the Redis API for Geode is listening. If set to the empty string or this property is not specified, localhost is requested from the operating system.");
     m.put(REDIS_ENABLED,
         "When the default value of false, the Redis API for Geode is not available.  Set to true to enable the Redis API for Geode.");
-    m.put(REDIS_PASSWORD,
+    m.put(REDIS_USERNAME,
         "Specifies the password that the server uses when a client attempts to authenticate. The default is none and no authentication will be required.");

Review comment:
       I think you need to change the description here as well - it still refers to a password.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/connection/AuthExecutor.java
##########
@@ -15,35 +15,55 @@
  */
 package org.apache.geode.redis.internal.executor.connection;
 
-import java.util.Arrays;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_INVALID_USERNAME_OR_PASSWORD;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToString;
+
 import java.util.List;
+import java.util.Properties;
 
-import org.apache.geode.redis.internal.RedisConstants;
 import org.apache.geode.redis.internal.executor.Executor;
 import org.apache.geode.redis.internal.executor.RedisResponse;
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.SecurityManager;
 
 public class AuthExecutor implements Executor {
 
   @Override
-  public RedisResponse executeCommand(Command command,
-      ExecutionHandlerContext context) {
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    byte[] password = context.getAuthPassword();
-    if (password == null) {
-      return RedisResponse.error(RedisConstants.ERROR_NO_PASS);
+    SecurityManager securityManager = context.getSecurityManager();
+    Properties props = new Properties();
+    if (commandElems.size() == 2) {
+      if (securityManager == null) {
+        return RedisResponse.error(ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED);
+      }
+      props.setProperty("security-username", context.getRedisUsername());
+      props.setProperty("security-password", bytesToString(commandElems.get(1)));

Review comment:
       I think you can use the constants from `SecurityManager` for strings like "security-username"




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jinmeiliao commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jinmeiliao commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705566385



##########
File path: geode-apis-compatible-with-redis/README.md
##########
@@ -56,14 +53,40 @@ Your Geode instance should now be up and running (1 locator and 1 server) and re
 To confirm the server is listening, in a separate terminal run:
 
 ```console
-redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> -a <compatibleWithRedisPassword> ping
+redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> ping
 ```
 
 - Replace `<compatibleWithRedisBindAddress>`, `<compatibleWithRedisPort>`, and `<compatibleWithRedisPassword>` with the same values as the server.
 
 If the server is functioning properly, you should see a response of `PONG`.
 
-### <a name="adding-a-server"></a>Optional - Adding an additional Geode server with compatible with Redis APIS
+## <a name="security"></a>Security
+
+Security is implemented slightly differently to Redis. To enable security, a Security Manager needs to be configured. This Security Manager will authenticate `AUTH username password` commands. Instead of a default, system-wide password, a default username can be set using the `geode-compatible-with-redis-username` parameter. This username is used when `AUTH` commands are sent with only a password.
+
+For example, the following gfsh command will configure a `SimpleSecurityManager`:
+
+```console
+start server \
+  --name=<serverName> \
+  --locators=<locatorPort> \
+  --compatible-with-redis-port=<compatibleWithRedisPort> \
+  --compatible-with-redis-bind-address=<compatibleWithRedisBindAddress> \
+  --compatible-with-redis-username=<compatibleWithRedisUsername> \
+  --J=-Dgemfire.security-manager=org.apache.geode.examples.SimpleSecurityManager
+```
+
+To confirm that the server is working, in a separate terminal run:
+
+```console
+redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> --user <compatibleWithRedisUsername> -a <compatibleWithRedisUsername> ping

Review comment:
       but for documentation, shouldn't it point out -a takes password? The fact that it uses the username here is only for a specific security manager. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705556458



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/NettyRedisServer.java
##########
@@ -86,18 +85,20 @@
   private final Channel serverChannel;
   private final int serverPort;
   private final DistributedMember member;
+  private final SecurityManager securityManager;
 
   public NettyRedisServer(Supplier<DistributionConfig> configSupplier,
       RegionProvider regionProvider, PubSub pubsub, Supplier<Boolean> allowUnsupportedSupplier,
       Runnable shutdownInvoker, int port, String requestedAddress, RedisStats redisStats,
-      DistributedMember member) {
+      DistributedMember member, SecurityManager securityManager) {

Review comment:
       Done

##########
File path: geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
##########
@@ -1317,7 +1317,7 @@ static Class _getAttributeType(String attName) {
         "Specifies the address on which the Redis API for Geode is listening. If set to the empty string or this property is not specified, localhost is requested from the operating system.");
     m.put(REDIS_ENABLED,
         "When the default value of false, the Redis API for Geode is not available.  Set to true to enable the Redis API for Geode.");
-    m.put(REDIS_PASSWORD,
+    m.put(REDIS_USERNAME,
         "Specifies the password that the server uses when a client attempts to authenticate. The default is none and no authentication will be required.");

Review comment:
       Done

##########
File path: geode-core/src/main/java/org/apache/geode/distributed/internal/DistributionConfig.java
##########
@@ -3515,24 +3515,24 @@ static InetAddress _getDefaultMcastAddress() {
   boolean DEFAULT_REDIS_ENABLED = false;
 
   /**
-   * Returns the value of the {@link ConfigurationProperties#REDIS_PASSWORD} property
+   * Returns the value of the {@link ConfigurationProperties#REDIS_USERNAME} property
    * <p>
    * Returns the value of the
-   * {@link ConfigurationProperties#REDIS_PASSWORD} property
+   * {@link ConfigurationProperties#REDIS_USERNAME} property
    *
    * @return the authentication password for GeodeRedisServer

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jomartin-999 commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jomartin-999 commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r703828649



##########
File path: geode-apis-compatible-with-redis/README.md
##########
@@ -56,14 +53,40 @@ Your Geode instance should now be up and running (1 locator and 1 server) and re
 To confirm the server is listening, in a separate terminal run:
 
 ```console
-redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> -a <compatibleWithRedisPassword> ping
+redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> ping
 ```
 
 - Replace `<compatibleWithRedisBindAddress>`, `<compatibleWithRedisPort>`, and `<compatibleWithRedisPassword>` with the same values as the server.
 
 If the server is functioning properly, you should see a response of `PONG`.
 
-### <a name="adding-a-server"></a>Optional - Adding an additional Geode server with compatible with Redis APIS
+## <a name="security"></a>Security
+
+Security is implemented slightly differently to Redis. To enable security, a Security Manager needs to be configured. This Security Manager will authenticate `AUTH username password` commands. Instead of a default, system-wide password, a default username can be set using the `geode-compatible-with-redis-username` parameter. This username is used when `AUTH` commands are sent with only a password.
+
+For example, the following gfsh command will configure a `SimpleSecurityManager`:
+
+```console
+start server \
+  --name=<serverName> \
+  --locators=<locatorPort> \
+  --compatible-with-redis-port=<compatibleWithRedisPort> \
+  --compatible-with-redis-bind-address=<compatibleWithRedisBindAddress> \
+  --compatible-with-redis-username=<compatibleWithRedisUsername> \
+  --J=-Dgemfire.security-manager=org.apache.geode.examples.SimpleSecurityManager
+```
+
+To confirm that the server is working, in a separate terminal run:
+
+```console
+redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> --user <compatibleWithRedisUsername> -a <compatibleWithRedisUsername> ping

Review comment:
       I believe they will need to add "-c" when starting the cli in cluster mode.
   
   We should also link to or have an example of how one would actually use a security manager in production.  For example, users would include the jar in their server startup...and do they have to include it in every startup?  Also noting that that if they don't start with a security manager, they'll have to....delete the instance and start a new one?

##########
File path: geode-apis-compatible-with-redis/README.md
##########
@@ -56,14 +53,40 @@ Your Geode instance should now be up and running (1 locator and 1 server) and re
 To confirm the server is listening, in a separate terminal run:
 
 ```console
-redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> -a <compatibleWithRedisPassword> ping
+redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> ping

Review comment:
       Users also need to add a -c (and maybe the port) in order to connect in cluster mode.

##########
File path: geode-apis-compatible-with-redis/README.md
##########
@@ -14,7 +14,7 @@ Note: This feature is experimental and is subject to change in future releases o
 
 The Geode APIs compatible with Redis allow Geode to function as a drop-in replacement for a highly-available Redis data store, letting Redis applications take advantage of Geode’s scaling capabilities without changing their client code. 
 
-Redis clients connect to a Geode server in the same way they connect to a Redis server, using a hostname and a port number, with optional password authentication.
+Redis clients connect to a Geode server in the same way they connect to a Redis server, using a hostname and a port number, with optional username/password authentication.

Review comment:
       I think we need to update this section include something about clients that support CLUSTER commands only, at this time.
   
   Are we actually going to accept username and password in the same way that Redis does?  or just password...
   
   We should be explicit in the forms that we accept. Also, aren't we removing the password parameter?  I think we need to make this are a bit clearer on what actually happens.

##########
File path: geode-apis-compatible-with-redis/README.md
##########
@@ -56,14 +53,40 @@ Your Geode instance should now be up and running (1 locator and 1 server) and re
 To confirm the server is listening, in a separate terminal run:
 
 ```console
-redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> -a <compatibleWithRedisPassword> ping
+redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> ping
 ```
 
 - Replace `<compatibleWithRedisBindAddress>`, `<compatibleWithRedisPort>`, and `<compatibleWithRedisPassword>` with the same values as the server.
 
 If the server is functioning properly, you should see a response of `PONG`.
 
-### <a name="adding-a-server"></a>Optional - Adding an additional Geode server with compatible with Redis APIS
+## <a name="security"></a>Security
+
+Security is implemented slightly differently to Redis. To enable security, a Security Manager needs to be configured. This Security Manager will authenticate `AUTH username password` commands. Instead of a default, system-wide password, a default username can be set using the `geode-compatible-with-redis-username` parameter. This username is used when `AUTH` commands are sent with only a password.

Review comment:
       Can we link the **Security Manager** mention to the Security Manager section in the docs?
   
   I think here we need to be explicit in saying that we support both AUTH (password only) and AUTH (username & password).  Right now it sounds like we only accept the latter.
   
   We should also mention that if no username is set, the system will default [insert whatever default username we select here].




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] DonalEvans commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r703855601



##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/connection/AbstractAuthIntegrationTest.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.redis.internal.executor.connection;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.RedisURI;
+import org.junit.Test;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.Protocol;
+
+
+public abstract class AbstractAuthIntegrationTest {
+
+  static final String USERNAME = "default";

Review comment:
       Could this be set as `DistributionConfig#DEFAULT_REDIS_USERNAME` to avoid the test failing if the default value changes?

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/connection/AbstractAuthIntegrationTest.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.redis.internal.executor.connection;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.RedisURI;
+import org.junit.Test;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.Protocol;
+
+
+public abstract class AbstractAuthIntegrationTest {
+
+  static final String USERNAME = "default";
+  // Since we're going to use a SimpleSecurityManager where password == username means success
+  static final String PASSWORD = USERNAME;
+  Jedis jedis;
+
+  protected abstract void setupCacheWithSecurity(boolean withSecurityManager) throws Exception;

Review comment:
       The value passed for `withSecurityManager` is always `true`. Do we need this method to take an argument if we also have the `setupCacheWithoutSecurity()` method? Alternately, are tests needed for the case where `withSecurityManager` is false?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705557456



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/connection/AuthExecutor.java
##########
@@ -15,35 +15,55 @@
  */
 package org.apache.geode.redis.internal.executor.connection;
 
-import java.util.Arrays;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_INVALID_USERNAME_OR_PASSWORD;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToString;
+
 import java.util.List;
+import java.util.Properties;
 
-import org.apache.geode.redis.internal.RedisConstants;
 import org.apache.geode.redis.internal.executor.Executor;
 import org.apache.geode.redis.internal.executor.RedisResponse;
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.SecurityManager;
 
 public class AuthExecutor implements Executor {
 
   @Override
-  public RedisResponse executeCommand(Command command,
-      ExecutionHandlerContext context) {
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    byte[] password = context.getAuthPassword();
-    if (password == null) {
-      return RedisResponse.error(RedisConstants.ERROR_NO_PASS);
+    SecurityManager securityManager = context.getSecurityManager();
+    Properties props = new Properties();
+    if (commandElems.size() == 2) {
+      if (securityManager == null) {
+        return RedisResponse.error(ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED);
+      }
+      props.setProperty("security-username", context.getRedisUsername());
+      props.setProperty("security-password", bytesToString(commandElems.get(1)));
+    } else {
+      if (securityManager == null) {
+        String receivedUsername = new String(commandElems.get(1));
+        if (receivedUsername.equalsIgnoreCase(context.getRedisUsername())) {
+          return RedisResponse.ok();

Review comment:
       I made this change. In this case the returned error is `AUTH called without a Security Manager configured.`




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705558905



##########
File path: geode-core/src/main/java/org/apache/geode/distributed/internal/AbstractDistributionConfig.java
##########
@@ -1317,7 +1317,7 @@ static Class _getAttributeType(String attName) {
         "Specifies the address on which the Redis API for Geode is listening. If set to the empty string or this property is not specified, localhost is requested from the operating system.");
     m.put(REDIS_ENABLED,
         "When the default value of false, the Redis API for Geode is not available.  Set to true to enable the Redis API for Geode.");
-    m.put(REDIS_PASSWORD,
+    m.put(REDIS_USERNAME,
         "Specifies the password that the server uses when a client attempts to authenticate. The default is none and no authentication will be required.");

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705556249



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/connection/AuthExecutor.java
##########
@@ -15,35 +15,55 @@
  */
 package org.apache.geode.redis.internal.executor.connection;
 
-import java.util.Arrays;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_INVALID_USERNAME_OR_PASSWORD;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToString;
+
 import java.util.List;
+import java.util.Properties;
 
-import org.apache.geode.redis.internal.RedisConstants;
 import org.apache.geode.redis.internal.executor.Executor;
 import org.apache.geode.redis.internal.executor.RedisResponse;
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.SecurityManager;
 
 public class AuthExecutor implements Executor {
 
   @Override
-  public RedisResponse executeCommand(Command command,
-      ExecutionHandlerContext context) {
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    byte[] password = context.getAuthPassword();
-    if (password == null) {
-      return RedisResponse.error(RedisConstants.ERROR_NO_PASS);
+    SecurityManager securityManager = context.getSecurityManager();

Review comment:
       Done.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705555292



##########
File path: geode-apis-compatible-with-redis/README.md
##########
@@ -56,14 +53,40 @@ Your Geode instance should now be up and running (1 locator and 1 server) and re
 To confirm the server is listening, in a separate terminal run:
 
 ```console
-redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> -a <compatibleWithRedisPassword> ping
+redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> ping
 ```
 
 - Replace `<compatibleWithRedisBindAddress>`, `<compatibleWithRedisPort>`, and `<compatibleWithRedisPassword>` with the same values as the server.
 
 If the server is functioning properly, you should see a response of `PONG`.
 
-### <a name="adding-a-server"></a>Optional - Adding an additional Geode server with compatible with Redis APIS
+## <a name="security"></a>Security
+
+Security is implemented slightly differently to Redis. To enable security, a Security Manager needs to be configured. This Security Manager will authenticate `AUTH username password` commands. Instead of a default, system-wide password, a default username can be set using the `geode-compatible-with-redis-username` parameter. This username is used when `AUTH` commands are sent with only a password.
+
+For example, the following gfsh command will configure a `SimpleSecurityManager`:
+
+```console
+start server \
+  --name=<serverName> \
+  --locators=<locatorPort> \
+  --compatible-with-redis-port=<compatibleWithRedisPort> \
+  --compatible-with-redis-bind-address=<compatibleWithRedisBindAddress> \
+  --compatible-with-redis-username=<compatibleWithRedisUsername> \
+  --J=-Dgemfire.security-manager=org.apache.geode.examples.SimpleSecurityManager
+```
+
+To confirm that the server is working, in a separate terminal run:
+
+```console
+redis-cli -h <compatibleWithRedisBindAddress> -p <compatibleWithRedisPort> --user <compatibleWithRedisUsername> -a <compatibleWithRedisUsername> ping

Review comment:
       One is the username and one is the password (-a). Since we're saying to use the `SimpleSecurityManager` they should be the same.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705558797



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/connection/AuthExecutor.java
##########
@@ -15,35 +15,55 @@
  */
 package org.apache.geode.redis.internal.executor.connection;
 
-import java.util.Arrays;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_INVALID_USERNAME_OR_PASSWORD;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToString;
+
 import java.util.List;
+import java.util.Properties;
 
-import org.apache.geode.redis.internal.RedisConstants;
 import org.apache.geode.redis.internal.executor.Executor;
 import org.apache.geode.redis.internal.executor.RedisResponse;
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.security.AuthenticationFailedException;
+import org.apache.geode.security.SecurityManager;
 
 public class AuthExecutor implements Executor {
 
   @Override
-  public RedisResponse executeCommand(Command command,
-      ExecutionHandlerContext context) {
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    byte[] password = context.getAuthPassword();
-    if (password == null) {
-      return RedisResponse.error(RedisConstants.ERROR_NO_PASS);
+    SecurityManager securityManager = context.getSecurityManager();
+    Properties props = new Properties();
+    if (commandElems.size() == 2) {
+      if (securityManager == null) {
+        return RedisResponse.error(ERROR_AUTH_CALLED_WITHOUT_PASSWORD_CONFIGURED);
+      }
+      props.setProperty("security-username", context.getRedisUsername());
+      props.setProperty("security-password", bytesToString(commandElems.get(1)));

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r704548493



##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/connection/AbstractAuthIntegrationTest.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.redis.internal.executor.connection;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.RedisURI;
+import org.junit.Test;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.Protocol;
+
+
+public abstract class AbstractAuthIntegrationTest {
+
+  static final String USERNAME = "default";
+  // Since we're going to use a SimpleSecurityManager where password == username means success
+  static final String PASSWORD = USERNAME;
+  Jedis jedis;
+
+  protected abstract void setupCacheWithSecurity(boolean withSecurityManager) throws Exception;

Review comment:
       I've removed the parameter.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6844: GEODE-9546: Integrate Security Manager into Radish AUTH flow

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6844:
URL: https://github.com/apache/geode/pull/6844#discussion_r705556726



##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/StartServerCommand.java
##########
@@ -290,7 +290,7 @@ ResultModel doStartServer(String memberName, Boolean assignBuckets, String bindA
         redisPort);
     StartMemberUtils.setPropertyIfNotNull(gemfireProperties,
         ConfigurationProperties.REDIS_BIND_ADDRESS, redisBindAddress);
-    StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.REDIS_PASSWORD,
+    StartMemberUtils.setPropertyIfNotNull(gemfireProperties, ConfigurationProperties.REDIS_USERNAME,
         redisPassword);

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org