You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by mh...@apache.org on 2020/10/22 16:47:38 UTC

[geode] branch feature/GEODE-7665 updated: GEODE-7845: Now behaving with clients of various versions. (#5645)

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

mhanson pushed a commit to branch feature/GEODE-7665
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/feature/GEODE-7665 by this push:
     new 05055cf  GEODE-7845: Now behaving with clients of various versions. (#5645)
05055cf is described below

commit 05055cf4f4066b65e93f905648dbd4957ae6ef72
Author: mhansonp <ha...@vmware.com>
AuthorDate: Thu Oct 22 09:46:47 2020 -0700

    GEODE-7845: Now behaving with clients of various versions. (#5645)
    
    - added functionality that would allow the tests to be run using various versions of the clients against and and new versions of the server.
---
 .../integrationTest/resources/assembly_content.txt |  1 -
 .../main/java/org/apache/geode/cache/Region.java   |  2 +-
 .../cache/ServerVersionMismatchException.java      | 34 --------
 .../internal/cache/PartitionedRegionClear.java     |  8 +-
 .../sanctioned-geode-core-serializables.txt        |  1 -
 .../internal/cache/PartitionedRegionClearTest.java |  5 +-
 ...ePartitionRegionClearServerVersionMismatch.java | 94 +++++++++++++++-------
 7 files changed, 73 insertions(+), 72 deletions(-)

diff --git a/geode-assembly/src/integrationTest/resources/assembly_content.txt b/geode-assembly/src/integrationTest/resources/assembly_content.txt
index 553785a..549150f 100644
--- a/geode-assembly/src/integrationTest/resources/assembly_content.txt
+++ b/geode-assembly/src/integrationTest/resources/assembly_content.txt
@@ -245,7 +245,6 @@ javadoc/org/apache/geode/cache/RoleEvent.html
 javadoc/org/apache/geode/cache/RoleException.html
 javadoc/org/apache/geode/cache/Scope.html
 javadoc/org/apache/geode/cache/SerializedCacheValue.html
-javadoc/org/apache/geode/cache/ServerVersionMismatchException.html
 javadoc/org/apache/geode/cache/StatisticsDisabledException.html
 javadoc/org/apache/geode/cache/SubscriptionAttributes.html
 javadoc/org/apache/geode/cache/SynchronizationCommitConflictException.html
diff --git a/geode-core/src/main/java/org/apache/geode/cache/Region.java b/geode-core/src/main/java/org/apache/geode/cache/Region.java
index 5162bd5..1961e81 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/Region.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/Region.java
@@ -1307,7 +1307,7 @@ public interface Region<K, V> extends ConcurrentMap<K, V> {
    * @throws PartitionedRegionPartialClearException when data is partially cleared on partitioned
    *         region. It is caller responsibility to handle the partial data clear either by retrying
    *         the clear operation or continue working with the partially cleared partitioned region.
-   * @throws ServerVersionMismatchException when data was not cleared because one or more
+   * @throws UnsupportedOperationException when data was not cleared because one or more
    *         of the member servers' version was too old to understand the clear message.
    */
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/cache/ServerVersionMismatchException.java b/geode-core/src/main/java/org/apache/geode/cache/ServerVersionMismatchException.java
deleted file mode 100644
index 1d4231a..0000000
--- a/geode-core/src/main/java/org/apache/geode/cache/ServerVersionMismatchException.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.cache;
-
-import java.util.List;
-
-/**
- * Indicates a failure to perform an operation on a Partitioned Region due to
- * server versions not meeting requirements.
- *
- * @since GEODE 1.14.0
- */
-public class ServerVersionMismatchException extends CacheRuntimeException {
-  private static final long serialVersionUID = -3004093739855972548L;
-
-  public ServerVersionMismatchException(List<String> members, String featureName,
-      String version) {
-    super(
-        "A server's " + members + " version was too old (< " + version + ") for : " + featureName);
-
-  }
-}
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionClear.java b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionClear.java
index 0e5acfc..2bec6f2 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionClear.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegionClear.java
@@ -27,7 +27,6 @@ import org.apache.geode.cache.CacheWriterException;
 import org.apache.geode.cache.Operation;
 import org.apache.geode.cache.OperationAbortedException;
 import org.apache.geode.cache.PartitionedRegionPartialClearException;
-import org.apache.geode.cache.ServerVersionMismatchException;
 import org.apache.geode.cache.partition.PartitionRegionHelper;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.MembershipListener;
@@ -349,13 +348,14 @@ public class PartitionedRegionClear {
           && (internalDistributedMember.getVersion().isOlderThan(KnownVersion.GEODE_1_14_0))) {
         if (!memberNames.contains(internalDistributedMember.getName())) {
           memberNames.add(internalDistributedMember.getName());
-          logger.info("MLH adding " + internalDistributedMember.getName());
         }
       }
     }
     if (!memberNames.isEmpty()) {
-      throw new ServerVersionMismatchException(memberNames, "Partitioned Region Clear",
-          KnownVersion.GEODE_1_14_0.toString());
+      throw new UnsupportedOperationException(
+          "A server's " + memberNames + " version was too old (< "
+              + KnownVersion.GEODE_1_14_0 + ") for : Partitioned Region Clear");
+
     }
   }
 
diff --git a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
index 86e2372..644fbc2 100644
--- a/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
+++ b/geode-core/src/main/resources/org/apache/geode/internal/sanctioned-geode-core-serializables.txt
@@ -92,7 +92,6 @@ org/apache/geode/cache/ResourceException,true,-5559328592343363268
 org/apache/geode/cache/ResumptionAction,true,6632254151314915610,ordinal:byte
 org/apache/geode/cache/RoleException,true,-7521056108445887394
 org/apache/geode/cache/Scope,true,5534399159504301602,ordinal:int
-org/apache/geode/cache/ServerVersionMismatchException,true,-3004093739855972548
 org/apache/geode/cache/StatisticsDisabledException,true,-2987721454129719551
 org/apache/geode/cache/SynchronizationCommitConflictException,true,2619806460255259492
 org/apache/geode/cache/TimeoutException,true,-6260761691185737442
diff --git a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionClearTest.java b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionClearTest.java
index bd78fd0..4d8a14e 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionClearTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/cache/PartitionedRegionClearTest.java
@@ -37,7 +37,6 @@ import org.mockito.ArgumentCaptor;
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.cache.PartitionedRegionPartialClearException;
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache.ServerVersionMismatchException;
 import org.apache.geode.distributed.DistributedLockService;
 import org.apache.geode.distributed.internal.DMStats;
 import org.apache.geode.distributed.internal.DistributionManager;
@@ -559,7 +558,7 @@ public class PartitionedRegionClearTest {
   }
 
   @Test
-  public void doClearThrowsServerVersionMismatchException() {
+  public void doClearThrowsUnsupportedOperationException() {
     boolean cacheWrite = false;
     RegionEventImpl regionEvent = mock(RegionEventImpl.class);
     when(partitionedRegion.hasListener()).thenReturn(false);
@@ -615,7 +614,7 @@ public class PartitionedRegionClearTest {
         catchThrowable(() -> spyPartitionedRegionClear.doClear(regionEvent, cacheWrite));
 
     assertThat(thrown)
-        .isInstanceOf(ServerVersionMismatchException.class)
+        .isInstanceOf(UnsupportedOperationException.class)
         .hasMessage(
             "A server's [oldMember] version was too old (< GEODE 1.14.0) for : Partitioned Region Clear");
   }
diff --git a/geode-core/src/upgradeTest/java/org/apache/geode/internal/cache/rollingupgrade/RollingUpgradePartitionRegionClearServerVersionMismatch.java b/geode-core/src/upgradeTest/java/org/apache/geode/internal/cache/rollingupgrade/RollingUpgradePartitionRegionClearServerVersionMismatch.java
index 144ea38..b3643fe 100644
--- a/geode-core/src/upgradeTest/java/org/apache/geode/internal/cache/rollingupgrade/RollingUpgradePartitionRegionClearServerVersionMismatch.java
+++ b/geode-core/src/upgradeTest/java/org/apache/geode/internal/cache/rollingupgrade/RollingUpgradePartitionRegionClearServerVersionMismatch.java
@@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.catchThrowable;
 import java.util.Collection;
 import java.util.List;
 
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -32,7 +33,6 @@ import org.junit.runners.Parameterized;
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.cache.ServerVersionMismatchException;
 import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.ClientRegionFactory;
 import org.apache.geode.cache.client.ClientRegionShortcut;
@@ -46,12 +46,12 @@ import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactor
 import org.apache.geode.test.version.VersionManager;
 
 /**
- * This test class exists to test the ServerVersionMismatchException
- * A ServerVersionMismatchException is thrown when a cluster has a server that is previous to
- * version 1.14.0 which doesn't support the Partitioned Region Clear feature.
- *
- * When the exception is thrown it is expected to contain the members that have the bad version,
- * the version number necessary, and the feature that is not supported.
+ * This test class exists to test the ServerOperationException A ServerOperationException is thrown
+ * when a cluster has a server that is previous to version 1.14.0 which doesn't support the
+ * Partitioned Region Clear feature.
+ * <p>
+ * When the exception is thrown it is expected to contain the members that have the bad version, the
+ * version number necessary, and the feature that is not supported.
  */
 
 
@@ -65,6 +65,9 @@ public class RollingUpgradePartitionRegionClearServerVersionMismatch {
   @Parameterized.Parameter
   public String oldVersion;
 
+  private MemberVM serverOld;
+  private ClientVM clientVM;
+
   @Parameterized.Parameters(name = "from_v{0}")
   public static Collection<String> data() {
     List<String> result = VersionManager.getInstance().getVersionsWithoutCurrent();
@@ -88,11 +91,10 @@ public class RollingUpgradePartitionRegionClearServerVersionMismatch {
     locator = cluster.startLocatorVM(0,
         l -> l.withSystemProperty("gemfire.allow_old_members_to_join_for_testing", "true")
             .withProperty(DistributionConfig.ENABLE_CLUSTER_CONFIGURATION_NAME, "false"));
-    final int locatorPort = locator.getPort();
+    int locatorPort = locator.getPort();
 
     serverNew = cluster.startServerVM(1, locatorPort);
-    MemberVM serverOld =
-        cluster.startServerVM(2, oldVersion, s -> s.withConnectionToLocator(locatorPort));
+    serverOld = cluster.startServerVM(2, oldVersion, s -> s.withConnectionToLocator(locatorPort));
 
     MemberVM.invokeInEveryMember(() -> {
       Cache cache = getCache();
@@ -109,20 +111,28 @@ public class RollingUpgradePartitionRegionClearServerVersionMismatch {
       region.put("A", "ValueA");
       region.put("B", "ValueB");
     });
+  }
+
+  @After
+  public void after() {
+    locator.stop();
+    serverNew.stop();
+    serverOld.stop();
 
   }
 
   /**
-   * testClient_ServerVersionMismatchException - validates that when a client invokes a partitioned
-   * region clear on a cluster where one server is running an unsupported version for this feature
-   * we return a ServerVersionMismatchException
+   * testClient_UnsupportedOperationExceptionCurrentServerVersion - validates that when a client
+   * invokes a partitioned region clear on a cluster where one server is running an
+   * unsupported version for this feature we return a UnsupportedOperationException
    */
   @Test
-  public void testClient_ServerVersionMismatchException() throws Exception {
+  public void testClient_UnsupportedOperationExceptionCurrentServerVersion() throws Exception {
     IgnoredException.addIgnoredException(ServerOperationException.class);
-    final int locatorPort = locator.getPort();
+
     // Get a client VM
-    ClientVM clientVM = cluster.startClientVM(3, c -> c.withLocatorConnection(locatorPort));
+    int serverPort = serverNew.getPort();
+    clientVM = cluster.startClientVM(3, oldVersion, c -> c.withServerConnection(serverPort));
 
     clientVM.invoke(() -> {
       // Validate we have a cache and region
@@ -134,26 +144,54 @@ public class RollingUpgradePartitionRegionClearServerVersionMismatch {
       Region<String, String> region = clientRegionFactory.create("regionA");
       assertThat(region).isNotNull();
 
-      // Validate that we get a ServerVersionMismatchException wrapped in a ServerOperationException
+      // Validate that we get a UnsupportedOperationException wrapped in a ServerOperationException
       Throwable thrown = catchThrowable(region::clear);
+      assertThat(thrown).isNotNull();
       assertThat(thrown).isInstanceOf(ServerOperationException.class);
-      assertThat(thrown).hasCauseInstanceOf(ServerVersionMismatchException.class);
+      Throwable cause = thrown.getCause();
+      assertThat(cause).isInstanceOf(UnsupportedOperationException.class);
+      assertThat(cause.getMessage()).contains(expectedMessage);
+    });
+  }
 
-      // Validate that the message is exactly as we expect it.
-      ServerVersionMismatchException serverVersionMismatchException =
-          (ServerVersionMismatchException) thrown.getCause();
-      assertThat(serverVersionMismatchException.getMessage()).isEqualTo(expectedMessage);
+  @Test
+  public void testClient_UnsupportedOperationExceptionOldServerVersion() throws Exception {
+    IgnoredException.addIgnoredException(ServerOperationException.class);
+
+    // Get a client VM
+    int serverPort = serverOld.getPort();
+    clientVM = cluster.startClientVM(3, oldVersion, c -> c.withServerConnection(serverPort));
+
+    clientVM.invoke(() -> {
+      // Validate we have a cache and region
+      ClientCache clientCache = getClientCache();
+      assertThat(clientCache).isNotNull();
+      ClientRegionFactory<String, String> clientRegionFactory =
+          clientCache.createClientRegionFactory(ClientRegionShortcut.PROXY);
+      Region<String, String> region = clientRegionFactory.create("regionA");
+      assertThat(region).isNotNull();
+
+      // Validate that we get a UnsupportedOperationException wrapped in a ServerOperationException
+      Throwable thrown = catchThrowable(region::clear);
+      assertThat(thrown).isNotNull();
+      assertThat(thrown).isInstanceOf(ServerOperationException.class);
+      assertThat(thrown.getMessage()).contains("While performing a remote clear region");
+      assertThat(thrown.getCause()).isNotNull();
+      Throwable cause = thrown.getCause();
+      assertThat(cause).isInstanceOf(UnsupportedOperationException.class);
+      assertThat(cause.getMessage()).isNull();
     });
   }
 
+
   /**
-   * testServer_ServerVersionMismatchException - validates that when a partitioned region clear is
+   * testServer_UnsupportedOperationException - validates that when a partitioned region clear is
    * invoked on a cluster where one server is running an unsupported version for this feature we
-   * return a ServerVersionMismatchException
+   * return a UnsupportedOperationException
    */
   @Test
-  public void testServer_ServerVersionMismatchException() {
-    IgnoredException.addIgnoredException(ServerOperationException.class);
+  public void testServer_UnsupportedOperationException() {
+    IgnoredException.addIgnoredException(UnsupportedOperationException.class);
 
     serverNew.invoke(() -> {
       // Validate we have a cache and region
@@ -164,8 +202,8 @@ public class RollingUpgradePartitionRegionClearServerVersionMismatch {
       assertThat(region).isNotNull();
 
       // Validate that the message is exactly as we expect it.
-      assertThatThrownBy(region::clear).isInstanceOf(ServerVersionMismatchException.class)
-          .hasMessage(expectedMessage);
+      assertThatThrownBy(region::clear).isInstanceOf(UnsupportedOperationException.class)
+          .hasMessageContaining(expectedMessage);
 
       assertThat(region.get("A")).isEqualTo("ValueA");
       assertThat(region.get("B")).isEqualTo("ValueB");