You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2020/07/23 23:47:19 UTC

[geode] branch support/1.13 updated: GEODE-8331: allow GFSH to connect to other versions of cluster (#5375)

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

jinmeiliao pushed a commit to branch support/1.13
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/support/1.13 by this push:
     new 903eff9  GEODE-8331: allow GFSH to connect to other versions of cluster (#5375)
903eff9 is described below

commit 903eff973128a9995e66decdd1b5be89172f55f9
Author: Jinmei Liao <ji...@pivotal.io>
AuthorDate: Thu Jul 16 08:47:34 2020 -0700

    GEODE-8331: allow GFSH to connect to other versions of cluster (#5375)
---
 geode-gfsh/build.gradle                            |  9 +++
 .../internal/cli/commands/ConnectCommand.java      | 31 +++-----
 .../geode/management/internal/cli/help/Helper.java | 23 ++++++
 .../cli/remote/OnlineCommandProcessor.java         |  6 +-
 .../geode/management/internal/cli/shell/Gfsh.java  |  1 -
 .../internal/cli/shell/GfshExecutionStrategy.java  | 20 ++++-
 .../internal/cli/commands/ConnectCommandTest.java  | 50 ++++--------
 .../cli/remote/OnlineCommandProcessorTest.java     |  5 +-
 .../cli/shell/GfshExecutionStrategyTest.java       | 14 ++++
 .../geode/management/GfshCompatibilityTest.java    | 89 ++++++++++++++++++++++
 10 files changed, 190 insertions(+), 58 deletions(-)

diff --git a/geode-gfsh/build.gradle b/geode-gfsh/build.gradle
index 212b9d9..d82854f 100644
--- a/geode-gfsh/build.gradle
+++ b/geode-gfsh/build.gradle
@@ -53,6 +53,15 @@ dependencies {
     testCompileOnly(platform(project(':boms:geode-all-bom')))
     testCompileOnly('io.swagger:swagger-annotations')
 
+    upgradeTestImplementation(project(':geode-junit'))
+    upgradeTestImplementation(project(':geode-dunit'))
+
+    upgradeTestImplementation('org.awaitility:awaitility')
+    upgradeTestImplementation('org.assertj:assertj-core')
+    upgradeTestImplementation('junit:junit')
+    upgradeTestImplementation('xml-apis:xml-apis:2.0.2')
+    upgradeTestRuntimeOnly(project(path: ':geode-old-versions', configuration: 'testOutput'))
+
     implementation('net.sf.jopt-simple:jopt-simple')
 
     //Log4j is used everywhere
diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java
index eeb4f6b..2531b80 100644
--- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java
+++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/ConnectCommand.java
@@ -168,22 +168,22 @@ public class ConnectCommand extends OfflineGfshCommand {
       return result;
     }
 
-    String gfshVersion = gfsh.getVersion();
+    // since 1.14, only allow gfsh to connect to cluster that's older than 1.10
     String remoteVersion = null;
+    String gfshVersion = gfsh.getVersion();
     try {
-      String gfshGeodeSerializationVersion = gfsh.getGeodeSerializationVersion();
-      String remoteGeodeSerializationVersion = invoker.getRemoteGeodeSerializationVersion();
-      if (hasSameMajorMinor(gfshGeodeSerializationVersion, remoteGeodeSerializationVersion)) {
+      remoteVersion = invoker.getRemoteVersion();
+      int minorVersion = Integer.parseInt(versionComponent(remoteVersion, VERSION_MINOR));
+      if (versionComponent(remoteVersion, VERSION_MAJOR).equals("1") && minorVersion >= 10 ||
+          versionComponent(remoteVersion, VERSION_MAJOR).equals("9") && minorVersion >= 9) {
+        InfoResultModel versionInfo = result.addInfo("versionInfo");
+        versionInfo.addLine("You are connected to a cluster of version: " + remoteVersion);
         return result;
       }
-    } catch (Exception e) {
-      // we failed to get the remote geode serialization version; get remote product version for
-      // error message
-      try {
-        remoteVersion = invoker.getRemoteVersion();
-      } catch (Exception ex) {
-        gfsh.logInfo("failed to get the the remote version.", ex);
-      }
+    } catch (Exception ex) {
+      // if unable to get the remote version, we are certainly talking to
+      // a pre-1.5 cluster
+      gfsh.logInfo("failed to get the the remote version.", ex);
     }
 
     // will reach here only when remoteVersion is not available or does not match
@@ -197,13 +197,6 @@ public class ConnectCommand extends OfflineGfshCommand {
     }
   }
 
-  private static boolean hasSameMajorMinor(String gfshVersion, String remoteVersion) {
-    return versionComponent(remoteVersion, VERSION_MAJOR)
-        .equalsIgnoreCase(versionComponent(gfshVersion, VERSION_MAJOR))
-        && versionComponent(remoteVersion, VERSION_MINOR)
-            .equalsIgnoreCase(versionComponent(gfshVersion, VERSION_MINOR));
-  }
-
   private static String versionComponent(String version, int component) {
     String[] versionComponents = StringUtils.split(version, '.');
     return versionComponents.length >= component + 1 ? versionComponents[component] : "";
diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/help/Helper.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/help/Helper.java
index 9dc1c3f..33161f7 100644
--- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/help/Helper.java
+++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/help/Helper.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -465,4 +466,26 @@ public class Helper {
   public Set<String> getCommands() {
     return commands.keySet();
   }
+
+  public Method getCommandMethod(String command) {
+    return commands.get(command);
+
+  }
+
+  // methods added for future option comparison
+  public Set<String> getOptions(String command) {
+    Method method = getCommandMethod(command);
+    Set<String> optionList = new HashSet<>();
+    Annotation[][] annotations = method.getParameterAnnotations();
+    if (annotations == null || annotations.length == 0) {
+      // can't validate arguments if command doesn't have any
+      return optionList;
+    }
+
+    for (Annotation[] annotation : annotations) {
+      CliOption cliOption = getAnnotation(annotation, CliOption.class);
+      optionList.add(getPrimaryKey(cliOption));
+    }
+    return optionList;
+  }
 }
diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessor.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessor.java
index cbeea88..fa975f8 100644
--- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessor.java
+++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessor.java
@@ -104,7 +104,11 @@ public class OnlineCommandProcessor implements CommandProcessor {
     ParseResult parseResult = parseCommand(commentLessLine);
 
     if (parseResult == null) {
-      return ResultModel.createError("Could not parse command string. " + command);
+      String version = GemFireVersion.getGemFireVersion();
+      String message = "Could not parse command string. " + command + "." + System.lineSeparator() +
+          "The command or some options in this command may not be supported by this locator(" +
+          version + ") gfsh is connected with.";
+      return ResultModel.createError(message);
     }
 
     Method method = parseResult.getMethod();
diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
index ea35931..b954c1e 100755
--- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
+++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/shell/Gfsh.java
@@ -675,7 +675,6 @@ public class Gfsh extends JLineShell {
     if (full) {
       return GemFireVersion.asString();
     } else {
-
       return GemFireVersion.getGemFireVersion();
     }
   }
diff --git a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
index d78b267..f05da28 100755
--- a/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
+++ b/geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategy.java
@@ -26,6 +26,7 @@ import org.springframework.shell.event.ParseResult;
 import org.springframework.util.Assert;
 
 import org.apache.geode.internal.ClassPathLoader;
+import org.apache.geode.internal.GemFireVersion;
 import org.apache.geode.management.cli.CliMetaData;
 import org.apache.geode.management.cli.Result.Status;
 import org.apache.geode.management.internal.cli.CliAroundInterceptor;
@@ -203,8 +204,23 @@ public class GfshExecutionStrategy implements ExecutionStrategy {
     // it can also be a Path to a temp file downloaded from the rest http request
     ResultModel commandResult = null;
     if (response instanceof String) {
-      commandResult = ResultModel.fromJson((String) response);
-
+      try {
+        commandResult = ResultModel.fromJson((String) response);
+      } catch (Exception e) {
+        logWrapper.severe("Unable to parse the remote response.", e);
+        String clientVersion = GemFireVersion.getGemFireVersion();
+        String remoteVersion = null;
+        try {
+          remoteVersion = shell.getOperationInvoker().getRemoteVersion();
+        } catch (Exception exception) {
+          // unable to get the remote version (pre-1.5.0 manager does not have this capability)
+          // ignore
+        }
+        String message = "Unable to parse the remote response. This might due to gfsh client "
+            + "version(" + clientVersion + ") mismatch with the remote cluster version"
+            + ((remoteVersion == null) ? "." : "(" + remoteVersion + ").");
+        return ResultModel.createError(message);
+      }
     } else if (response instanceof Path) {
       tempFile = (Path) response;
     }
diff --git a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandTest.java b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandTest.java
index 0d7a0db..329efde 100644
--- a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandTest.java
+++ b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/commands/ConnectCommandTest.java
@@ -382,57 +382,39 @@ public class ConnectCommandTest {
   }
 
   @Test
-  public void connectToManagerWithGreaterPatchVersion() {
-    when(gfsh.getGeodeSerializationVersion()).thenReturn("1.5.1");
-    when(operationInvoker.getRemoteGeodeSerializationVersion()).thenReturn("1.5.2");
+  public void connectToManagerOlderThan1_10() {
+    when(operationInvoker.getRemoteVersion()).thenReturn("1.10");
     when(operationInvoker.isConnected()).thenReturn(true);
-    when(resultModel.getStatus()).thenReturn(Result.Status.OK);
-
-    gfshParserRule.executeAndAssertThat(connectCommand, "connect --locator=localhost:4040")
-        .statusIsSuccess();
-  }
-
-  @Test
-  public void connectToManagerWithNoPatchVersion() {
-    when(gfsh.getGeodeSerializationVersion()).thenReturn("1.5.1");
-    when(operationInvoker.getRemoteGeodeSerializationVersion()).thenReturn("1.5");
-    when(operationInvoker.isConnected()).thenReturn(true);
-    when(resultModel.getStatus()).thenReturn(Result.Status.OK);
 
-    gfshParserRule.executeAndAssertThat(connectCommand, "connect --locator=localhost:4040")
-        .statusIsSuccess();
-  }
-
-  @Test
-  public void connectToManagerWithLessorPatchVersion() {
-    when(gfsh.getGeodeSerializationVersion()).thenReturn("1.5.1");
-    when(operationInvoker.getRemoteGeodeSerializationVersion()).thenReturn("1.5.0");
-    when(operationInvoker.isConnected()).thenReturn(true);
-    when(resultModel.getStatus()).thenReturn(Result.Status.OK);
+    ResultModel resultModel = new ResultModel();
+    when(connectCommand.jmxConnect(any(), anyBoolean(), any(), any(), anyBoolean()))
+        .thenReturn(resultModel);
 
     gfshParserRule.executeAndAssertThat(connectCommand, "connect --locator=localhost:4040")
-        .statusIsSuccess();
+        .statusIsSuccess()
+        .containsOutput("You are connected to a cluster of version: 1.10");
   }
 
   @Test
-  public void connectToOlderManagerWithNewerGfsh() {
-    when(gfsh.getVersion()).thenReturn("1.5");
+  public void connectToOlderManagerWithNoRemoteVersion() {
+    when(gfsh.getVersion()).thenReturn("1.14");
     when(operationInvoker.getRemoteVersion())
         .thenThrow(new RuntimeException("release version not available"));
     when(operationInvoker.isConnected()).thenReturn(true);
 
     gfshParserRule.executeAndAssertThat(connectCommand, "connect --locator=localhost:4040")
-        .statusIsError().containsOutput("Cannot use a 1.5 gfsh client to connect to this cluster.");
+        .statusIsError()
+        .containsOutput("Cannot use a 1.14 gfsh client to connect to this cluster.");
   }
 
   @Test
-  public void connectToAValidManager() {
-    when(gfsh.getGeodeSerializationVersion()).thenReturn("1.5");
-    when(operationInvoker.getRemoteGeodeSerializationVersion()).thenReturn("1.5");
+  public void connectToManagerBefore1_10() {
+    when(gfsh.getVersion()).thenReturn("1.14");
+    when(operationInvoker.getRemoteVersion()).thenReturn("1.9");
     when(operationInvoker.isConnected()).thenReturn(true);
 
-    when(resultModel.getStatus()).thenReturn(Result.Status.OK);
     gfshParserRule.executeAndAssertThat(connectCommand, "connect --locator=localhost:4040")
-        .statusIsSuccess();
+        .statusIsError()
+        .containsOutput("Cannot use a 1.14 gfsh client to connect to a 1.9 cluster");
   }
 }
diff --git a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessorTest.java b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessorTest.java
index 472f4ca..6c04a5b 100644
--- a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessorTest.java
+++ b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/remote/OnlineCommandProcessorTest.java
@@ -89,6 +89,9 @@ public class OnlineCommandProcessorTest {
   public void handlesParsingError() {
     ResultModel commandResult = onlineCommandProcessor.executeCommand("foo --bar");
     assertThat(commandResult).isInstanceOf(ResultModel.class);
-    assertThat(commandResult.toString()).contains("Could not parse command string. foo --bar");
+    assertThat(commandResult.toString())
+        .contains("Could not parse command string. foo --bar")
+        .contains(
+            "The command or some options in this command may not be supported by this locator");
   }
 }
diff --git a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyTest.java b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyTest.java
index 287a462..7ffb205 100644
--- a/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyTest.java
+++ b/geode-gfsh/src/test/java/org/apache/geode/management/internal/cli/shell/GfshExecutionStrategyTest.java
@@ -103,6 +103,20 @@ public class GfshExecutionStrategyTest {
   }
 
   @Test
+  public void testOnLineCommandWhenGfshReceivesInvalidJson() throws Exception {
+    when(parsedCommand.getMethod()).thenReturn(Commands.class.getDeclaredMethod("onlineCommand"));
+    when(parsedCommand.getInstance()).thenReturn(new Commands());
+    when(gfsh.isConnectedAndReady()).thenReturn(true);
+    OperationInvoker invoker = mock(OperationInvoker.class);
+
+    when(invoker.processCommand(any(CommandRequest.class))).thenReturn("invalid-json");
+    when(gfsh.getOperationInvoker()).thenReturn(invoker);
+    Result result = (Result) gfshExecutionStrategy.execute(parsedCommand);
+    assertThat(result.getStatus()).isEqualTo(Result.Status.ERROR);
+    assertThat(result.nextLine().trim()).contains("Unable to parse the remote response.");
+  }
+
+  @Test
   public void resolveInterceptorClassName() throws Exception {
     when(parsedCommand.getMethod())
         .thenReturn(Commands.class.getDeclaredMethod("interceptedCommand"));
diff --git a/geode-gfsh/src/upgradeTest/java/org/apache/geode/management/GfshCompatibilityTest.java b/geode-gfsh/src/upgradeTest/java/org/apache/geode/management/GfshCompatibilityTest.java
new file mode 100644
index 0000000..07690ec
--- /dev/null
+++ b/geode-gfsh/src/upgradeTest/java/org/apache/geode/management/GfshCompatibilityTest.java
@@ -0,0 +1,89 @@
+/*
+ * 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.management;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.BackwardCompatibilityTest;
+import org.apache.geode.test.junit.rules.GfshCommandRule;
+import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+import org.apache.geode.test.version.TestVersion;
+import org.apache.geode.test.version.VersionManager;
+
+@Category({BackwardCompatibilityTest.class})
+@RunWith(Parameterized.class)
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+public class GfshCompatibilityTest {
+  private final String oldVersion;
+
+  @Parameterized.Parameters(name = "{0}")
+  public static Collection<String> data() {
+    List<String> result = VersionManager.getInstance().getVersionsWithoutCurrent();
+    return result;
+  }
+
+  public GfshCompatibilityTest(String oldVersion) {
+    this.oldVersion = oldVersion;
+  }
+
+  private MemberVM oldLocator;
+
+  @Rule
+  public GfshCommandRule gfsh = new GfshCommandRule();
+
+  @Rule
+  public ClusterStartupRule cluster = new ClusterStartupRule();
+
+  @Test
+  public void currentGfshConnectToOlderVersionsOfLocator() throws Exception {
+    oldLocator = cluster.startLocatorVM(0, oldVersion);
+    int locatorPort = oldLocator.getPort();
+    cluster.startServerVM(1, oldVersion,
+        s -> s.withConnectionToLocator(locatorPort));
+    // pre 1.10, it should not be able to connect
+    if (TestVersion.compare(oldVersion, "1.5.0") < 0) {
+      gfsh.connect(oldLocator.getPort(), GfshCommandRule.PortType.locator);
+      assertThat(gfsh.isConnected()).isFalse();
+      assertThat(gfsh.getGfshOutput()).contains("Cannot use a")
+          .contains("gfsh client to connect to this cluster.");
+    } else if (TestVersion.compare(oldVersion, "1.10.0") < 0) {
+      gfsh.connect(oldLocator.getPort(), GfshCommandRule.PortType.locator);
+      assertThat(gfsh.isConnected()).isFalse();
+      assertThat(gfsh.getGfshOutput()).contains("Cannot use a")
+          .contains("gfsh client to connect to a " + oldVersion + " cluster.");
+    }
+    // post 1.10 (including) should connect and be able to execute command
+    else {
+      gfsh.connectAndVerify(oldLocator);
+      assertThat(gfsh.getGfshOutput())
+          .contains("You are connected to a cluster of version: " + oldVersion);
+      gfsh.executeAndAssertThat("list members")
+          .statusIsSuccess();
+    }
+  }
+
+}