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

[geode] 01/03: GEODE-8136: Move UncheckedUtils to geode-common (#5123)

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

burcham pushed a commit to branch backport-1-13-GEODE-8652-and-friends
in repository https://gitbox.apache.org/repos/asf/geode.git

commit ca51909fe78be707376d9d411d709a4cea3bbdb2
Author: Kirk Lund <kl...@apache.org>
AuthorDate: Tue May 26 10:24:44 2020 -0700

    GEODE-8136: Move UncheckedUtils to geode-common (#5123)
    
    Create UncheckedUtilsTest to unit test UncheckedUtils.
    
    Extract FunctionExecution methods to new TypedFunctionService in
    geode-core.
    
    (cherry picked from commit 253d667b27423e55601e925f7a20f2fd6f0efc31)
---
 .../geode/util/internal}/UncheckedUtils.java       | 19 ++++---
 .../geode/util/internal/UncheckedUtilsTest.java    | 60 ++++++++++++++++++++++
 .../cache/PartitionedRegionSingleHopDUnitTest.java | 14 ++---
 ...istributedRegionFunctionExecutionDUnitTest.java | 59 +++++++--------------
 ...oningWithColocationAndPersistenceDUnitTest.java | 12 ++---
 .../FunctionExecutionOnLonerRegressionTest.java    | 27 ++--------
 .../client/internal/ClientMetadataService.java     |  4 +-
 .../geode/internal/cache/GemFireCacheImpl.java     | 49 ++++--------------
 .../cache/InternalCacheForClientAccess.java        |  6 +--
 .../apache/geode/internal/cache/LocalRegion.java   |  7 ++-
 .../util/TypedFunctionService.java}                | 20 +++++---
 .../tier/sockets/CacheClientProxyFactory.java      |  4 +-
 .../internal/ClusterAlertMessagingTest.java        |  6 +--
 .../apache/geode/internal/tcp/TCPConduitTest.java  |  4 +-
 14 files changed, 151 insertions(+), 140 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/util/UncheckedUtils.java b/geode-common/src/main/java/org/apache/geode/util/internal/UncheckedUtils.java
similarity index 68%
copy from geode-core/src/main/java/org/apache/geode/internal/cache/util/UncheckedUtils.java
copy to geode-common/src/main/java/org/apache/geode/util/internal/UncheckedUtils.java
index c03e990..61dbd8d 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/util/UncheckedUtils.java
+++ b/geode-common/src/main/java/org/apache/geode/util/internal/UncheckedUtils.java
@@ -12,18 +12,23 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.util;
-
-import org.apache.geode.cache.execute.Execution;
+package org.apache.geode.util.internal;
 
