You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2018/01/05 22:01:52 UTC

[geode] branch develop updated: GEODE-4192 GetServer request should return error if no servers found

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

bschuchardt 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 a675572  GEODE-4192 GetServer request should return error if no servers found
a675572 is described below

commit a67557216148460c97843dfef4124b2230c1e804
Author: Bruce Schuchardt <bs...@pivotal.io>
AuthorDate: Fri Jan 5 14:01:25 2018 -0800

    GEODE-4192 GetServer request should return error if no servers found
    
    I modified the GetServerOperationHandler to return a failure if there are
    no suitable servers.  There is a new NO_AVAILABLE_SERVER error code for
    this.
    
    I altered the test for this condition to require an error response.
    
    I altered the experimental driver to look for the new error code and
    added tests in DriverConnectionTest to ensure that it's working.
---
 .../geode/internal/protocol/ProtocolErrorCode.java |   3 +-
 .../geode/experimental/driver/ProtobufDriver.java  |  14 ++-
 .../experimental/driver/DriverConnectionTest.java  | 101 +++++++++++++++++++++
 .../src/main/proto/v1/basicTypes.proto             |   1 +
 .../src/main/proto/v1/locator_API.proto            |   1 +
 .../v1/operations/GetServerOperationHandler.java   |  17 ++--
 .../GetServerOperationHandlerJUnitTest.java        |  18 ++--
 7 files changed, 131 insertions(+), 24 deletions(-)

