You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2017/05/11 21:07:26 UTC

[47/52] [abbrv] geode git commit: GEODE-2632: change dependencies on GemFireCacheImpl to InternalCache

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionSingleHopOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionSingleHopOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionSingleHopOp.java
index 1697b07..cf14e28 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionSingleHopOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionSingleHopOp.java
@@ -33,7 +33,7 @@ import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.internal.Version;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.execute.AbstractExecution;
 import org.apache.geode.internal.cache.execute.BucketMovedException;
@@ -68,7 +68,7 @@ public class ExecuteRegionFunctionSingleHopOp {
     if (function.isHA()) {
       maxRetryAttempts = mRetryAttempts;
     }
-    ClientMetadataService cms = ((GemFireCacheImpl) region.getCache()).getClientMetadataService();
+    ClientMetadataService cms = ((InternalCache) region.getCache()).getClientMetadataService();
 
     final boolean isDebugEnabled = logger.isDebugEnabled();
     if (isDebugEnabled) {
@@ -114,7 +114,7 @@ public class ExecuteRegionFunctionSingleHopOp {
     if (isHA) {
       maxRetryAttempts = mRetryAttempts;
     }
-    ClientMetadataService cms = ((GemFireCacheImpl) region.getCache()).getClientMetadataService();
+    ClientMetadataService cms = ((InternalCache) region.getCache()).getClientMetadataService();
 
     final boolean isDebugEnabled = logger.isDebugEnabled();
     if (isDebugEnabled) {
@@ -243,7 +243,7 @@ public class ExecuteRegionFunctionSingleHopOp {
       }
       getMessage().addObjPart(args);
       getMessage().addObjPart(memberMappedArg);
-      getMessage().addBytesPart(new byte[] {allBuckets ? (byte) 1 : (byte) 0});
+      getMessage().addBytesPart(new byte[] {(byte) (allBuckets ? 1 : 0)});
       getMessage().addIntPart(routingObjects.size());
       for (Object key : routingObjects) {
         if (allBuckets) {
@@ -284,7 +284,7 @@ public class ExecuteRegionFunctionSingleHopOp {
       getMessage().addStringOrObjPart(functionId);
       getMessage().addObjPart(args);
       getMessage().addObjPart(memberMappedArg);
-      getMessage().addBytesPart(new byte[] {allBuckets ? (byte) 1 : (byte) 0});
+      getMessage().addBytesPart(new byte[] {(byte) (allBuckets ? 1 : 0)});
       getMessage().addIntPart(routingObjects.size());
       for (Object key : routingObjects) {
         if (allBuckets) {
@@ -307,13 +307,13 @@ public class ExecuteRegionFunctionSingleHopOp {
     }
 
     private void addBytes(byte functionState) {
-      if (GemFireCacheImpl
-          .getClientFunctionTimeout() == GemFireCacheImpl.DEFAULT_CLIENT_FUNCTION_TIMEOUT) {
+      if (ConnectionImpl
+          .getClientFunctionTimeout() == ConnectionImpl.DEFAULT_CLIENT_FUNCTION_TIMEOUT) {
         getMessage().addBytesPart(new byte[] {functionState});
       } else {
         byte[] bytes = new byte[5];
         bytes[0] = functionState;
-        Part.encodeInt(GemFireCacheImpl.getClientFunctionTimeout(), bytes, 1);
+        Part.encodeInt(ConnectionImpl.getClientFunctionTimeout(), bytes, 1);
         getMessage().addBytesPart(bytes);
       }
     }
@@ -336,7 +336,7 @@ public class ExecuteRegionFunctionSingleHopOp {
               Object resultResponse = executeFunctionResponseMsg.getPart(0).getObject();
               Object result;
               if (resultResponse instanceof ArrayList) {
-                result = ((ArrayList) resultResponse).get(0);
+                result = ((List) resultResponse).get(0);
               } else {
                 result = resultResponse;
               }
@@ -344,13 +344,12 @@ public class ExecuteRegionFunctionSingleHopOp {
                 FunctionException ex = ((FunctionException) result);
                 if (isDebugEnabled) {
                   logger.debug(
-                      "ExecuteRegionFunctionSingleHopOpImpl#processResponse: received Exception. {}",
+                      "ExecuteRegionFunctionSingleHopOpImpl#processResponse: received Exception.",
                       ex.getCause());
                 }
                 if (ex instanceof InternalFunctionException) {
                   Throwable cause = ex.getCause();
-                  DistributedMember memberID =
-                      (DistributedMember) ((ArrayList) resultResponse).get(1);
+                  DistributedMember memberID = (DistributedMember) ((List) resultResponse).get(1);
                   this.resultCollector.addResult(memberID, cause);
                   FunctionStats
                       .getFunctionStats(this.functionId, this.executor.getRegion().getSystem())
@@ -374,8 +373,7 @@ public class ExecuteRegionFunctionSingleHopOp {
                     new InternalFunctionInvocationTargetException(
                         ((CacheClosedException) result).getMessage());
                 if (resultResponse instanceof ArrayList) {
-                  DistributedMember memberID =
-                      (DistributedMember) ((ArrayList) resultResponse).get(1);
+                  DistributedMember memberID = (DistributedMember) ((List) resultResponse).get(1);
                   this.failedNodes.add(memberID.getId());
                 }
                 exception = new FunctionException(fite);
@@ -383,8 +381,7 @@ public class ExecuteRegionFunctionSingleHopOp {
                 String s = "While performing a remote " + getOpName();
                 exception = new ServerOperationException(s, (Throwable) result);
               } else {
-                DistributedMember memberID =
-                    (DistributedMember) ((ArrayList) resultResponse).get(1);
+                DistributedMember memberID = (DistributedMember) ((List) resultResponse).get(1);
                 this.resultCollector.addResult(memberID, result);
                 FunctionStats
                     .getFunctionStats(this.functionId, this.executor.getRegion().getSystem())

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/client/internal/GetOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/GetOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/GetOp.java
index 3c4fe10..271c92e 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/GetOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/GetOp.java
@@ -25,7 +25,6 @@ import org.apache.geode.cache.client.ServerOperationException;
 import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.cache.EntryEventImpl;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.Token;
 import org.apache.geode.internal.cache.tier.MessageType;
@@ -60,7 +59,7 @@ public class GetOp {
    */
   public static Object execute(ExecutablePool pool, LocalRegion region, Object key,
       Object callbackArg, boolean prSingleHopEnabled, EntryEventImpl clientEvent) {
-    ClientMetadataService cms = ((GemFireCacheImpl) region.getCache()).getClientMetadataService();
+    ClientMetadataService cms = region.getCache().getClientMetadataService();
     GetOpImpl op = new GetOpImpl(region, key, callbackArg, prSingleHopEnabled, clientEvent);
 
     if (logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/client/internal/InternalClientCache.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/InternalClientCache.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/InternalClientCache.java
new file mode 100644
index 0000000..537d632
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/InternalClientCache.java
@@ -0,0 +1,37 @@
+/*
+ * 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.client.internal;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionAttributes;
+import org.apache.geode.cache.RegionExistsException;
+import org.apache.geode.cache.TimeoutException;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.client.Pool;
+import org.apache.geode.cache.client.PoolFactory;
+
+public interface InternalClientCache extends ClientCache {
+
+  boolean isClient();
+
+  void determineDefaultPool();
+
+  Pool determineDefaultPool(PoolFactory poolFactory);
+
+  <K, V> Region<K, V> basicCreateRegion(String name, RegionAttributes<K, V> attrs)
+      throws RegionExistsException, TimeoutException;
+
+  Pool getDefaultPool();
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/client/internal/LiveServerPinger.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/LiveServerPinger.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/LiveServerPinger.java
index 066e737..c5ec03c 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/LiveServerPinger.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/LiveServerPinger.java
@@ -25,6 +25,7 @@ import org.apache.logging.log4j.Logger;
 import org.apache.geode.cache.client.internal.EndpointManager.EndpointListenerAdapter;
 import org.apache.geode.cache.client.internal.PoolImpl.PoolTask;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogService;
 
 /**
@@ -93,7 +94,7 @@ public class LiveServerPinger extends EndpointListenerAdapter {
             logger.debug("Error occurred while pinging server: {} - {}", endpoint.getLocation(),
                 e.getMessage());
           }
-          GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+          InternalCache cache = GemFireCacheImpl.getInstance();
           if (cache != null) {
             ClientMetadataService cms = cache.getClientMetadataService();
             cms.removeBucketServerLocation(endpoint.getLocation());

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/client/internal/PdxRegistryRecoveryListener.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/PdxRegistryRecoveryListener.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/PdxRegistryRecoveryListener.java
index 93063e0..8c52551 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/PdxRegistryRecoveryListener.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/PdxRegistryRecoveryListener.java
@@ -19,6 +19,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.pdx.internal.TypeRegistry;
 
@@ -72,7 +73,7 @@ public class PdxRegistryRecoveryListener extends EndpointManager.EndpointListene
       logger.debug("PdxRegistryRecoveryListener - EndpointNowInUse. Now have {} endpoints", count);
     }
     if (count == 1) {
-      GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+      InternalCache cache = GemFireCacheImpl.getInstance();
       if (cache == null) {
         return;
       }

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java
index 88369e1..133d7a8 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/PoolImpl.java
@@ -14,11 +14,35 @@
  */
 package org.apache.geode.cache.client.internal;
 
+import static org.apache.commons.lang.StringUtils.isEmpty;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.logging.log4j.Logger;
+
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.CancelException;
 import org.apache.geode.StatisticsFactory;
 import org.apache.geode.SystemFailure;
-import org.apache.geode.cache.*;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.CacheClosedException;
+import org.apache.geode.cache.CacheFactory;
+import org.apache.geode.cache.NoSubscriptionServersAvailableException;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionService;
 import org.apache.geode.cache.client.Pool;
 import org.apache.geode.cache.client.ServerConnectivityException;
 import org.apache.geode.cache.client.SubscriptionNotEnabledException;
@@ -31,26 +55,21 @@ import org.apache.geode.distributed.PoolCancelledException;
 import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ServerLocation;
-import org.apache.geode.internal.statistics.DummyStatisticsFactory;
 import org.apache.geode.internal.ScheduledThreadPoolExecutorWithKeepAlive;
 import org.apache.geode.internal.admin.ClientStatsManager;
-import org.apache.geode.internal.cache.*;
+import org.apache.geode.internal.cache.EventID;
+import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.PoolFactoryImpl;
+import org.apache.geode.internal.cache.PoolManagerImpl;
+import org.apache.geode.internal.cache.PoolStats;
 import org.apache.geode.internal.cache.tier.sockets.AcceptorImpl;
 import org.apache.geode.internal.cache.tier.sockets.ClientProxyMembershipID;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
-import org.apache.logging.log4j.Logger;
-
-import java.net.InetSocketAddress;
-import java.util.*;
-import java.util.Map.Entry;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.geode.internal.statistics.DummyStatisticsFactory;
 
 /**
  * Manages the client side of client to server connections and client queues.
@@ -58,26 +77,29 @@ import java.util.concurrent.atomic.AtomicInteger;
  * @since GemFire 5.7
  */
 public class PoolImpl implements InternalPool {
+
   public static final String ON_DISCONNECT_CLEAR_PDXTYPEIDS =
       DistributionConfig.GEMFIRE_PREFIX + "ON_DISCONNECT_CLEAR_PDXTYPEIDS";
 
   private static final Logger logger = LogService.getLogger();
 
-  public static final int HANDSHAKE_TIMEOUT =
-      Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "PoolImpl.HANDSHAKE_TIMEOUT",
-          AcceptorImpl.DEFAULT_HANDSHAKE_TIMEOUT_MS).intValue();
-  public static final long SHUTDOWN_TIMEOUT = Long
-      .getLong(DistributionConfig.GEMFIRE_PREFIX + "PoolImpl.SHUTDOWN_TIMEOUT", 30000).longValue();
-  public static final int BACKGROUND_TASK_POOL_SIZE = Integer
-      .getInteger(DistributionConfig.GEMFIRE_PREFIX + "PoolImpl.BACKGROUND_TASK_POOL_SIZE", 20)
-      .intValue();
-  public static final int BACKGROUND_TASK_POOL_KEEP_ALIVE =
-      Integer
-          .getInteger(
-              DistributionConfig.GEMFIRE_PREFIX + "PoolImpl.BACKGROUND_TASK_POOL_KEEP_ALIVE", 1000)
-          .intValue();
-  // For durable client tests only. Connection Sources read this flag
-  // and return an empty list of servers.
+  private static final int HANDSHAKE_TIMEOUT =
+      Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "PoolImpl.HANDSHAKE_TIMEOUT",
+          AcceptorImpl.DEFAULT_HANDSHAKE_TIMEOUT_MS);
+
+  public static final long SHUTDOWN_TIMEOUT =
+      Long.getLong(DistributionConfig.GEMFIRE_PREFIX + "PoolImpl.SHUTDOWN_TIMEOUT", 30000);
+
+  private static final int BACKGROUND_TASK_POOL_SIZE = Integer
+      .getInteger(DistributionConfig.GEMFIRE_PREFIX + "PoolImpl.BACKGROUND_TASK_POOL_SIZE", 20);
+
+  private static final int BACKGROUND_TASK_POOL_KEEP_ALIVE = Integer.getInteger(
+      DistributionConfig.GEMFIRE_PREFIX + "PoolImpl.BACKGROUND_TASK_POOL_KEEP_ALIVE", 1000);
+
+  /**
+   * For durable client tests only. Connection Sources read this flag and return an empty list of
+   * servers.
+   */
   public volatile static boolean TEST_DURABLE_IS_NET_DOWN = false;
 
   private final String name;
@@ -152,7 +174,7 @@ public class PoolImpl implements InternalPool {
     } catch (RuntimeException e) {
       try {
         destroy(false);
-      } catch (RuntimeException e2) {
+      } catch (RuntimeException ignore) {
         // do nothing
       }
       throw e;
@@ -218,9 +240,8 @@ public class PoolImpl implements InternalPool {
       statFactory = ds;
     }
     this.stats = this.startDisabled ? null
-        : new PoolStats(statFactory,
-            getName() + "->" + (serverGroup == null || serverGroup.equals("") ? "[any servers]"
-                : "[" + getServerGroup() + "]"));
+        : new PoolStats(statFactory, getName() + "->"
+            + (isEmpty(serverGroup) ? "[any servers]" : "[" + getServerGroup() + "]"));
 
     source = getSourceImpl(((PoolFactoryImpl.PoolAttributes) attributes).locatorCallback);
     endpointManager = new EndpointManagerImpl(name, ds, this.cancelCriterion, this.stats);
@@ -521,7 +542,7 @@ public class PoolImpl implements InternalPool {
       if (cnt > 0) {
         throw new IllegalStateException(
             LocalizedStrings.PoolImpl_POOL_COULD_NOT_BE_DESTROYED_BECAUSE_IT_IS_STILL_IN_USE_BY_0_REGIONS
-                .toLocalizedString(Integer.valueOf(cnt)));
+                .toLocalizedString(cnt));
       }
     }
     if (this.pm.unregister(this)) {
@@ -867,8 +888,6 @@ public class PoolImpl implements InternalPool {
 
   /**
    * Hook to return connections that were acquired using acquireConnection.
-   * 
-   * @param conn
    */
   public void returnConnection(Connection conn) {
     manager.returnConnection(conn);
@@ -900,9 +919,9 @@ public class PoolImpl implements InternalPool {
    */
   public Map getThreadIdToSequenceIdMap() {
     if (this.queueManager == null)
-      return Collections.EMPTY_MAP;
+      return Collections.emptyMap();
     if (this.queueManager.getState() == null)
-      return Collections.EMPTY_MAP;
+      return Collections.emptyMap();
     return this.queueManager.getState().getThreadIdToSequenceIdMap();
   }
 
@@ -928,7 +947,7 @@ public class PoolImpl implements InternalPool {
         Exception e = new Exception(msg);
         try {
           processException(e, con);
-        } catch (ServerConnectivityException expected) {
+        } catch (ServerConnectivityException ignore) {
         } finally {
           logger.info("<ExpectedException action=remove>{}</ExpectedException>", msg);
         }
@@ -1033,7 +1052,7 @@ public class PoolImpl implements InternalPool {
    * redundant server. An empty list is returned if we have no redundant servers.
    */
   public List<String> getRedundantNames() {
-    List result = Collections.EMPTY_LIST;
+    List result = Collections.emptyList();
     if (this.queueManager != null) {
       QueueManager.QueueConnections cons = this.queueManager.getAllConnections();
       List<Connection> backupCons = cons.getBackups();
@@ -1055,7 +1074,7 @@ public class PoolImpl implements InternalPool {
    * redundant server. An empty list is returned if we have no redundant servers.
    */
   public List<ServerLocation> getRedundants() {
-    List result = Collections.EMPTY_LIST;
+    List result = Collections.emptyList();
     if (this.queueManager != null) {
       QueueManager.QueueConnections cons = this.queueManager.getAllConnections();
       List<Connection> backupCons = cons.getBackups();
@@ -1176,8 +1195,8 @@ public class PoolImpl implements InternalPool {
     logger.debug("PoolImpl - endpointsNetDownForDUnitTest");
     setTEST_DURABLE_IS_NET_DOWN(true);
     try {
-      java.lang.Thread.sleep(this.pingInterval * 2);
-    } catch (java.lang.InterruptedException ex) {
+      Thread.sleep(this.pingInterval * 2);
+    } catch (java.lang.InterruptedException ignore) {
       // do nothing.
     }
 
@@ -1195,8 +1214,8 @@ public class PoolImpl implements InternalPool {
   public void endpointsNetUpForDUnitTest() {
     setTEST_DURABLE_IS_NET_DOWN(false);
     try {
-      java.lang.Thread.sleep(this.pingInterval * 2);
-    } catch (java.lang.InterruptedException ex) {
+      Thread.sleep(this.pingInterval * 2);
+    } catch (java.lang.InterruptedException ignore) {
       // do nothing.
     }
   }
@@ -1292,10 +1311,9 @@ public class PoolImpl implements InternalPool {
       } catch (VirtualMachineError e) {
         SystemFailure.initiateFailure(e);
         throw e;
-      } catch (CancelException e) {
-        // throw e;
+      } catch (CancelException ignore) {
         if (logger.isDebugEnabled()) {
-          logger.debug("Pool task <{}> cancelled", this, logger.isTraceEnabled() ? e : null);
+          logger.debug("Pool task <{}> cancelled", this);
         }
       } catch (Throwable t) {
         logger.error(LocalizedMessage
@@ -1391,7 +1409,7 @@ public class PoolImpl implements InternalPool {
     for (Entry<Object, Object> entry : properties.entrySet()) {
       props.setProperty((String) entry.getKey(), (String) entry.getValue());
     }
-    ProxyCache proxy = new ProxyCache(props, (GemFireCacheImpl) cache, this);
+    ProxyCache proxy = new ProxyCache(props, (InternalCache) cache, this);
     synchronized (this.proxyCacheList) {
       this.proxyCacheList.add(proxy);
     }
@@ -1405,20 +1423,13 @@ public class PoolImpl implements InternalPool {
     if (re != null) {
       return re;
     }
-    Cache cache = GemFireCacheImpl.getInstance();
+    InternalCache cache = GemFireCacheImpl.getInstance();
     if (cache == null) {
       if (cacheCriterion != null) {
         return cacheCriterion.generateCancelledException(e);
       }
     } else {
-      if (cacheCriterion == null) {
-        cacheCriterion = cache.getCancelCriterion();
-      } else if (cacheCriterion != cache.getCancelCriterion()) {
-        /*
-         * If the cache instance has somehow changed, we need to get a reference to the new
-         * criterion. This is pretty unlikely because the cache closes all the pools when it shuts
-         * down, but I wanted to be safe.
-         */
+      if (cacheCriterion == null || cacheCriterion != cache.getCancelCriterion()) {
         cacheCriterion = cache.getCancelCriterion();
       }
       return cacheCriterion.generateCancelledException(e);
@@ -1433,7 +1444,7 @@ public class PoolImpl implements InternalPool {
       if (reason != null) {
         return reason;
       }
-      Cache cache = GemFireCacheImpl.getInstance();
+      InternalCache cache = GemFireCacheImpl.getInstance();
       if (cache == null) {
         if (cacheCriterion != null) {
           return cacheCriterion.cancelInProgress();
@@ -1462,7 +1473,7 @@ public class PoolImpl implements InternalPool {
   }
 
   public boolean getKeepAlive() {
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+    InternalCache cache = GemFireCacheImpl.getInstance();
     if (cache == null) {
       return keepAlive;
     }

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
index f4a8d5b..8953e8c 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ProxyCache.java
@@ -15,6 +15,13 @@
 package org.apache.geode.cache.client.internal;
 
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
 import org.apache.geode.CancelCriterion;
 import org.apache.geode.cache.CacheClosedException;
 import org.apache.geode.cache.Region;
@@ -23,18 +30,11 @@ import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.query.QueryService;
 import org.apache.geode.cache.query.internal.ProxyQueryService;
 import org.apache.geode.distributed.internal.ServerLocation;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.pdx.PdxInstance;
 import org.apache.geode.pdx.PdxInstanceFactory;
 import org.apache.geode.pdx.internal.PdxInstanceFactoryImpl;
 
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Properties;
-import java.util.Set;
-
 /**
  * A wrapper class over an actual Cache instance. This is used when the multiuser-authentication
  * attribute is set to true. Application must use its {@link #getRegion(String)} API instead that of
@@ -50,17 +50,24 @@ import java.util.Set;
  */
 public class ProxyCache implements RegionService {
 
-  private final GemFireCacheImpl cache;
+  /**
+   * package-private to avoid synthetic accessor
+   * <p>
+   * TODO: if this is only in inside client then this should be InternalClientCache
+   */
+  final InternalCache cache;
+
   private UserAttributes userAttributes;
   private ProxyQueryService proxyQueryService;
   private boolean isClosed = false;
   private final Stopper stopper = new Stopper();
 
-  public ProxyCache(Properties properties, GemFireCacheImpl cache, PoolImpl pool) {
+  public ProxyCache(Properties properties, InternalCache cache, PoolImpl pool) {
     this.userAttributes = new UserAttributes(properties, pool);
     this.cache = cache;
   }
 
+  @Override
   public void close() {
     close(false);
   }
@@ -76,18 +83,16 @@ public class ProxyCache implements RegionService {
         this.proxyQueryService.closeCqs(keepAlive);
       }
       UserAttributes.userAttributes.set(this.userAttributes);
-      Iterator<ServerLocation> iter = this.userAttributes.getServerToId().keySet().iterator();
-      while (iter.hasNext()) {
-        ProxyCacheCloseOp.executeOn(iter.next(), (PoolImpl) this.userAttributes.getPool(),
+      for (final ServerLocation serverLocation : this.userAttributes.getServerToId().keySet()) {
+        ProxyCacheCloseOp.executeOn(serverLocation, (ExecutablePool) this.userAttributes.getPool(),
             this.userAttributes.getCredentials(), keepAlive);
       }
-      ArrayList<ProxyCache> proxyCache =
-          ((PoolImpl) this.userAttributes.getPool()).getProxyCacheList();
+      List<ProxyCache> proxyCache = ((PoolImpl) this.userAttributes.getPool()).getProxyCacheList();
       synchronized (proxyCache) {
         proxyCache.remove(this);
       }
     } finally {
-      // @todo I think some NPE will be caused by this code.
+      // TODO: I think some NPE will be caused by this code.
       // It would be safer to not null things out.
       // It is really bad that we null out and then set isClosed true.
       this.isClosed = true;
@@ -98,28 +103,19 @@ public class ProxyCache implements RegionService {
     }
   }
 
-  // TODO remove this method
-  public String getName() {
-    return this.cache.getName();
-  }
-
+  @Override
   public QueryService getQueryService() {
     preOp();
     if (this.proxyQueryService == null) {
       this.proxyQueryService =
-          new ProxyQueryService(this, userAttributes.getPool().getQueryService());
+          new ProxyQueryService(this, this.userAttributes.getPool().getQueryService());
     }
     return this.proxyQueryService;
   }
 
+  @Override
   public <K, V> Region<K, V> getRegion(String path) {
     preOp();
-    // TODO Auto-generated method stub
-    // ProxyRegion region = this.proxyRegionList.get(path);
-    // if (region != null) {
-    // return region;
-    // }
-    // else {
     if (this.cache.getRegion(path) == null) {
       return null;
     } else {
@@ -129,9 +125,9 @@ public class ProxyCache implements RegionService {
       }
       return new ProxyRegion(this, this.cache.getRegion(path));
     }
-    // }
   }
 
+  @Override
   public boolean isClosed() {
     return this.isClosed;
   }
@@ -170,11 +166,6 @@ public class ProxyCache implements RegionService {
   }
 
   protected class Stopper extends CancelCriterion {
-    /*
-     * (non-Javadoc)
-     * 
-     * @see org.apache.geode.CancelCriterion#cancelInProgress()
-     */
     @Override
     public String cancelInProgress() {
       String reason = cache.getCancelCriterion().cancelInProgress();
@@ -187,11 +178,6 @@ public class ProxyCache implements RegionService {
       return null;
     }
 
-    /*
-     * (non-Javadoc)
-     * 
-     * @see org.apache.geode.CancelCriterion#generateCancelledException(java.lang.Throwable)
-     */
     @Override
     public RuntimeException generateCancelledException(Throwable e) {
       String reason = cancelInProgress();
@@ -209,7 +195,7 @@ public class ProxyCache implements RegionService {
 
       try {
         return new CacheClosedException(reason, e);
-      } catch (IllegalStateException e2) {
+      } catch (IllegalStateException ignore) {
         // Bug 39496 (Jrockit related) Give up. The following
         // error is not entirely sane but gives the correct general picture.
         return new CacheClosedException(reason);
@@ -217,6 +203,7 @@ public class ProxyCache implements RegionService {
     }
   }
 
+  @Override
   public CancelCriterion getCancelCriterion() {
     return this.stopper;
   }
@@ -233,14 +220,16 @@ public class ProxyCache implements RegionService {
     return Collections.unmodifiableSet(rootRegions);
   }
 
+  @Override
   public PdxInstanceFactory createPdxInstanceFactory(String className) {
     return PdxInstanceFactoryImpl.newCreator(className, true);
   }
 
-  public PdxInstanceFactory createPdxInstanceFactory(String className, boolean b) {
-    return PdxInstanceFactoryImpl.newCreator(className, b);
+  public PdxInstanceFactory createPdxInstanceFactory(String className, boolean expectDomainClass) {
+    return PdxInstanceFactoryImpl.newCreator(className, expectDomainClass);
   }
 
+  @Override
   public PdxInstance createPdxEnum(String className, String enumName, int enumOrdinal) {
     return PdxInstanceFactoryImpl.createPdxEnum(className, enumName, enumOrdinal, this.cache);
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/client/internal/QueueManagerImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/QueueManagerImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/QueueManagerImpl.java
index 965ee57..6c367e9 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/QueueManagerImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/QueueManagerImpl.java
@@ -34,10 +34,10 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.geode.GemFireConfigException;
 import org.apache.logging.log4j.Logger;
 
 import org.apache.geode.CancelException;
+import org.apache.geode.GemFireConfigException;
 import org.apache.geode.GemFireException;
 import org.apache.geode.SystemFailure;
 import org.apache.geode.cache.InterestResultPolicy;
@@ -52,13 +52,14 @@ import org.apache.geode.cache.query.internal.CqStateImpl;
 import org.apache.geode.cache.query.internal.DefaultQueryService;
 import org.apache.geode.cache.query.internal.cq.ClientCQ;
 import org.apache.geode.cache.query.internal.cq.CqService;
-import org.apache.geode.cache.query.internal.cq.InternalCqQuery;
 import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.ServerLocation;
+import org.apache.geode.i18n.StringId;
 import org.apache.geode.internal.Assert;
 import org.apache.geode.internal.cache.ClientServerObserver;
 import org.apache.geode.internal.cache.ClientServerObserverHolder;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.tier.InterestType;
 import org.apache.geode.internal.cache.tier.sockets.CacheClientUpdater;
@@ -69,14 +70,12 @@ import org.apache.geode.internal.logging.InternalLogWriter;
 import org.apache.geode.internal.logging.LogService;
 import org.apache.geode.internal.logging.log4j.LocalizedMessage;
 import org.apache.geode.security.GemFireSecurityException;
-import org.apache.geode.i18n.StringId;
 
 /**
  * Manages Client Queues. Responsible for creating callback connections and satisfying redundancy
  * requirements.
  * 
  * @since GemFire 5.7
- * 
  */
 public class QueueManagerImpl implements QueueManager {
   private static final Logger logger = LogService.getLogger();
@@ -154,7 +153,7 @@ public class QueueManagerImpl implements QueueManager {
     if (primary != null) {
       ClientUpdater cu = primary.getUpdater();
       if (cu != null) {
-        result = ((CacheClientUpdater) cu).isAlive();
+        result = cu.isAlive();
       }
     }
     return result;
@@ -175,7 +174,7 @@ public class QueueManagerImpl implements QueueManager {
             && pool.getPoolOrCacheCancelInProgress() == null) {
           try {
             lock.wait();
-          } catch (InterruptedException e) {
+          } catch (InterruptedException ignore) {
             Thread.currentThread().interrupt();
             break;
           }
@@ -221,7 +220,7 @@ public class QueueManagerImpl implements QueueManager {
           logger.warn(LocalizedMessage.create(
               LocalizedStrings.QueueManagerImpl_TIMEOUT_WAITING_FOR_RECOVERY_THREAD_TO_COMPLETE));
         }
-      } catch (InterruptedException e1) {
+      } catch (InterruptedException ignore) {
         Thread.currentThread().interrupt();
         logger.debug("Interrupted waiting for recovery thread termination");
       }
@@ -332,7 +331,7 @@ public class QueueManagerImpl implements QueueManager {
     while (primary == null) {
       try {
         primary = (QueueConnectionImpl) getAllConnections().getPrimary();
-      } catch (NoSubscriptionServersAvailableException e) {
+      } catch (NoSubscriptionServersAvailableException ignore) {
         primary = null;
         break;
       }
@@ -606,7 +605,7 @@ public class QueueManagerImpl implements QueueManager {
   }
 
   private void cqsConnected() {
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+    InternalCache cache = GemFireCacheImpl.getInstance();
     if (cache != null) {
       CqService cqService = cache.getCqService();
       // Primary queue was found, alert the affected cqs if necessary
@@ -616,7 +615,7 @@ public class QueueManagerImpl implements QueueManager {
 
   private void cqsDisconnected() {
     // No primary queue was found, alert the affected cqs if necessary
-    GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+    InternalCache cache = GemFireCacheImpl.getInstance();
     if (cache != null) {
       CqService cqService = cache.getCqService();
       cqService.cqsDisconnected(pool);
@@ -659,8 +658,7 @@ public class QueueManagerImpl implements QueueManager {
           if (printRedundancyNotSatisfiedError) {
             logger.info(LocalizedMessage.create(
                 LocalizedStrings.QueueManagerImpl_REDUNDANCY_LEVEL_0_IS_NOT_SATISFIED_BUT_THERE_ARE_NO_MORE_SERVERS_AVAILABLE_REDUNDANCY_IS_CURRENTLY_1,
-                new Object[] {Integer.valueOf(redundancyLevel),
-                    Integer.valueOf(getCurrentRedundancy())}));
+                new Object[] {redundancyLevel, getCurrentRedundancy()}));
           }
         }
         printRedundancyNotSatisfiedError = false;// printed above

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/client/internal/RegisterInterestOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/RegisterInterestOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/RegisterInterestOp.java
index dd1118d..ce9d8f0 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/RegisterInterestOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/RegisterInterestOp.java
@@ -14,22 +14,22 @@
  */
 package org.apache.geode.cache.client.internal;
 
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.geode.InternalGemFireError;
+import org.apache.geode.cache.InterestResultPolicy;
+import org.apache.geode.cache.client.ServerOperationException;
+import org.apache.geode.distributed.internal.ServerLocation;
 import org.apache.geode.internal.Version;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
+import org.apache.geode.internal.cache.tier.InterestType;
 import org.apache.geode.internal.cache.tier.MessageType;
-import org.apache.geode.internal.cache.tier.sockets.Message;
 import org.apache.geode.internal.cache.tier.sockets.ChunkedMessage;
+import org.apache.geode.internal.cache.tier.sockets.Message;
 import org.apache.geode.internal.cache.tier.sockets.Part;
 import org.apache.geode.internal.cache.tier.sockets.VersionedObjectList;
-import org.apache.geode.InternalGemFireError;
-import org.apache.geode.internal.cache.tier.InterestType;
-import org.apache.geode.cache.InterestResultPolicy;
-import org.apache.geode.cache.client.ServerOperationException;
-import org.apache.geode.distributed.internal.ServerLocation;
-
-import java.util.ArrayList;
-import java.util.List;
 
 /**
  * Does a region registerInterest on a server
@@ -159,17 +159,15 @@ public class RegisterInterestOp {
 
     @Override
     protected Object processResponse(Message m, Connection con) throws Exception {
-      ChunkedMessage msg = (ChunkedMessage) m;
-      msg.readHeader();
-      switch (msg.getMessageType()) {
+      ChunkedMessage chunkedMessage = (ChunkedMessage) m;
+      chunkedMessage.readHeader();
+      switch (chunkedMessage.getMessageType()) {
         case MessageType.RESPONSE_FROM_PRIMARY: {
-          ArrayList serverKeys = new ArrayList();
-          VersionedObjectList serverEntries = null;
-          LocalRegion r = null;
+          LocalRegion localRegion = null;
 
           try {
-            r = (LocalRegion) GemFireCacheImpl.getInstance().getRegion(this.region);
-          } catch (Exception ex) {
+            localRegion = (LocalRegion) GemFireCacheImpl.getInstance().getRegion(this.region);
+          } catch (Exception ignore) {
             // ignore but read message
             // GemFireCacheImpl.getInstance().getLogger().config("hitesh error " + ex.getClass());
           }
@@ -179,12 +177,14 @@ public class RegisterInterestOp {
           listOfList.add(list);
 
           // Process the chunks
+          List serverKeys = new ArrayList();
+          VersionedObjectList serverEntries = null;
           do {
             // Read the chunk
-            msg.receiveChunk();
+            chunkedMessage.receiveChunk();
 
             // Deserialize the result
-            Part part = msg.getPart(0);
+            Part part = chunkedMessage.getPart(0);
 
             Object partObj = part.getObject();
             if (partObj instanceof Throwable) {
@@ -203,9 +203,9 @@ public class RegisterInterestOp {
                 list.clear();
                 list.add(partObj);
 
-                if (r != null) {
+                if (localRegion != null) {
                   try {
-                    r.refreshEntriesFromServerKeys(con, listOfList,
+                    localRegion.refreshEntriesFromServerKeys(con, listOfList,
                         InterestResultPolicy.KEYS_VALUES);
                   } catch (Exception ex) {
                     // GemFireCacheImpl.getInstance().getLogger().config("hitesh error2 " +
@@ -218,7 +218,7 @@ public class RegisterInterestOp {
               }
             }
 
-          } while (!msg.isLastChunk());
+          } while (!chunkedMessage.isLastChunk());
           if (serverEntries != null) {
             list.clear();
             list.add(serverEntries); // serverEntries will always be empty.
@@ -228,13 +228,13 @@ public class RegisterInterestOp {
         }
         case MessageType.RESPONSE_FROM_SECONDARY:
           // Read the chunk
-          msg.receiveChunk();
+          chunkedMessage.receiveChunk();
           return null;
         case MessageType.EXCEPTION:
           // Read the chunk
-          msg.receiveChunk();
+          chunkedMessage.receiveChunk();
           // Deserialize the result
-          Part part = msg.getPart(0);
+          Part part = chunkedMessage.getPart(0);
           // Get the exception toString part.
           // This was added for c++ thin client and not used in java
           // Part exceptionToStringPart = msg.getPart(1);
@@ -244,14 +244,14 @@ public class RegisterInterestOp {
         }
         case MessageType.REGISTER_INTEREST_DATA_ERROR:
           // Read the chunk
-          msg.receiveChunk();
+          chunkedMessage.receiveChunk();
 
           // Deserialize the result
-          String errorMessage = msg.getPart(0).getString();
+          String errorMessage = chunkedMessage.getPart(0).getString();
           String s = this + ": While performing a remote " + getOpName() + ": ";
           throw new ServerOperationException(s + errorMessage);
         default:
-          throw new InternalGemFireError("Unknown message type " + msg.getMessageType());
+          throw new InternalGemFireError("Unknown message type " + chunkedMessage.getMessageType());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/execute/internal/FunctionServiceManager.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/execute/internal/FunctionServiceManager.java b/geode-core/src/main/java/org/apache/geode/cache/execute/internal/FunctionServiceManager.java
index bb31989..f6573a7 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/execute/internal/FunctionServiceManager.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/execute/internal/FunctionServiceManager.java
@@ -14,12 +14,24 @@
  */
 package org.apache.geode.cache.execute.internal;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.geode.cache.Region;
 import org.apache.geode.cache.RegionService;
 import org.apache.geode.cache.client.ClientCache;
 import org.apache.geode.cache.client.ClientCacheFactory;
 import org.apache.geode.cache.client.Pool;
 import org.apache.geode.cache.client.PoolManager;
+import org.apache.geode.cache.client.internal.InternalClientCache;
 import org.apache.geode.cache.client.internal.ProxyCache;
 import org.apache.geode.cache.client.internal.ProxyRegion;
 import org.apache.geode.cache.execute.Execution;
@@ -33,12 +45,13 @@ import org.apache.geode.distributed.internal.DistributionConfig;
 import org.apache.geode.internal.InternalEntity;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
-import org.apache.geode.internal.cache.execute.*;
+import org.apache.geode.internal.cache.execute.DistributedRegionFunctionExecutor;
+import org.apache.geode.internal.cache.execute.MemberFunctionExecutor;
+import org.apache.geode.internal.cache.execute.PartitionedRegionFunctionExecutor;
+import org.apache.geode.internal.cache.execute.ServerFunctionExecutor;
+import org.apache.geode.internal.cache.execute.ServerRegionFunctionExecutor;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-
 /**
  * Provides the entry point into execution of user defined {@linkplain Function}s.
  * <p>
@@ -52,8 +65,9 @@ import java.util.concurrent.ConcurrentHashMap;
  * @since GemFire 7.0
  */
 public class FunctionServiceManager {
-  private final static ConcurrentHashMap<String, Function> idToFunctionMap =
-      new ConcurrentHashMap<String, Function>();
+
+  private static final ConcurrentHashMap<String, Function> idToFunctionMap =
+      new ConcurrentHashMap<>();
 
   /**
    * use when the optimization to execute onMember locally is not desired.
@@ -61,8 +75,9 @@ public class FunctionServiceManager {
   public static final boolean RANDOM_onMember =
       Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "randomizeOnMember");
 
-  public FunctionServiceManager() {}
-
+  public FunctionServiceManager() {
+    // do nothing
+  }
 
   /**
    * Returns an {@link Execution} object that can be used to execute a data dependent function on
@@ -80,12 +95,11 @@ public class FunctionServiceManager {
    * with DataPolicy.PARTITION, it executes on members where the data resides as specified by the
    * filter.
    * 
-   * @param region
    * @return Execution
    * @throws FunctionException if the region passed in is null
    * @since GemFire 6.0
    */
-  public final Execution onRegion(Region region) {
+  public Execution onRegion(Region region) {
     if (region == null) {
       throw new FunctionException(
           LocalizedStrings.FunctionService_0_PASSED_IS_NULL.toLocalizedString("Region instance "));
@@ -97,9 +111,9 @@ public class FunctionServiceManager {
       Pool pool = PoolManager.find(poolName);
       if (pool.getMultiuserAuthentication()) {
         if (region instanceof ProxyRegion) {
-          ProxyRegion pr = (ProxyRegion) region;
-          region = pr.getRealRegion();
-          proxyCache = (ProxyCache) pr.getAuthenticatedCache();
+          ProxyRegion proxyRegion = (ProxyRegion) region;
+          region = proxyRegion.getRealRegion();
+          proxyCache = proxyRegion.getAuthenticatedCache();
         } else {
           throw new UnsupportedOperationException();
         }
@@ -127,7 +141,7 @@ public class FunctionServiceManager {
    * @throws FunctionException if Pool instance passed in is null
    * @since GemFire 6.0
    */
-  public final Execution onServer(Pool pool, String... groups) {
+  public Execution onServer(Pool pool, String... groups) {
     if (pool == null) {
       throw new FunctionException(
           LocalizedStrings.FunctionService_0_PASSED_IS_NULL.toLocalizedString("Pool instance "));
@@ -150,7 +164,7 @@ public class FunctionServiceManager {
    * @throws FunctionException if Pool instance passed in is null
    * @since GemFire 6.0
    */
-  public final Execution onServers(Pool pool, String... groups) {
+  public Execution onServers(Pool pool, String... groups) {
     if (pool == null) {
       throw new FunctionException(
           LocalizedStrings.FunctionService_0_PASSED_IS_NULL.toLocalizedString("Pool instance "));
@@ -177,23 +191,24 @@ public class FunctionServiceManager {
    *         pool
    * @since GemFire 6.5
    */
-  public final Execution onServer(RegionService regionService, String... groups) {
+  public Execution onServer(RegionService regionService, String... groups) {
     if (regionService == null) {
       throw new FunctionException(LocalizedStrings.FunctionService_0_PASSED_IS_NULL
           .toLocalizedString("RegionService instance "));
     }
     if (regionService instanceof GemFireCacheImpl) {
-      GemFireCacheImpl gfc = (GemFireCacheImpl) regionService;
-      if (!gfc.isClient()) {
+      InternalClientCache internalCache = (InternalClientCache) regionService;
+      if (!internalCache.isClient()) {
         throw new FunctionException("The cache was not a client cache");
-      } else if (gfc.getDefaultPool() != null) {
-        return onServer(gfc.getDefaultPool(), groups);
+      } else if (internalCache.getDefaultPool() != null) {
+        return onServer(internalCache.getDefaultPool(), groups);
       } else {
         throw new FunctionException("The client cache does not have a default pool");
       }
     } else {
-      ProxyCache pc = (ProxyCache) regionService;
-      return new ServerFunctionExecutor(pc.getUserAttributes().getPool(), false, pc, groups);
+      ProxyCache proxyCache = (ProxyCache) regionService;
+      return new ServerFunctionExecutor(proxyCache.getUserAttributes().getPool(), false, proxyCache,
+          groups);
     }
   }
 
@@ -209,23 +224,24 @@ public class FunctionServiceManager {
    *         pool
    * @since GemFire 6.5
    */
-  public final Execution onServers(RegionService regionService, String... groups) {
+  public Execution onServers(RegionService regionService, String... groups) {
     if (regionService == null) {
       throw new FunctionException(LocalizedStrings.FunctionService_0_PASSED_IS_NULL
           .toLocalizedString("RegionService instance "));
     }
     if (regionService instanceof GemFireCacheImpl) {
-      GemFireCacheImpl gfc = (GemFireCacheImpl) regionService;
-      if (!gfc.isClient()) {
+      InternalClientCache internalCache = (InternalClientCache) regionService;
+      if (!internalCache.isClient()) {
         throw new FunctionException("The cache was not a client cache");
-      } else if (gfc.getDefaultPool() != null) {
-        return onServers(gfc.getDefaultPool(), groups);
+      } else if (internalCache.getDefaultPool() != null) {
+        return onServers(internalCache.getDefaultPool(), groups);
       } else {
         throw new FunctionException("The client cache does not have a default pool");
       }
     } else {
-      ProxyCache pc = (ProxyCache) regionService;
-      return new ServerFunctionExecutor(pc.getUserAttributes().getPool(), true, pc, groups);
+      ProxyCache proxyCache = (ProxyCache) regionService;
+      return new ServerFunctionExecutor(proxyCache.getUserAttributes().getPool(), true, proxyCache,
+          groups);
     }
   }
 
@@ -242,7 +258,7 @@ public class FunctionServiceManager {
    * @since GemFire 6.0
    * 
    */
-  public final Execution onMember(DistributedSystem system, DistributedMember distributedMember) {
+  public Execution onMember(DistributedSystem system, DistributedMember distributedMember) {
     if (system == null) {
       throw new FunctionException(LocalizedStrings.FunctionService_0_PASSED_IS_NULL
           .toLocalizedString("DistributedSystem instance "));
@@ -265,7 +281,7 @@ public class FunctionServiceManager {
    * @throws FunctionException if DistributedSystem instance passed is null
    * @since GemFire 6.0
    */
-  public final Execution onMembers(DistributedSystem system, String... groups) {
+  public Execution onMembers(DistributedSystem system, String... groups) {
     if (system == null) {
       throw new FunctionException(LocalizedStrings.FunctionService_0_PASSED_IS_NULL
           .toLocalizedString("DistributedSystem instance "));
@@ -294,8 +310,7 @@ public class FunctionServiceManager {
    * @throws FunctionException if DistributedSystem instance passed is null
    * @since GemFire 6.0
    */
-  public final Execution onMembers(DistributedSystem system,
-      Set<DistributedMember> distributedMembers) {
+  public Execution onMembers(DistributedSystem system, Set<DistributedMember> distributedMembers) {
     if (system == null) {
       throw new FunctionException(LocalizedStrings.FunctionService_0_PASSED_IS_NULL
           .toLocalizedString("DistributedSystem instance "));
@@ -311,12 +326,11 @@ public class FunctionServiceManager {
    * Returns the {@link Function} defined by the functionId, returns null if no function is found
    * for the specified functionId
    * 
-   * @param functionId
    * @return Function
    * @throws FunctionException if functionID passed is null
    * @since GemFire 6.0
    */
-  public final Function getFunction(String functionId) {
+  public Function getFunction(String functionId) {
     if (functionId == null) {
       throw new FunctionException(LocalizedStrings.FunctionService_0_PASSED_IS_NULL
           .toLocalizedString("functionId instance "));
@@ -336,7 +350,7 @@ public class FunctionServiceManager {
    * @throws FunctionException if function instance passed is null or Function.getId() returns null
    * @since GemFire 6.0
    */
-  public final void registerFunction(Function function) {
+  public void registerFunction(Function function) {
     if (function == null) {
       throw new FunctionException(LocalizedStrings.FunctionService_0_PASSED_IS_NULL
           .toLocalizedString("function instance "));
@@ -361,7 +375,7 @@ public class FunctionServiceManager {
    * @throws FunctionException if function instance passed is null or Function.getId() returns null
    * @since GemFire 6.0
    */
-  public final void unregisterFunction(String functionId) {
+  public void unregisterFunction(String functionId) {
     if (functionId == null) {
       throw new FunctionException(LocalizedStrings.FunctionService_0_PASSED_IS_NULL
           .toLocalizedString("functionId instance "));
@@ -375,7 +389,7 @@ public class FunctionServiceManager {
    * @throws FunctionException if function instance passed is null or Function.getId() returns null
    * @since GemFire 6.0
    */
-  public final boolean isRegistered(String functionId) {
+  public boolean isRegistered(String functionId) {
     if (functionId == null) {
       throw new FunctionException(LocalizedStrings.FunctionService_0_PASSED_IS_NULL
           .toLocalizedString("functionId instance "));
@@ -389,7 +403,7 @@ public class FunctionServiceManager {
    * @return A view of registered functions as a Map of {@link Function#getId()} to {@link Function}
    * @since GemFire 6.0
    */
-  public final Map<String, Function> getRegisteredFunctions() {
+  public Map<String, Function> getRegisteredFunctions() {
     // We have to remove the internal functions before returning the map to the users
     final Map<String, Function> tempIdToFunctionMap = new HashMap<String, Function>();
     for (Map.Entry<String, Function> entry : idToFunctionMap.entrySet()) {
@@ -400,7 +414,7 @@ public class FunctionServiceManager {
     return tempIdToFunctionMap;
   }
 
-  public final void unregisterAllFunctions() {
+  public void unregisterAllFunctions() {
     // Unregistering all the functions registered with the FunctionService.
     Map<String, Function> functions = new HashMap<String, Function>(idToFunctionMap);
     for (String functionId : idToFunctionMap.keySet()) {
@@ -409,25 +423,22 @@ public class FunctionServiceManager {
   }
 
   /**
-   * @param region
    * @return true if the method is called on a region has a {@link Pool}.
    * @since GemFire 6.0
    */
-  private final boolean isClientRegion(Region region) {
+  private boolean isClientRegion(Region region) {
     LocalRegion localRegion = (LocalRegion) region;
     return localRegion.hasServerProxy();
   }
 
-
-  public final Execution onMember(DistributedSystem system, String... groups) {
+  public Execution onMember(DistributedSystem system, String... groups) {
     if (system == null) {
       throw new FunctionException(LocalizedStrings.FunctionService_0_PASSED_IS_NULL
           .toLocalizedString("DistributedSystem instance "));
     }
-    Set<DistributedMember> members = new HashSet<DistributedMember>();
+    Set<DistributedMember> members = new HashSet<>();
     for (String group : groups) {
-      List<DistributedMember> grpMembers =
-          new ArrayList<DistributedMember>(system.getGroupMembers(group));
+      List<DistributedMember> grpMembers = new ArrayList<>(system.getGroupMembers(group));
       if (!grpMembers.isEmpty()) {
         if (!RANDOM_onMember && grpMembers.contains(system.getDistributedMember())) {
           members.add(system.getDistributedMember());

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/partition/PartitionRegionHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/partition/PartitionRegionHelper.java b/geode-core/src/main/java/org/apache/geode/cache/partition/PartitionRegionHelper.java
index bf79edb..41c022c 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/partition/PartitionRegionHelper.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/partition/PartitionRegionHelper.java
@@ -16,7 +16,6 @@ package org.apache.geode.cache.partition;
 
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
@@ -35,7 +34,7 @@ import org.apache.geode.cache.execute.RegionFunctionContext;
 import org.apache.geode.distributed.DistributedMember;
 import org.apache.geode.internal.cache.ColocationHelper;
 import org.apache.geode.internal.cache.FixedPartitionAttributesImpl;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
+import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.LocalDataSet;
 import org.apache.geode.internal.cache.PartitionedRegion;
 import org.apache.geode.internal.cache.PartitionedRegion.RecoveryLock;
@@ -50,7 +49,6 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
 /**
  * Utility methods for handling partitioned Regions, for example during execution of {@link Function
  * Functions} on a Partitioned Region.
- * 
  * <p>
  * Example of a Function using utility methods:
  * 
@@ -68,12 +66,14 @@ import org.apache.geode.internal.i18n.LocalizedStrings;
  *  // ...
  * </pre>
  *
- * 
  * @since GemFire 6.0
  * @see FunctionService#onRegion(Region)
  */
 public final class PartitionRegionHelper {
-  private PartitionRegionHelper() {}
+
+  private PartitionRegionHelper() {
+    // do nothing
+  }
 
   /**
    * Given a partitioned Region, return a map of
@@ -157,8 +157,8 @@ public final class PartitionRegionHelper {
    * @since GemFire 6.0
    */
   public static Set<PartitionRegionInfo> getPartitionRegionInfo(final Cache cache) {
-    Set<PartitionRegionInfo> prDetailsSet = new TreeSet<PartitionRegionInfo>();
-    fillInPartitionedRegionInfo((GemFireCacheImpl) cache, prDetailsSet, false);
+    Set<PartitionRegionInfo> prDetailsSet = new TreeSet<>();
+    fillInPartitionedRegionInfo((InternalCache) cache, prDetailsSet, false);
     return prDetailsSet;
   }
 
@@ -172,26 +172,25 @@ public final class PartitionRegionHelper {
    */
   public static PartitionRegionInfo getPartitionRegionInfo(final Region<?, ?> region) {
     try {
-      PartitionedRegion pr = isPartitionedCheck(region);
-      GemFireCacheImpl cache = (GemFireCacheImpl) region.getCache();
-      return pr.getRedundancyProvider().buildPartitionedRegionInfo(false,
-          cache.getInternalResourceManager().getLoadProbe()); // may return null
-    } catch (ClassCastException e) {
+      PartitionedRegion partitionedRegion = isPartitionedCheck(region);
+      InternalCache cache = (InternalCache) region.getCache();
+      return partitionedRegion.getRedundancyProvider().buildPartitionedRegionInfo(false,
+          cache.getInternalResourceManager().getLoadProbe());
+    } catch (ClassCastException ignore) {
       // not a PR so return null
     }
     return null;
   }
 
-  private static void fillInPartitionedRegionInfo(GemFireCacheImpl cache, final Set prDetailsSet,
+  private static void fillInPartitionedRegionInfo(final InternalCache cache, final Set prDetailsSet,
       final boolean internal) {
     // TODO: optimize by fetching all PR details from each member at once
-    Set<PartitionedRegion> prSet = cache.getPartitionedRegions();
-    if (prSet.isEmpty()) {
+    Set<PartitionedRegion> partitionedRegions = cache.getPartitionedRegions();
+    if (partitionedRegions.isEmpty()) {
       return;
     }
-    for (Iterator<PartitionedRegion> iter = prSet.iterator(); iter.hasNext();) {
-      PartitionedRegion pr = iter.next();
-      PartitionRegionInfo prDetails = pr.getRedundancyProvider()
+    for (PartitionedRegion partitionedRegion : partitionedRegions) {
+      PartitionRegionInfo prDetails = partitionedRegion.getRedundancyProvider()
           .buildPartitionedRegionInfo(internal, cache.getInternalResourceManager().getLoadProbe());
       if (prDetails != null) {
         prDetailsSet.add(prDetails);

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/query/internal/AbstractCompiledValue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/AbstractCompiledValue.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/AbstractCompiledValue.java
index bed0680..85ad6fa 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/AbstractCompiledValue.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/AbstractCompiledValue.java
@@ -248,7 +248,7 @@ public abstract class AbstractCompiledValue implements CompiledValue, Filter, OQ
   }
 
   // This function needs to be appropriately overridden in the derived classes
-  public void generateCanonicalizedExpression(StringBuffer clauseBuffer, ExecutionContext context)
+  public void generateCanonicalizedExpression(StringBuilder clauseBuffer, ExecutionContext context)
       throws AmbiguousNameException, TypeMismatchException, NameResolutionException {
     clauseBuffer.insert(0, System.currentTimeMillis());
     clauseBuffer.insert(0, this.getClass());

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledAggregateFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledAggregateFunction.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledAggregateFunction.java
index 21607de..9544bbb 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledAggregateFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledAggregateFunction.java
@@ -156,7 +156,7 @@ public class CompiledAggregateFunction extends AbstractCompiledValue {
   }
 
   @Override
-  public void generateCanonicalizedExpression(StringBuffer clauseBuffer, ExecutionContext context)
+  public void generateCanonicalizedExpression(StringBuilder clauseBuffer, ExecutionContext context)
       throws AmbiguousNameException, TypeMismatchException, NameResolutionException {
     clauseBuffer.insert(0, ')');
     if (this.expr != null) {

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledBindArgument.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledBindArgument.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledBindArgument.java
index 106d389..03ec478 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledBindArgument.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledBindArgument.java
@@ -43,7 +43,7 @@ public class CompiledBindArgument extends AbstractCompiledValue {
   }
 
   @Override
-  public void generateCanonicalizedExpression(StringBuffer clauseBuffer, ExecutionContext context)
+  public void generateCanonicalizedExpression(StringBuilder clauseBuffer, ExecutionContext context)
       throws AmbiguousNameException, TypeMismatchException, NameResolutionException {
 
     // When compiling a new query, a context is created where there are no bind arguments at this

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledComparison.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledComparison.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledComparison.java
index d47509f..360d655 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledComparison.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledComparison.java
@@ -479,7 +479,7 @@ public class CompiledComparison extends AbstractCompiledValue
       }
     }
     if (conditioningNeeded) {
-      return QueryUtils.getconditionedIndexResults(set, indexInfo, context, indexFieldsSize,
+      return QueryUtils.getConditionedIndexResults(set, indexInfo, context, indexFieldsSize,
           completeExpansionNeeded, iterOperands, indpndntItr);
     } else {
       return set;
@@ -566,7 +566,7 @@ public class CompiledComparison extends AbstractCompiledValue
       } finally {
         observer.afterIndexLookup(data);
       }
-      return QueryUtils.getconditionedRelationshipIndexResultsExpandedToTopOrCGJLevel(data,
+      return QueryUtils.getConditionedRelationshipIndexResultsExpandedToTopOrCGJLevel(data,
           indxInfo, context, completeExpansionNeeded, iterOperands, indpdntItrs);
     } else {
       // Asif . We are in this block , this itself guarantees that this

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledFunction.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledFunction.java
index 0c81d24..349e3f4 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledFunction.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledFunction.java
@@ -18,8 +18,6 @@ package org.apache.geode.cache.query.internal;
 import java.util.*;
 import org.apache.geode.cache.query.*;
 import org.apache.geode.internal.i18n.LocalizedStrings;
-import org.apache.geode.pdx.internal.PdxString;
-
 
 
 /**
@@ -89,7 +87,7 @@ public class CompiledFunction extends AbstractCompiledValue {
   }
 
   @Override
-  public void generateCanonicalizedExpression(StringBuffer clauseBuffer, ExecutionContext context)
+  public void generateCanonicalizedExpression(StringBuilder clauseBuffer, ExecutionContext context)
       throws AmbiguousNameException, TypeMismatchException, NameResolutionException {
     clauseBuffer.insert(0, ')');
     int len = this._args.length;

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledGroupBySelect.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledGroupBySelect.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledGroupBySelect.java
index 0ad093d..1c4a691 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledGroupBySelect.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledGroupBySelect.java
@@ -67,7 +67,7 @@ public class CompiledGroupBySelect extends CompiledSelect {
     if (aggMap != null) {
       int i = 0;
       for (Map.Entry<Integer, CompiledAggregateFunction> entry : aggMap.entrySet()) {
-        this.aggregateColsPos.set(entry.getKey().intValue());
+        this.aggregateColsPos.set(entry.getKey());
         this.aggregateFunctions[i++] = entry.getValue();
       }
     }
@@ -100,7 +100,7 @@ public class CompiledGroupBySelect extends CompiledSelect {
       if (param == null && aggFunc.getFunctionType() == OQLLexerTokenTypes.COUNT) {
         // * case of *, substitue a dummy parameter of compiled literal = 0 to
         // satisfy the code
-        param = new CompiledLiteral(Integer.valueOf(0));
+        param = new CompiledLiteral(0);
 
       } else if (param == null) {
         throw new QueryInvalidException("aggregate function passed invalid parameter");
@@ -468,7 +468,7 @@ public class CompiledGroupBySelect extends CompiledSelect {
   private boolean checkProjectionInGroupBy(Object[] projElem, ExecutionContext context)
       throws AmbiguousNameException, TypeMismatchException, NameResolutionException {
     boolean found = false;
-    StringBuffer projAttribBuffer = new StringBuffer();
+    StringBuilder projAttribBuffer = new StringBuilder();
     CompiledValue cvProj = (CompiledValue) TypeUtils.checkCast(projElem[1], CompiledValue.class);
     cvProj.generateCanonicalizedExpression(projAttribBuffer, context);
     String projAttribStr = projAttribBuffer.toString();
@@ -482,7 +482,7 @@ public class CompiledGroupBySelect extends CompiledSelect {
         }
 
         // the grpup by expr is not an alias check for path
-        StringBuffer groupByExprBuffer = new StringBuffer();
+        StringBuilder groupByExprBuffer = new StringBuilder();
         grpBy.generateCanonicalizedExpression(groupByExprBuffer, context);
         final String grpByExprStr = groupByExprBuffer.toString();
 

http://git-wip-us.apache.org/repos/asf/geode/blob/21f405b8/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledID.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledID.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledID.java
index cfb02f7..6eb716b 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledID.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/CompiledID.java
@@ -82,7 +82,7 @@ public class CompiledID extends AbstractCompiledValue {
   }
 
   @Override
-  public void generateCanonicalizedExpression(StringBuffer clauseBuffer, ExecutionContext context)
+  public void generateCanonicalizedExpression(StringBuilder clauseBuffer, ExecutionContext context)
       throws AmbiguousNameException, TypeMismatchException, NameResolutionException {
     // The compiled ID can be an iterator variable or it can be a path variable.
     // So first resolve the type of variable using ExecutionContext