+/**
+ * Utilities for casting and working with unchecked raw types.
+ */
 @SuppressWarnings({"unchecked", "unused"})
 public class UncheckedUtils {
 
-  public static <T> T cast(Object object) {
-    return (T) object;
+  protected UncheckedUtils() {
+    // do not instantiate
   }
 
-  public static <IN, OUT, AGG> Execution<IN, OUT, AGG> cast(Execution execution) {
-    return execution;
+  /**
+   * Casts an instance of a raw type to a parameterized type. Preference should be given to
+   * converting all code from using raw types to using parameterized types when possible.
+   */
+  public static <T> T uncheckedCast(Object object) {
+    return (T) object;
   }
 }
diff --git a/geode-common/src/test/java/org/apache/geode/util/internal/UncheckedUtilsTest.java b/geode-common/src/test/java/org/apache/geode/util/internal/UncheckedUtilsTest.java
new file mode 100644
index 0000000..7c282b7
--- /dev/null
+++ b/geode-common/src/test/java/org/apache/geode/util/internal/UncheckedUtilsTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.util.internal;
+
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.catchThrowable;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+@SuppressWarnings("unchecked")
+public class UncheckedUtilsTest {
+
+  @Test
+  public void uncheckedCast_rawList_empty() {
+    List rawList = new ArrayList();
+
+    List<String> value = uncheckedCast(rawList);
+
+    assertThat(value).isSameAs(rawList);
+  }
+
+  @Test
+  public void uncheckedCast_rawList_nonEmpty() {
+    List rawList = new ArrayList();
+    rawList.add("1");
+    rawList.add("2");
+
+    List<String> value = uncheckedCast(rawList);
+
+    assertThat(value).isSameAs(rawList);
+  }
+
+  @Test
+  public void uncheckedCast_rawList_wrongTypes() {
+    List rawList = new ArrayList();
+    rawList.add(1);
+    rawList.add(2);
+    List<String> wrongType = uncheckedCast(rawList);
+
+    Throwable thrown = catchThrowable(() -> wrongType.get(0));
+
+    assertThat(thrown).isInstanceOf(ClassCastException.class);
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionSingleHopDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionSingleHopDUnitTest.java
index 943c503..43019bf 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionSingleHopDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/PartitionedRegionSingleHopDUnitTest.java
@@ -22,7 +22,6 @@ import static org.apache.geode.cache.RegionShortcut.PARTITION;
 import static org.apache.geode.cache.RegionShortcut.PARTITION_PERSISTENT;
 import static org.apache.geode.distributed.ConfigurationProperties.ENABLE_CLUSTER_CONFIGURATION;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
-import static org.apache.geode.internal.cache.util.UncheckedUtils.cast;
 import static org.apache.geode.internal.lang.SystemPropertyHelper.GEMFIRE_PREFIX;
 import static org.apache.geode.management.ManagementService.getExistingManagementService;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
@@ -32,6 +31,7 @@ import static org.apache.geode.test.dunit.VM.getController;
 import static org.apache.geode.test.dunit.VM.getVM;
 import static org.apache.geode.test.dunit.VM.getVMId;
 import static org.apache.geode.test.dunit.rules.DistributedRule.getDistributedSystemProperties;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.Mockito.mock;
 
@@ -76,7 +76,6 @@ import org.apache.geode.cache.client.internal.ClientPartitionAdvisor;
 import org.apache.geode.cache.client.internal.InternalClientCache;
 import org.apache.geode.cache.execute.FunctionAdapter;
 import org.apache.geode.cache.execute.FunctionContext;
-import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.RegionFunctionContext;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.distributed.LocatorLauncher;
@@ -84,6 +83,7 @@ import org.apache.geode.distributed.ServerLauncher;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.internal.cache.BucketAdvisor.ServerBucketProfile;
 import org.apache.geode.internal.cache.execute.InternalFunctionInvocationTargetException;
+import org.apache.geode.internal.cache.execute.util.TypedFunctionService;
 import org.apache.geode.management.ManagementService;
 import org.apache.geode.management.membership.MembershipEvent;
 import org.apache.geode.management.membership.UniversalMembershipListenerAdapter;
@@ -1336,22 +1336,22 @@ public class PartitionedRegionSingleHopDUnitTest implements Serializable {
   }
 
   private void executeFunctions(Region<Object, Object> region) {
-    cast(FunctionService.onRegion(region))
+    TypedFunctionService.onRegion(region)
         .withFilter(filter(0))
         .execute(new PutFunction())
         .getResult();
 
-    cast(FunctionService.onRegion(region))
+    TypedFunctionService.onRegion(region)
         .withFilter(filter(0, 1))
         .execute(new PutFunction())
         .getResult();
 
-    cast(FunctionService.onRegion(region))
+    TypedFunctionService.onRegion(region)
         .withFilter(filter(0, 1, 2, 3))
         .execute(new PutFunction())
         .getResult();
 
-    cast(FunctionService.onRegion(region))
+    TypedFunctionService.onRegion(region)
         .execute(new PutFunction())
         .getResult();
   }
@@ -1389,7 +1389,7 @@ public class PartitionedRegionSingleHopDUnitTest implements Serializable {
   }
 
   private InternalCache getInternalCache(ServerLauncher serverLauncher) {
-    return cast(serverLauncher.getCache());
+    return uncheckedCast(serverLauncher.getCache());
   }
 
   private void waitForLocalBucketsCreation() {
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/execute/DistributedRegionFunctionExecutionDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/execute/DistributedRegionFunctionExecutionDUnitTest.java
index b95a68b..b308b20 100755
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/execute/DistributedRegionFunctionExecutionDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/execute/DistributedRegionFunctionExecutionDUnitTest.java
@@ -22,13 +22,13 @@ import static org.apache.geode.distributed.ConfigurationProperties.NAME;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTHENTICATOR;
 import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTH_INIT;
 import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
-import static org.apache.geode.internal.cache.execute.DistributedRegionFunctionExecutionDUnitTest.UncheckedUtils.cast;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.getTimeout;
 import static org.apache.geode.test.dunit.IgnoredException.addIgnoredException;
 import static org.apache.geode.test.dunit.VM.getController;
 import static org.apache.geode.test.dunit.VM.getVM;
 import static org.apache.geode.test.dunit.VM.toArray;
 import static org.apache.geode.util.internal.GeodeGlossary.GEMFIRE_PREFIX;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.catchThrowable;
 
@@ -59,7 +59,6 @@ import org.apache.geode.cache.client.Pool;
 import org.apache.geode.cache.client.PoolFactory;
 import org.apache.geode.cache.client.PoolManager;
 import org.apache.geode.cache.client.internal.InternalClientCache;
-import org.apache.geode.cache.execute.Execution;
 import org.apache.geode.cache.execute.Function;
 import org.apache.geode.cache.execute.FunctionContext;
 import org.apache.geode.cache.execute.FunctionException;
@@ -70,6 +69,7 @@ import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.execute.util.TypedFunctionService;
 import org.apache.geode.security.templates.DummyAuthenticator;
 import org.apache.geode.security.templates.UserPasswordAuthInit;
 import org.apache.geode.test.dunit.AsyncInvocation;
@@ -257,7 +257,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
     empty.invoke(() -> populateRegion(200));
 
     AsyncInvocation executeFunctionInReplicate1 = replicate1.invokeAsync(() -> {
-      ResultCollector<String, List<String>> resultCollector = FunctionServiceCast
+      ResultCollector<String, List<String>> resultCollector = TypedFunctionService
           .<Void, String, List<String>>onRegion(getRegion())
           .withFilter(filter)
           .execute(LongRunningFunction.class.getSimpleName(), getTimeout().toMillis(),
@@ -302,7 +302,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
 
     replicate1.invoke(() -> {
       Throwable thrown = catchThrowable(() -> {
-        FunctionServiceCast
+        TypedFunctionService
             .<Void, String, List<String>>onRegion(getRegion())
             .withFilter(filter)
             .execute(LongRunningFunction.class.getSimpleName(), 1000, MILLISECONDS);
@@ -950,7 +950,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
     }
 
     client.invoke(() -> {
-      ResultCollector<Boolean, List<Boolean>> resultCollector = FunctionServiceCast
+      ResultCollector<Boolean, List<Boolean>> resultCollector = TypedFunctionService
           .<Boolean, Boolean, List<Boolean>>onRegion(getRegion())
           .setArguments(true)
           .execute(inlineFunction("Success", true));
@@ -1158,13 +1158,13 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
   }
 
   private void executeNoResultFunction() {
-    FunctionServiceCast
+    TypedFunctionService
         .onRegion(getRegion())
         .execute(new NoResultFunction());
   }
 
   private List<Boolean> executeDistributedRegionFunction() {
-    return FunctionServiceCast
+    return TypedFunctionService
         .<Boolean, Boolean, List<Boolean>>onRegion(getRegion())
         .withFilter(filter)
         .setArguments(false)
@@ -1173,7 +1173,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
   }
 
   private void executeThrowsRuntimeExceptionFunction() {
-    FunctionServiceCast
+    TypedFunctionService
         .<Void, Void, Void>onRegion(getRegion())
         .withFilter(filter)
         .execute(new ThrowsRuntimeExceptionFunction());
@@ -1187,7 +1187,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
       filter.add("execKey-" + 100 + i);
     }
 
-    ResultCollector<Object, List<Object>> resultCollector = FunctionServiceCast
+    ResultCollector<Object, List<Object>> resultCollector = TypedFunctionService
         .<Boolean, Object, List<Object>>onRegion(getRegion())
         .withFilter(filter)
         .setArguments(true)
@@ -1200,7 +1200,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
         .as("First element of " + resultCollector.getResult())
         .isInstanceOf(CustomRuntimeException.class);
 
-    resultCollector = FunctionServiceCast
+    resultCollector = TypedFunctionService
         .<Set<String>, Object, List<Object>>onRegion(getRegion())
         .withFilter(filter)
         .setArguments(filter)
@@ -1220,7 +1220,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
 
   private void executeNoLastResultFunction() {
     Throwable thrown = catchThrowable(() -> {
-      FunctionServiceCast
+      TypedFunctionService
           .onRegion(getRegion())
           .withFilter(filter)
           .execute(new NoLastResultFunction())
@@ -1234,7 +1234,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
   private void executeUnregisteredFunction() {
     FunctionService.unregisterFunction(new DistributedRegionFunction().getId());
 
-    FunctionServiceCast
+    TypedFunctionService
         .<Void, Boolean, List<Boolean>>onRegion(getRegion())
         .withFilter(filter)
         .execute(new DistributedRegionFunction())
@@ -1242,7 +1242,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
   }
 
   private void executeFunctionFunctionInvocationTargetException() {
-    ResultCollector<Integer, List<Integer>> resultCollector = FunctionServiceCast
+    ResultCollector<Integer, List<Integer>> resultCollector = TypedFunctionService
         .<Boolean, Integer, List<Integer>>onRegion(getRegion())
         .setArguments(true)
         .execute(ThrowsFunctionInvocationTargetExceptionFunction.class.getSimpleName());
@@ -1253,7 +1253,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
 
   private void executeFunctionFunctionInvocationTargetExceptionWithoutHA() {
     Throwable thrown = catchThrowable(() -> {
-      FunctionServiceCast
+      TypedFunctionService
           .<Boolean, Integer, List<Integer>>onRegion(getRegion())
           .setArguments(true)
           .execute(ThrowsFunctionInvocationTargetExceptionFunction.class.getSimpleName())
@@ -1268,7 +1268,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
   }
 
   private void executeFunctionFunctionInvocationTargetException_ClientServer() {
-    ResultCollector<Integer, List<Integer>> resultCollector = FunctionServiceCast
+    ResultCollector<Integer, List<Integer>> resultCollector = TypedFunctionService
         .<Boolean, Integer, List<Integer>>onRegion(getRegion())
         .setArguments(true)
         .execute(ThrowsFunctionInvocationTargetExceptionFunction.class.getSimpleName());
@@ -1279,7 +1279,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
 
   private void executeFunctionFunctionInvocationTargetException_ClientServer_WithoutHA() {
     Throwable thrown = catchThrowable(() -> {
-      FunctionServiceCast
+      TypedFunctionService
           .<Boolean, Integer, List<Integer>>onRegion(getRegion())
           .setArguments(true)
           .execute(ThrowsFunctionInvocationTargetExceptionFunction.class.getSimpleName())
@@ -1294,7 +1294,7 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
   }
 
   private static <K, V> Region<K, V> getRegion() {
-    return cast(REGION.get());
+    return uncheckedCast(REGION.get());
   }
 
   private static void setRegion(Region<?, ?> region) {
@@ -1327,7 +1327,8 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
     @Override
     public void execute(FunctionContext<Object> context) {
       if (context.getArguments() instanceof Set) {
-        Set<Integer> arguments = cast(context.getArguments());
+        Set<Integer> arguments =
+            uncheckedCast(context.getArguments());
         for (int i = 0; i < arguments.size(); i++) {
           context.getResultSender().sendResult(i);
         }
@@ -1557,26 +1558,4 @@ public class DistributedRegionFunctionExecutionDUnitTest implements Serializable
       return -1;
     }
   }
-
-  @SuppressWarnings("unchecked")
-  static class FunctionServiceCast {
-
-    /**
-     * Provide unchecked cast of FunctionService.onRegion.
-     */
-    static <IN, OUT, AGG> Execution<IN, OUT, AGG> onRegion(Region<?, ?> region) {
-      return FunctionService.onRegion(region);
-    }
-  }
-
-  @SuppressWarnings({"unchecked", "unused"})
-  static class UncheckedUtils {
-
-    /**
-     * Provide unchecked cast of specified Object.
-     */
-    static <T> T cast(Object object) {
-      return (T) object;
-    }
-  }
 }
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/partitioned/fixed/FixedPartitioningWithColocationAndPersistenceDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/partitioned/fixed/FixedPartitioningWithColocationAndPersistenceDUnitTest.java
index ea7c834..99db19a 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/partitioned/fixed/FixedPartitioningWithColocationAndPersistenceDUnitTest.java
+++ b/geode-core/src/distributedTest/java/org/apache/geode/internal/cache/partitioned/fixed/FixedPartitioningWithColocationAndPersistenceDUnitTest.java
@@ -36,10 +36,10 @@ import static org.apache.geode.internal.cache.partitioned.fixed.FixedPartitionin
 import static org.apache.geode.internal.cache.partitioned.fixed.FixedPartitioningWithColocationAndPersistenceDUnitTest.Quarter.Q2;
 import static org.apache.geode.internal.cache.partitioned.fixed.FixedPartitioningWithColocationAndPersistenceDUnitTest.Quarter.Q3;
 import static org.apache.geode.internal.cache.partitioned.fixed.FixedPartitioningWithColocationAndPersistenceDUnitTest.Quarter.Q4;
-import static org.apache.geode.internal.cache.util.UncheckedUtils.cast;
 import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
 import static org.apache.geode.test.dunit.VM.getVM;
 import static org.apache.geode.test.dunit.VM.getVMId;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.catchThrowable;
 import static org.mockito.Mockito.mock;
@@ -1502,9 +1502,9 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest implements S
     PartitionedRegionDataStore dataStore = partitionedRegion.getDataStore();
 
     if (dataStore != null) {
-      return cast(new LocalDataSet(partitionedRegion, dataStore.getAllLocalBucketIds()));
+      return uncheckedCast(new LocalDataSet(partitionedRegion, dataStore.getAllLocalBucketIds()));
     }
-    return cast(new LocalDataSet(partitionedRegion, emptySet()));
+    return uncheckedCast(new LocalDataSet(partitionedRegion, emptySet()));
   }
 
   private void validateQuartersData() throws ParseException {
@@ -1555,20 +1555,20 @@ public class FixedPartitioningWithColocationAndPersistenceDUnitTest implements S
       InternalDistributedMember idmForShipment = shipments.getBucketPrimary(i);
 
       // take all the keys from the shipment for each bucket
-      Set<CustomerId> customerKey = cast(customers.getBucketKeys(i));
+      Set<CustomerId> customerKey = uncheckedCast(customers.getBucketKeys(i));
       assertThat(customerKey).isNotNull();
 
       for (CustomerId customerId : customerKey) {
         assertThat(customers.get(customerId)).isNotNull();
 
-        Set<OrderId> orderKey = cast(orders.getBucketKeys(i));
+        Set<OrderId> orderKey = uncheckedCast(orders.getBucketKeys(i));
         for (OrderId orderId : orderKey) {
           assertThat(orders.get(orderId)).isNotNull();
           if (orderId.getCustomerId().equals(customerId)) {
             assertThat(idmForOrder).isEqualTo(idmForCustomer);
           }
 
-          Set<ShipmentId> shipmentKey = cast(shipments.getBucketKeys(i));
+          Set<ShipmentId> shipmentKey = uncheckedCast(shipments.getBucketKeys(i));
           for (ShipmentId shipmentId : shipmentKey) {
             assertThat(shipments.get(shipmentId)).isNotNull();
             if (shipmentId.getOrderId().equals(orderId)) {
diff --git a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/execute/FunctionExecutionOnLonerRegressionTest.java b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/execute/FunctionExecutionOnLonerRegressionTest.java
index 08b75cf..d0c4cbc 100644
--- a/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/execute/FunctionExecutionOnLonerRegressionTest.java
+++ b/geode-core/src/integrationTest/java/org/apache/geode/internal/cache/execute/FunctionExecutionOnLonerRegressionTest.java
@@ -19,7 +19,7 @@ import static org.apache.geode.cache.RegionShortcut.REPLICATE;
 import static org.apache.geode.distributed.ConfigurationProperties.LOCATORS;
 import static org.apache.geode.distributed.ConfigurationProperties.MCAST_PORT;
 import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
-import static org.apache.geode.internal.cache.execute.FunctionExecutionOnLonerRegressionTest.UncheckedUtils.cast;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 
 import java.util.Collection;
@@ -35,15 +35,14 @@ import org.junit.experimental.categories.Category;
 
 import org.apache.geode.cache.CacheFactory;
 import org.apache.geode.cache.Region;
-import org.apache.geode.cache.execute.Execution;
 import org.apache.geode.cache.execute.FunctionContext;
-import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.RegionFunctionContext;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.cache.partition.PartitionRegionHelper;
 import org.apache.geode.distributed.internal.DistributionManager;
 import org.apache.geode.distributed.internal.LonerDistributionManager;
 import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.execute.util.TypedFunctionService;
 import org.apache.geode.test.junit.categories.FunctionServiceTest;
 
 /**
@@ -88,7 +87,7 @@ public class FunctionExecutionOnLonerRegressionTest {
 
     populateRegion(region);
 
-    ResultCollector<Collection<String>, Collection<String>> resultCollector = FunctionServiceCast
+    ResultCollector<Collection<String>, Collection<String>> resultCollector = TypedFunctionService
         .<Void, Collection<String>, Collection<String>>onRegion(region)
         .withFilter(keysForGet)
         .execute(new TestFunction(DataSetSupplier.PARTITIONED));
@@ -105,7 +104,7 @@ public class FunctionExecutionOnLonerRegressionTest {
 
     populateRegion(region);
 
-    ResultCollector<Collection<String>, Collection<String>> resultCollector = FunctionServiceCast
+    ResultCollector<Collection<String>, Collection<String>> resultCollector = TypedFunctionService
         .<Void, Collection<String>, Collection<String>>onRegion(region)
         .withFilter(keysForGet)
         .execute(new TestFunction(DataSetSupplier.REPLICATE));
@@ -167,7 +166,7 @@ public class FunctionExecutionOnLonerRegressionTest {
     @Override
     public void execute(FunctionContext<String> context) {
       RegionFunctionContext regionFunctionContext = (RegionFunctionContext) context;
-      Set<String> keys = cast(regionFunctionContext.getFilter());
+      Set<String> keys = uncheckedCast(regionFunctionContext.getFilter());
       String lastKey = keys.iterator().next();
       keys.remove(lastKey);
 
@@ -185,20 +184,4 @@ public class FunctionExecutionOnLonerRegressionTest {
       return getClass().getName();
     }
   }
-
-  @SuppressWarnings({"unchecked", "WeakerAccess"})
-  private static class FunctionServiceCast {
-
-    static <IN, OUT, AGG> Execution<IN, OUT, AGG> onRegion(Region<?, ?> region) {
-      return FunctionService.onRegion(region);
-    }
-  }
-
-  @SuppressWarnings({"unchecked", "unused"})
-  static class UncheckedUtils {
-
-    static <T> T cast(Object object) {
-      return (T) object;
-    }
-  }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientMetadataService.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientMetadataService.java
index 0aaa9df..8b89bb8 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientMetadataService.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientMetadataService.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.cache.client.internal;
 
-import static org.apache.geode.internal.cache.util.UncheckedUtils.cast;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -230,7 +230,7 @@ public class ClientMetadataService {
 
     for (Map.Entry entry : serverToBuckets.entrySet()) {
       ServerLocation server = (ServerLocation) entry.getKey();
-      Set<Integer> buckets = cast(entry.getValue());
+      Set<Integer> buckets = uncheckedCast(entry.getValue());
       for (Integer bucket : buckets) {
         // use LinkedHashSet to maintain the order of keys
         // the keys will be iterated several times
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
index 2f6b0b4..2cd1f3c 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/GemFireCacheImpl.java
@@ -33,10 +33,6 @@ import static org.apache.geode.distributed.internal.ClusterDistributionManager.L
 import static org.apache.geode.distributed.internal.DistributionConfig.DEFAULT_DURABLE_CLIENT_ID;
 import static org.apache.geode.distributed.internal.InternalDistributedSystem.getAnyInstance;
 import static org.apache.geode.internal.cache.ColocationHelper.getColocatedChildRegions;
-import static org.apache.geode.internal.cache.GemFireCacheImpl.UncheckedUtils.asDistributedMemberSet;
-import static org.apache.geode.internal.cache.GemFireCacheImpl.UncheckedUtils.createMapArray;
-import static org.apache.geode.internal.cache.GemFireCacheImpl.UncheckedUtils.uncheckedCast;
-import static org.apache.geode.internal.cache.GemFireCacheImpl.UncheckedUtils.uncheckedRegionAttributes;
 import static org.apache.geode.internal.cache.LocalRegion.setThreadInitLevelRequirement;
 import static org.apache.geode.internal.cache.PartitionedRegion.DISK_STORE_FLUSHED;
 import static org.apache.geode.internal.cache.PartitionedRegion.OFFLINE_EQUAL_PERSISTED;
@@ -44,11 +40,11 @@ import static org.apache.geode.internal.cache.PartitionedRegion.PRIMARY_BUCKETS_
 import static org.apache.geode.internal.cache.PartitionedRegionHelper.PARTITION_LOCK_SERVICE_NAME;
 import static org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType.HEAP_MEMORY;
 import static org.apache.geode.internal.cache.control.InternalResourceManager.ResourceType.OFFHEAP_MEMORY;
-import static org.apache.geode.internal.cache.util.UncheckedUtils.cast;
 import static org.apache.geode.internal.logging.CoreLoggingExecutors.newThreadPoolWithFixedFeed;
 import static org.apache.geode.internal.tcp.ConnectionTable.threadWantsSharedResources;
 import static org.apache.geode.logging.internal.executors.LoggingExecutors.newFixedThreadPool;
 import static org.apache.geode.util.internal.GeodeGlossary.GEMFIRE_PREFIX;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 
 import java.io.BufferedReader;
 import java.io.ByteArrayInputStream;
@@ -1812,7 +1808,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
             && partitionedRegion.getDataPolicy() == DataPolicy.PERSISTENT_PARTITION) {
           int numBuckets = partitionedRegion.getTotalNumberOfBuckets();
           Map<InternalDistributedMember, PersistentMemberID>[] bucketMaps =
-              createMapArray(numBuckets);
+              uncheckedCast(new Map[numBuckets]);
           PartitionedRegionDataStore dataStore = partitionedRegion.getDataStore();
 
           // lock all the primary buckets
@@ -2666,18 +2662,18 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
   @Override
   public Set<DistributedMember> getAdminMembers() {
-    return asDistributedMemberSet(dm.getAdminMemberSet());
+    return uncheckedCast(dm.getAdminMemberSet());
   }
 
   @Override
   public Set<DistributedMember> getMembers(Region region) {
     if (region instanceof DistributedRegion) {
       DistributedRegion distributedRegion = (DistributedRegion) region;
-      return asDistributedMemberSet(distributedRegion.getDistributionAdvisor().adviseCacheOp());
+      return uncheckedCast(distributedRegion.getDistributionAdvisor().adviseCacheOp());
     }
     if (region instanceof PartitionedRegion) {
       PartitionedRegion partitionedRegion = (PartitionedRegion) region;
-      return asDistributedMemberSet(partitionedRegion.getRegionAdvisor().adviseAllPRNodes());
+      return uncheckedCast(partitionedRegion.getRegionAdvisor().adviseAllPRNodes());
     }
     return emptySet();
   }
@@ -3063,15 +3059,14 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
       system.handleResourceEvent(ResourceEvent.REGION_CREATE, region);
     }
 
-    return cast(region);
+    return uncheckedCast(region);
   }
 
   @Override
   public <K, V> RegionAttributes<K, V> invokeRegionBefore(InternalRegion parent, String name,
       RegionAttributes<K, V> attrs, InternalRegionArguments internalRegionArgs) {
     for (RegionListener listener : regionListeners) {
-      attrs =
-          uncheckedRegionAttributes(listener.beforeCreate(parent, name, attrs, internalRegionArgs));
+      attrs = uncheckedCast(listener.beforeCreate(parent, name, attrs, internalRegionArgs));
     }
     return attrs;
   }
@@ -3187,7 +3182,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
   @Override
   public <K, V> Region<K, V> getRegionByPath(String path) {
-    return cast(getInternalRegionByPath(path));
+    return uncheckedCast(getInternalRegionByPath(path));
   }
 
   @Override
@@ -3244,7 +3239,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
         stopper.checkCancelInProgress(null);
         return null;
       }
-      return cast(result);
+      return uncheckedCast(result);
     }
 
     String[] pathParts = parsePath(path);
@@ -3268,7 +3263,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
       logger.debug("GemFireCache.getRegion, calling getSubregion on rootRegion({}): {}",
           pathParts[0], pathParts[1]);
     }
-    return cast(rootRegion.getSubregion(pathParts[1], returnDestroyedRegion));
+    return uncheckedCast(rootRegion.getSubregion(pathParts[1], returnDestroyedRegion));
   }
 
   @Override
@@ -4058,7 +4053,7 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
 
   @Override
   public <K, V> RegionAttributes<K, V> getRegionAttributes(String id) {
-    return GemFireCacheImpl.UncheckedUtils.<K, V>uncheckedCast(namedRegionAttributes).get(id);
+    return uncheckedCast(namedRegionAttributes.get(id));
   }
 
   @Override
@@ -5176,28 +5171,6 @@ public class GemFireCacheImpl implements InternalCache, InternalClientCache, Has
     }
   }
 
-  @SuppressWarnings("unchecked")
-  static class UncheckedUtils {
-
-    static Map<InternalDistributedMember, PersistentMemberID>[] createMapArray(int size) {
-      return new Map[size];
-    }
-
-    static Set<DistributedMember> asDistributedMemberSet(
-        Set<InternalDistributedMember> internalDistributedMembers) {
-      return (Set) internalDistributedMembers;
-    }
-
-    static <K, V> RegionAttributes<K, V> uncheckedRegionAttributes(RegionAttributes region) {
-      return region;
-    }
-
-    static <K, V> Map<String, RegionAttributes<K, V>> uncheckedCast(
-        Map<String, RegionAttributes<?, ?>> namedRegionAttributes) {
-      return (Map) namedRegionAttributes;
-    }
-  }
-
   @FunctionalInterface
   @VisibleForTesting
   interface TXManagerImplFactory {
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java
index 10635fe..dbbb98b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/InternalCacheForClientAccess.java
@@ -16,7 +16,7 @@
  */
 package org.apache.geode.internal.cache;
 
-import static org.apache.geode.internal.cache.util.UncheckedUtils.cast;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 
 import java.io.File;
 import java.io.IOException;
@@ -162,7 +162,7 @@ public class InternalCacheForClientAccess implements InternalCache {
   public <K, V> Region<K, V> getRegion(String path, boolean returnDestroyedRegion) {
     Region result = delegate.getRegion(path, returnDestroyedRegion);
     checkForInternalRegion(result);
-    return cast(result);
+    return uncheckedCast(result);
   }
 
   @Override
@@ -176,7 +176,7 @@ public class InternalCacheForClientAccess implements InternalCache {
   public <K, V> Region<K, V> getRegionByPath(String path) {
     InternalRegion result = delegate.getInternalRegionByPath(path);
     checkForInternalRegion(result);
-    return cast(result);
+    return uncheckedCast(result);
   }
 
   @Override
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
index 849187b..098115b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
@@ -17,9 +17,9 @@ package org.apache.geode.internal.cache;
 import static org.apache.geode.internal.cache.LocalRegion.InitializationLevel.AFTER_INITIAL_IMAGE;
 import static org.apache.geode.internal.cache.LocalRegion.InitializationLevel.ANY_INIT;
 import static org.apache.geode.internal.cache.LocalRegion.InitializationLevel.BEFORE_INITIAL_IMAGE;
-import static org.apache.geode.internal.cache.util.UncheckedUtils.cast;
 import static org.apache.geode.internal.lang.SystemUtils.getLineSeparator;
 import static org.apache.geode.internal.offheap.annotations.OffHeapIdentifier.ENTRY_EVENT_NEW_VALUE;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -8943,7 +8943,10 @@ public class LocalRegion extends AbstractRegion implements LoaderHelperFactory,
         txState.getRealDeal(null, this);
       }
       try {
-        proxyResult = getServerProxy().putAll(cast(map), eventId, !event.isGenerateCallbacks(),
+        proxyResult = getServerProxy().putAll(
+            uncheckedCast(map),
+            eventId,
+            !event.isGenerateCallbacks(),
             event.getCallbackArgument());
         if (isDebugEnabled) {
           logger.debug("PutAll received response from server: {}", proxyResult);
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/util/UncheckedUtils.java b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/TypedFunctionService.java
similarity index 61%
rename from geode-core/src/main/java/org/apache/geode/internal/cache/util/UncheckedUtils.java
rename to geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/TypedFunctionService.java
index c03e990..3f21439 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/util/UncheckedUtils.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/TypedFunctionService.java
@@ -12,18 +12,26 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.internal.cache.util;
+package org.apache.geode.internal.cache.execute.util;
 
+import org.apache.geode.cache.Region;
 import org.apache.geode.cache.execute.Execution;
+import org.apache.geode.cache.execute.FunctionService;
 
+/**
+ * Utilities for casting and working around raw types in the {@link FunctionService} API.
+ */
 @SuppressWarnings({"unchecked", "unused"})
-public class UncheckedUtils {
+public class TypedFunctionService {
 
-  public static <T> T cast(Object object) {
-    return (T) object;
+  protected TypedFunctionService() {
+    // do not instantiate
   }
 
-  public static <IN, OUT, AGG> Execution<IN, OUT, AGG> cast(Execution execution) {
-    return execution;
+  /**
+   * Adds parameterized type support to {@link FunctionService#onRegion(Region)}.
+   */
+  public static <IN, OUT, AGG> Execution<IN, OUT, AGG> onRegion(Region<?, ?> region) {
+    return FunctionService.onRegion(region);
   }
 }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxyFactory.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxyFactory.java
index a3def49..ae26aa7 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxyFactory.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/CacheClientProxyFactory.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.internal.cache.tier.sockets;
 
-import static org.apache.geode.internal.cache.util.UncheckedUtils.cast;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 
 import java.lang.reflect.InvocationTargetException;
 import java.net.Socket;
@@ -55,7 +55,7 @@ public class CacheClientProxyFactory {
     }
     try {
       Class<InternalCacheClientProxyFactory> proxyClass =
-          cast(ClassPathLoader.getLatest().forName(proxyClassName));
+          uncheckedCast(ClassPathLoader.getLatest().forName(proxyClassName));
       return proxyClass.getConstructor().newInstance();
     } catch (ClassNotFoundException | NoSuchMethodException | InstantiationException
         | IllegalAccessException | InvocationTargetException e) {
diff --git a/geode-core/src/test/java/org/apache/geode/alerting/internal/ClusterAlertMessagingTest.java b/geode-core/src/test/java/org/apache/geode/alerting/internal/ClusterAlertMessagingTest.java
index f7bc8d8..eadf54b 100644
--- a/geode-core/src/test/java/org/apache/geode/alerting/internal/ClusterAlertMessagingTest.java
+++ b/geode-core/src/test/java/org/apache/geode/alerting/internal/ClusterAlertMessagingTest.java
@@ -14,7 +14,7 @@
  */
 package org.apache.geode.alerting.internal;
 
-import static org.apache.geode.internal.cache.util.UncheckedUtils.cast;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.catchThrowable;
 import static org.mockito.ArgumentMatchers.anyLong;
@@ -141,7 +141,7 @@ public class ClusterAlertMessagingTest {
   public void sendAlertLogsWarning_ifAlertingIOExceptionIsCaught() {
     ExecutorService executor = currentThreadExecutorService();
     ClusterDistributionManager distributionManager = mock(ClusterDistributionManager.class);
-    Consumer<AlertingIOException> alertingIOExceptionLogger = cast(mock(Consumer.class));
+    Consumer<AlertingIOException> alertingIOExceptionLogger = uncheckedCast(mock(Consumer.class));
     ClusterAlertMessaging clusterAlertMessaging =
         spyClusterAlertMessaging(distributionManager, executor, alertingIOExceptionLogger);
     doThrow(new AlertingIOException(new IOException("Cannot form connection to alert listener")))
@@ -162,7 +162,7 @@ public class ClusterAlertMessagingTest {
   public void sendAlertLogsWarningOnce_ifAlertingIOExceptionIsCaught() {
     ExecutorService executor = currentThreadExecutorService();
     ClusterDistributionManager distributionManager = mock(ClusterDistributionManager.class);
-    Consumer<AlertingIOException> alertingIOExceptionLogger = cast(mock(Consumer.class));
+    Consumer<AlertingIOException> alertingIOExceptionLogger = uncheckedCast(mock(Consumer.class));
     ClusterAlertMessaging clusterAlertMessaging =
         spyClusterAlertMessaging(distributionManager, executor, alertingIOExceptionLogger);
     doThrow(new AlertingIOException(new IOException("Cannot form connection to alert listener")))
diff --git a/geode-core/src/test/java/org/apache/geode/internal/tcp/TCPConduitTest.java b/geode-core/src/test/java/org/apache/geode/internal/tcp/TCPConduitTest.java
index 0c30ce2..c0bf0a5 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/tcp/TCPConduitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/tcp/TCPConduitTest.java
@@ -16,7 +16,7 @@
  */
 package org.apache.geode.internal.tcp;
 
-import static org.apache.geode.internal.cache.util.UncheckedUtils.cast;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.catchThrowable;
 import static org.mockito.Mockito.anyBoolean;
@@ -62,7 +62,7 @@ public class TCPConduitTest {
 
   @Before
   public void setUp() throws Exception {
-    membership = cast(mock(Membership.class));
+    membership = uncheckedCast(mock(Membership.class));
     directChannel = mock(DirectChannel.class);
     connectionTable = mock(ConnectionTable.class);
     socketCreator = new SocketCreator(new SSLConfig.Builder().build());