You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by pi...@apache.org on 2018/03/15 17:49:49 UTC

[geode] branch develop updated: GEODE-4816: Add support for authentication to the experimental driver. (#1615)

This is an automated email from the ASF dual-hosted git repository.

pivotalsarge pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new ef2ebd7  GEODE-4816: Add support for authentication to the experimental driver. (#1615)
ef2ebd7 is described below

commit ef2ebd70cc5e2cbfdeb7dfbf1b133e47f30cc802
Author: Michael "Sarge" Dodge <md...@pivotal.io>
AuthorDate: Thu Mar 15 10:49:46 2018 -0700

    GEODE-4816: Add support for authentication to the experimental driver. (#1615)
---
 .../geode/experimental/driver/DriverFactory.java   | 34 +++++++-
 .../geode/experimental/driver/ProtobufChannel.java | 46 +++++++++--
 .../geode/experimental/driver/ProtobufDriver.java  |  7 +-
 .../experimental/driver/AuthenticationTest.java    | 92 ++++++++++++++++++++++
 ...rotobufConnectionAuthorizingStateProcessor.java |  4 +-
 .../protobuf/v1/AuthenticationIntegrationTest.java |  3 -
 6 files changed, 174 insertions(+), 12 deletions(-)

diff --git a/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/DriverFactory.java b/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/DriverFactory.java
index 267bb3d..b6cf205 100644
--- a/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/DriverFactory.java
+++ b/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/DriverFactory.java
@@ -37,6 +37,16 @@ public class DriverFactory {
   private Set<InetSocketAddress> locators = new HashSet<InetSocketAddress>();
 
   /**
+   * User identity as a string.
+   */
+  private String username = null;
+
+  /**
+   * User proof as a string.
+   */
+  private String password = null;
+
+  /**
    * Adds a locator at <code>host</code> and <code>port</code> to the set of locators to use.
    *
    * @param host Internet address or host name.
@@ -55,6 +65,28 @@ public class DriverFactory {
    * @throws Exception
    */
   public Driver create() throws Exception {
-    return new ProtobufDriver(locators);
+    return new ProtobufDriver(locators, username, password);
+  }
+
+  /**
+   * Specifies the user name with which to authenticate with the server.
+   *
+   * @param username User identity as a string; may be <code>null</code>.
+   * @return This driver factory.
+   */
+  public DriverFactory setUsername(String username) {
+    this.username = username;
+    return this;
+  }
+
+  /**
+   * Specifies the password with which to authenticate with the server.
+   *
+   * @param password User proof as a string; may be <code>null</code>.
+   * @return This driver factory.
+   */
+  public DriverFactory setPassword(String password) {
+    this.password = password;
+    return this;
   }
 }
diff --git a/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/ProtobufChannel.java b/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/ProtobufChannel.java
index b07d910..8ddbe6f 100644
--- a/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/ProtobufChannel.java
+++ b/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/ProtobufChannel.java
@@ -19,6 +19,7 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.InetSocketAddress;
 import java.net.Socket;
+import java.util.Objects;
 import java.util.Set;
 
 import org.apache.geode.internal.protocol.protobuf.ProtocolVersion;
@@ -27,6 +28,7 @@ import org.apache.geode.internal.protocol.protobuf.v1.ClientProtocol;
 import org.apache.geode.internal.protocol.protobuf.v1.ClientProtocol.ErrorResponse;
 import org.apache.geode.internal.protocol.protobuf.v1.ClientProtocol.Message;
 import org.apache.geode.internal.protocol.protobuf.v1.ClientProtocol.Message.MessageTypeCase;
+import org.apache.geode.internal.protocol.protobuf.v1.ConnectionAPI;
 import org.apache.geode.internal.protocol.protobuf.v1.LocatorAPI;
 
 class ProtobufChannel {
@@ -37,9 +39,10 @@ class ProtobufChannel {
    */
   final Socket socket;
 
-  public ProtobufChannel(final Set<InetSocketAddress> locators) throws IOException {
+  public ProtobufChannel(final Set<InetSocketAddress> locators, String username, String password)
+      throws IOException {
     this.locators = locators;
-    this.socket = connectToAServer();
+    this.socket = connectToAServer(username, password);
   }
 
   Message sendRequest(final Message request, MessageTypeCase expectedResult) throws IOException {
@@ -62,8 +65,8 @@ class ProtobufChannel {
     return this.socket.isClosed();
   }
 
-  private Socket connectToAServer() throws IOException {
-    InetSocketAddress server = findAServer();
+  private Socket connectToAServer(String username, String password) throws IOException {
+    InetSocketAddress server = findAServer(username, password);
     Socket socket = new Socket(server.getAddress(), server.getPort());
     socket.setTcpNoDelay(true);
     socket.setSendBufferSize(65535);
@@ -80,6 +83,11 @@ class ProtobufChannel {
         .getVersionAccepted()) {
       throw new IOException("Failed protocol version verification.");
     }
+
+    if (!Objects.isNull(username)) {
+      authenticate(username, password, outputStream, inputStream);
+    }
+
     return socket;
   }
 
@@ -88,7 +96,7 @@ class ProtobufChannel {
    *
    * @return The server chosen by the Locator service for this client
    */
-  private InetSocketAddress findAServer() throws IOException {
+  private InetSocketAddress findAServer(String username, String password) throws IOException {
     IOException lastException = null;
 
     for (InetSocketAddress locator : locators) {
@@ -109,6 +117,10 @@ class ProtobufChannel {
           throw new IOException("Failed ProtocolVersion.");
         }
 
+        if (!Objects.isNull(username)) {
+          authenticate(username, password, outputStream, inputStream);
+        }
+
         ClientProtocol.Message.newBuilder()
             .setGetServerRequest(LocatorAPI.GetServerRequest.newBuilder()).build()
             .writeDelimitedTo(outputStream);
@@ -143,6 +155,30 @@ class ProtobufChannel {
     }
   }
 
+  private void authenticate(String username, String password, OutputStream outputStream,
+      InputStream inputStream) throws IOException {
+    final ConnectionAPI.AuthenticationRequest.Builder builder =
+        ConnectionAPI.AuthenticationRequest.newBuilder();
+    builder.putCredentials("security-username", username);
+    builder.putCredentials("security-password", password);
+    final Message authenticationRequest =
+        Message.newBuilder().setAuthenticationRequest(builder).build();
+    authenticationRequest.writeDelimitedTo(outputStream);
+
+    final Message authenticationResponseMessage = Message.parseDelimitedFrom(inputStream);
+    final ErrorResponse errorResponse = authenticationResponseMessage.getErrorResponse();
+    if (!Objects.isNull(errorResponse) && errorResponse.hasError()) {
+      throw new IOException("Failed authentication for " + username + ": error code="
+          + errorResponse.getError().getErrorCode() + "; error message="
+          + errorResponse.getError().getMessage());
+    }
+    final ConnectionAPI.AuthenticationResponse authenticationResponse =
+        authenticationResponseMessage.getAuthenticationResponse();
+    if (!Objects.isNull(authenticationResponse) && !authenticationResponse.getAuthenticated()) {
+      throw new IOException("Failed authentication for " + username);
+    }
+  }
+
   private Message readResponse() throws IOException {
     final InputStream inputStream = socket.getInputStream();
     Message response = ClientProtocol.Message.parseDelimitedFrom(inputStream);
diff --git a/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/ProtobufDriver.java b/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/ProtobufDriver.java
index 9e8e8f9..bded2b0 100644
--- a/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/ProtobufDriver.java
+++ b/geode-experimental-driver/src/main/java/org/apache/geode/experimental/driver/ProtobufDriver.java
@@ -45,10 +45,13 @@ public class ProtobufDriver implements Driver {
    *
    * @param locators Set of Internet-address-or-host-name/port pairs of the locators to use to find
    *        GemFire servers that have Protobuf enabled.
+   * @param username User identity as a string; may be <code>null</code>.
+   * @param password User proof as a string; may be <code>null</code>.
    * @throws IOException
    */
-  ProtobufDriver(Set<InetSocketAddress> locators) throws IOException {
-    this.channel = new ProtobufChannel(locators);
+  ProtobufDriver(Set<InetSocketAddress> locators, String username, String password)
+      throws IOException {
+    this.channel = new ProtobufChannel(locators, username, password);
   }
 
   @Override
diff --git a/geode-experimental-driver/src/test/java/org/apache/geode/experimental/driver/AuthenticationTest.java b/geode-experimental-driver/src/test/java/org/apache/geode/experimental/driver/AuthenticationTest.java
new file mode 100644
index 0000000..ae4afd8
--- /dev/null
+++ b/geode-experimental-driver/src/test/java/org/apache/geode/experimental/driver/AuthenticationTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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.experimental.driver;
+
+import static org.apache.geode.internal.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.server.CacheServer;
+import org.apache.geode.distributed.ConfigurationProperties;
+import org.apache.geode.distributed.Locator;
+import org.apache.geode.security.SimpleTestSecurityManager;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class AuthenticationTest {
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  private static final String TEST_USERNAME = "cluster";
+  private static final String TEST_PASSWORD = TEST_USERNAME;
+  private Locator locator;
+  private Cache cache;
+  private Driver driver;
+  private int locatorPort;
+
+  @Before
+  public void createServer() throws Exception {
+    System.setProperty("geode.feature-protobuf-protocol", "true");
+
+    // Create a cache
+    CacheFactory cf = new CacheFactory();
+    cf.set(ConfigurationProperties.MCAST_PORT, "0");
+    cf.setSecurityManager(new SimpleTestSecurityManager());
+    cache = cf.create();
+
+    // Start a locator
+    locator = Locator.startLocatorAndDS(0, null, new Properties());
+    locatorPort = locator.getPort();
+
+    // do not start a cache server
+  }
+
+  @After
+  public void cleanup() {
+    locator.stop();
+    cache.close();
+  }
+
+  @Test
+  public void driverFailsToConnectWhenThereAreNoServers() throws Exception {
+    try {
+      driver = new DriverFactory().addLocator("localhost", locatorPort).create();
+    } catch (IOException e) {
+      // success
+      return;
+    }
+    throw new AssertionError("expected an IOException");
+  }
+
+  @Test
+  public void driverCanConnectWhenThereAreServers() throws Exception {
+    CacheServer server = cache.addCacheServer();
+    server.setPort(0);
+    server.start();
+    driver = new DriverFactory().addLocator("localhost", locatorPort).setUsername(TEST_USERNAME)
+        .setPassword(TEST_PASSWORD).create();
+    assertTrue(driver.isConnected());
+  }
+}
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/v1/state/ProtobufConnectionAuthorizingStateProcessor.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/v1/state/ProtobufConnectionAuthorizingStateProcessor.java
index da2e90e..b78172f 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/v1/state/ProtobufConnectionAuthorizingStateProcessor.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/v1/state/ProtobufConnectionAuthorizingStateProcessor.java
@@ -18,6 +18,7 @@ import org.apache.shiro.subject.Subject;
 import org.apache.shiro.util.ThreadState;
 
 import org.apache.geode.internal.protocol.protobuf.v1.BasicTypes;
+import org.apache.geode.internal.protocol.protobuf.v1.ClientProtocol;
 import org.apache.geode.internal.protocol.protobuf.v1.MessageExecutionContext;
 import org.apache.geode.internal.protocol.protobuf.v1.ProtobufOperationContext;
 import org.apache.geode.internal.protocol.protobuf.v1.ProtobufSerializationService;
@@ -49,7 +50,8 @@ public class ProtobufConnectionAuthorizingStateProcessor
     } catch (NotAuthorizedException e) {
       messageContext.getStatistics().incAuthorizationViolations();
       throw new OperationNotAuthorizedException(
-          "The user is not authorized to complete this operation");
+          "The user is not authorized to complete this operation: "
+              + ((ClientProtocol.Message) message).getMessageTypeCase());
     } finally {
       threadState.restore();
     }
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/v1/AuthenticationIntegrationTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/v1/AuthenticationIntegrationTest.java
index 2700c31..d4a62fc 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/v1/AuthenticationIntegrationTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/v1/AuthenticationIntegrationTest.java
@@ -41,9 +41,6 @@ import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.distributed.ConfigurationProperties;
 import org.apache.geode.internal.AvailablePortHelper;
 import org.apache.geode.internal.cache.InternalCache;
-import org.apache.geode.internal.cache.client.protocol.ClientProtocolServiceLoader;
-import org.apache.geode.internal.protocol.protobuf.statistics.ClientStatistics;
-import org.apache.geode.internal.protocol.protobuf.statistics.ProtobufClientStatistics;
 import org.apache.geode.internal.protocol.protobuf.v1.serializer.ProtobufProtocolSerializer;
 import org.apache.geode.management.internal.security.ResourceConstants;
 import org.apache.geode.security.AuthenticationFailedException;

-- 
To stop receiving notification emails like this one, please contact
pivotalsarge@apache.org.