diff --git a/geode-client-protocol/src/main/java/org/apache/geode/internal/protocol/ProtocolErrorCode.java b/geode-client-protocol/src/main/java/org/apache/geode/internal/protocol/ProtocolErrorCode.java
index bfa5e9f..3dd836a 100644
--- a/geode-client-protocol/src/main/java/org/apache/geode/internal/protocol/ProtocolErrorCode.java
+++ b/geode-client-protocol/src/main/java/org/apache/geode/internal/protocol/ProtocolErrorCode.java
@@ -21,7 +21,8 @@ public enum ProtocolErrorCode {
   AUTHENTICATION_NOT_SUPPORTED(13),
   AUTHORIZATION_FAILED(20),
   INVALID_REQUEST(50),
-  SERVER_ERROR(100);
+  SERVER_ERROR(100),
+  NO_AVAILABLE_SERVER(101);
 
   ProtocolErrorCode(int value) {
     codeValue = value;
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 1f6bfde..92080bd 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
@@ -130,12 +130,18 @@ public class ProtobufDriver implements Driver {
                 .setGetServerRequest(LocatorAPI.GetServerRequest.newBuilder()))
             .build().writeDelimitedTo(outputStream);
 
-        LocatorAPI.GetServerResponse getServerResponse = ClientProtocol.Message
-            .parseDelimitedFrom(inputStream).getResponse().getGetServerResponse();
-        if (!getServerResponse.hasServer()) {
-          continue;
+        ClientProtocol.Response response =
+            ClientProtocol.Message.parseDelimitedFrom(inputStream).getResponse();
+        ClientProtocol.ErrorResponse errorResponse = response.getErrorResponse();
+
+        if (errorResponse != null && errorResponse.hasError()) {
+          throw new IOException(
+              "Error finding server: error code= " + errorResponse.getError().getErrorCode()
+                  + "; error message=" + errorResponse.getError().getMessage());
         }
 
+        LocatorAPI.GetServerResponse getServerResponse = response.getGetServerResponse();
+
         BasicTypes.Server server = getServerResponse.getServer();
         return new InetSocketAddress(server.getHostname(), server.getPort());
       } catch (IOException e) {
diff --git a/geode-experimental-driver/src/test/java/org/apache/geode/experimental/driver/DriverConnectionTest.java b/geode-experimental-driver/src/test/java/org/apache/geode/experimental/driver/DriverConnectionTest.java
new file mode 100644
index 0000000..b12c5c7
--- /dev/null
+++ b/geode-experimental-driver/src/test/java/org/apache/geode/experimental/driver/DriverConnectionTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+
+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.RegionShortcut;
+import org.apache.geode.cache.server.CacheServer;
+import org.apache.geode.distributed.ConfigurationProperties;
+import org.apache.geode.distributed.Locator;
+import org.apache.geode.pdx.PdxInstance;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class DriverConnectionTest {
+  @Rule
+  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+
+  /** a JSON document */
+  private static final String jsonDocument =
+      "{" + System.lineSeparator() + "  \"name\" : \"Charlemagne\"," + System.lineSeparator()
+          + "  \"age\" : 1276," + System.lineSeparator() + "  \"nationality\" : \"french\","
+          + System.lineSeparator() + "  \"emailAddress\" : \"none\"" + System.lineSeparator() + "}";
+
+
+  private Locator locator;
+  private Cache cache;
+  private Driver driver;
+  private int locatorPort;
+
+  @Before
+  public void createServerAndDriver() throws Exception {
+    System.setProperty("geode.feature-protobuf-protocol", "true");
+
+    // Create a cache
+    CacheFactory cf = new CacheFactory();
+    cf.set(ConfigurationProperties.MCAST_PORT, "0");
+    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).create();
+  }
+
+}
diff --git a/geode-protobuf-messages/src/main/proto/v1/basicTypes.proto b/geode-protobuf-messages/src/main/proto/v1/basicTypes.proto
index e2c676c..2fa98b8 100644
--- a/geode-protobuf-messages/src/main/proto/v1/basicTypes.proto
+++ b/geode-protobuf-messages/src/main/proto/v1/basicTypes.proto
@@ -77,4 +77,5 @@ enum ErrorCode {
     AUTHORIZATION_FAILED = 20;
     INVALID_REQUEST = 50;
     SERVER_ERROR = 100;
+    NO_AVAILABLE_SERVER = 101;
 }
diff --git a/geode-protobuf-messages/src/main/proto/v1/locator_API.proto b/geode-protobuf-messages/src/main/proto/v1/locator_API.proto
index 0f3f574..1f41000 100644
--- a/geode-protobuf-messages/src/main/proto/v1/locator_API.proto
+++ b/geode-protobuf-messages/src/main/proto/v1/locator_API.proto
@@ -23,6 +23,7 @@ package org.apache.geode.internal.protocol.protobuf.v1;
 
 import "v1/basicTypes.proto";
 
+// this will return failure code LOCATOR_ERROR if unable to find a suitable server
 message GetServerRequest {
     repeated Server excludedServers = 1;
     string serverGroup = 2;
diff --git a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/v1/operations/GetServerOperationHandler.java b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/v1/operations/GetServerOperationHandler.java
index f3a5687..5f7615c 100644
--- a/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/v1/operations/GetServerOperationHandler.java
+++ b/geode-protobuf/src/main/java/org/apache/geode/internal/protocol/protobuf/v1/operations/GetServerOperationHandler.java
@@ -14,14 +14,11 @@
  */
 package org.apache.geode.internal.protocol.protobuf.v1.operations;
 
-import static org.apache.geode.internal.protocol.ProtocolErrorCode.INVALID_REQUEST;
+import static org.apache.geode.internal.protocol.ProtocolErrorCode.NO_AVAILABLE_SERVER;
 
-import java.util.ArrayList;
-import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-import java.util.stream.Collectors;
 
 import org.apache.geode.annotations.Experimental;
 import org.apache.geode.cache.client.internal.locator.ClientConnectionRequest;
@@ -33,14 +30,12 @@ import org.apache.geode.internal.protocol.Failure;
 import org.apache.geode.internal.protocol.MessageExecutionContext;
 import org.apache.geode.internal.protocol.Result;
 import org.apache.geode.internal.protocol.Success;
-import org.apache.geode.internal.protocol.operations.OperationHandler;
 import org.apache.geode.internal.protocol.operations.ProtobufOperationHandler;
 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.LocatorAPI;
 import org.apache.geode.internal.protocol.protobuf.v1.ProtobufSerializationService;
 import org.apache.geode.internal.protocol.protobuf.v1.utilities.ProtobufResponseUtilities;
-import org.apache.geode.internal.protocol.serialization.SerializationService;
 import org.apache.geode.internal.protocol.state.ConnectionTerminatingStateProcessor;
 
 @Experimental
@@ -78,15 +73,17 @@ public class GetServerOperationHandler
       serverLocation = connectionResponse.getServer();
     }
 
-    LocatorAPI.GetServerResponse.Builder builder = LocatorAPI.GetServerResponse.newBuilder();
+    if (serverLocation == null) {
+      return Failure.of(ProtobufResponseUtilities.makeErrorResponse(NO_AVAILABLE_SERVER,
+          "Unable to find a server for you"));
 
-    if (serverLocation != null) {
+    } else {
+      LocatorAPI.GetServerResponse.Builder builder = LocatorAPI.GetServerResponse.newBuilder();
       BasicTypes.Server.Builder serverBuilder = BasicTypes.Server.newBuilder();
       serverBuilder.setHostname(serverLocation.getHostName()).setPort(serverLocation.getPort());
       BasicTypes.Server server = serverBuilder.build();
       builder.setServer(server);
+      return Success.of(builder.build());
     }
-
-    return Success.of(builder.build());
   }
 }
diff --git a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/v1/operations/GetServerOperationHandlerJUnitTest.java b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/v1/operations/GetServerOperationHandlerJUnitTest.java
index bebd589..cb24092 100644
--- a/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/v1/operations/GetServerOperationHandlerJUnitTest.java
+++ b/geode-protobuf/src/test/java/org/apache/geode/internal/protocol/protobuf/v1/operations/GetServerOperationHandlerJUnitTest.java
@@ -14,29 +14,27 @@
  */
 package org.apache.geode.internal.protocol.protobuf.v1.operations;
 
+import static org.apache.geode.internal.protocol.protobuf.v1.BasicTypes.ErrorCode.NO_AVAILABLE_SERVER;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import java.util.ArrayList;
-
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import org.apache.geode.cache.client.internal.locator.ClientConnectionRequest;
 import org.apache.geode.cache.client.internal.locator.ClientConnectionResponse;
 import org.apache.geode.distributed.internal.InternalLocator;
-import org.apache.geode.distributed.internal.LocatorLoadSnapshot;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.distributed.internal.ServerLocator;
+import org.apache.geode.internal.protocol.Failure;
 import org.apache.geode.internal.protocol.Result;
 import org.apache.geode.internal.protocol.Success;
 import org.apache.geode.internal.protocol.TestExecutionContext;
 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.LocatorAPI;
 import org.apache.geode.internal.protocol.protobuf.v1.LocatorAPI.GetServerResponse;
 import org.apache.geode.internal.protocol.protobuf.v1.utilities.ProtobufRequestUtilities;
@@ -78,15 +76,17 @@ public class GetServerOperationHandlerJUnitTest extends OperationHandlerJUnitTes
   }
 
   @Test
-  public void testWhenServersFromSnapshotAreNullDoesNotReturnAServer() throws Exception {
+  public void testExceptionReturnedWhenNoServers() throws Exception {
     when(serverLocatorAdviseeMock.processRequest(any(Object.class))).thenReturn(null);
 
     LocatorAPI.GetServerRequest GetServerRequest =
         ProtobufRequestUtilities.createGetServerRequest();
     Result operationHandlerResult = getOperationHandlerResult(GetServerRequest);
-    assertTrue(operationHandlerResult instanceof Success);
-    GetServerResponse serverResponse = (GetServerResponse) operationHandlerResult.getMessage();
-    assertFalse(serverResponse.hasServer());
+    assertTrue(operationHandlerResult instanceof Failure);
+    Failure failure = (Failure) operationHandlerResult;
+    ClientProtocol.ErrorResponse errorResponse =
+        (ClientProtocol.ErrorResponse) failure.getErrorMessage();
+    assertEquals(NO_AVAILABLE_SERVER, errorResponse.getError().getErrorCode());
   }
 
   private Result getOperationHandlerResult(LocatorAPI.GetServerRequest GetServerRequest)

-- 
To stop receiving notification emails like this one, please contact
['"commits@geode.apache.org" <co...@geode.apache.org>'].