You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ud...@apache.org on 2018/09/07 18:59:47 UTC

[geode] branch feature/GEODE-5705 updated: GEODE-5705: fixed logic to read Strings avoid double switch statements. Cleaned up more locations where switch statements could have been used.

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

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


The following commit(s) were added to refs/heads/feature/GEODE-5705 by this push:
     new c7878ff  GEODE-5705: fixed logic to read Strings avoid double switch statements. Cleaned up more locations where switch statements could have been used.
c7878ff is described below

commit c7878ffe26e9efe6be08e7a1ef88dde396e85c48
Author: Udo Kohlmeyer <uk...@pivotal.io>
AuthorDate: Fri Sep 7 11:59:30 2018 -0700

    GEODE-5705: fixed logic to read Strings avoid double switch statements.
    Cleaned up more locations where switch statements could have been used.
---
 .../geode/internal/InternalDataSerializer.java     | 1194 ++++++++++----------
 .../java/org/apache/geode/internal/DSCODETest.java |    3 +-
 2 files changed, 576 insertions(+), 621 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java b/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
index a4115f0..db34804 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
@@ -132,15 +132,22 @@ import org.apache.geode.pdx.internal.TypeRegistry;
  * @since GemFire 3.5
  */
 public abstract class InternalDataSerializer extends DataSerializer {
+  // array is null
+  public static final byte NULL_ARRAY = -1;
+  /**
+   * array len encoded as int in next 4 bytes
+   *
+   * @since GemFire 5.7
+   */
+  public static final byte INT_ARRAY_LEN = -3;
+  public static final boolean LOAD_CLASS_EACH_TIME =
+      Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "loadClassOnEveryDeserialization");
   private static final Logger logger = LogService.getLogger();
-
   /**
    * Maps Class names to their DataSerializer. This is used to find a DataSerializer during
    * serialization.
    */
   private static final Map<String, DataSerializer> classesToSerializers = new ConcurrentHashMap<>();
-
-
   /**
    * This list contains classes that Geode's classes subclass, such as antlr AST classes which are
    * used by our Object Query Language. It also contains certain classes that are DataSerializable
@@ -192,22 +199,80 @@ public abstract class InternalDataSerializer extends DataSerializer {
 
           // geode-modules
           + ";org.apache.geode.modules.util.SessionCustomExpiry" + ";";
-
-
+  private static final String serializationVersionTxt =
+      System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "serializationVersion");
+  /**
+   * Change this constant to be the last one in SERIALIZATION_VERSION
+   */
+  private static final SERIALIZATION_VERSION latestVersion = SERIALIZATION_VERSION.v662;
+  private static final SERIALIZATION_VERSION serializationVersion = calculateSerializationVersion();
+  /**
+   * Maps the id of a serializer to its {@code DataSerializer}.
+   */
+  private static final ConcurrentMap/* <Integer, DataSerializer|Marker> */ idsToSerializers =
+      new ConcurrentHashMap();
+  /**
+   * Contains the classnames of the data serializers (and not the supported classes) not yet loaded
+   * into the vm as keys and their corresponding holder instances as values.
+   */
+  private static final ConcurrentHashMap<String, SerializerAttributesHolder> dsClassesToHolders =
+      new ConcurrentHashMap<>();
+  /**
+   * Contains the id of the data serializers not yet loaded into the vm as keys and their
+   * corresponding holder instances as values.
+   */
+  private static final ConcurrentHashMap<Integer, SerializerAttributesHolder> idsToHolders =
+      new ConcurrentHashMap<>();
+  /**
+   * Contains the classnames of supported classes as keys and their corresponding
+   * SerializerAttributesHolder instances as values. This applies only to the data serializers which
+   * have not been loaded into the vm.
+   */
+  private static final ConcurrentHashMap<String, SerializerAttributesHolder> supportedClassesToHolders =
+      new ConcurrentHashMap<>();
+  private static final Object listenersSync = new Object();
+  private static final byte TIME_UNIT_NANOSECONDS = -1;
+  private static final byte TIME_UNIT_MICROSECONDS = -2;
+  private static final byte TIME_UNIT_MILLISECONDS = -3;
+  private static final byte TIME_UNIT_SECONDS = -4;
+  private static final ConcurrentMap dsfidToClassMap =
+      logger.isTraceEnabled(LogMarker.SERIALIZER_WRITE_DSFID_VERBOSE) ? new ConcurrentHashMap()
+          : null;
+  /**
+   * array len encoded as unsigned short in next 2 bytes
+   *
+   * @since GemFire 5.7
+   */
+  private static final byte SHORT_ARRAY_LEN = -2;
+  private static final int MAX_BYTE_ARRAY_LEN = (byte) -4 & 0xFF;
+  private static final ThreadLocal<Boolean> pdxSerializationInProgress = new ThreadLocal<>();
+  // Variable Length long encoded as int in next 4 bytes
+  private static final byte INT_VL = 126;
+  // Variable Length long encoded as long in next 8 bytes
+  private static final byte LONG_VL = 127;
+  private static final int MAX_BYTE_VL = 125;
+  private static final CopyOnWriteHashMap<String, WeakReference<Class<?>>> classCache =
+      LOAD_CLASS_EACH_TIME ? null : new CopyOnWriteHashMap<>();
+  private static final Object cacheAccessLock = new Object();
   private static InputStreamFilter defaultSerializationFilter = new EmptyInputStreamFilter();
-
   /**
    * A deserialization filter for ObjectInputStreams
    */
   private static InputStreamFilter serializationFilter = defaultSerializationFilter;
-
-  private static final String serializationVersionTxt =
-      System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "serializationVersion");
-
   /**
    * support for old GemFire clients and WAN sites - needed to enable moving from GemFire to Geode
    */
   private static OldClientSupportService oldClientSupportService;
+  /**
+   * {@code RegistrationListener}s that receive callbacks when {@code DataSerializer}s and {@code
+   * Instantiator}s are registered. Note: copy-on-write access used for this set
+   */
+  private static volatile Set listeners = new HashSet();
+  private static DataSerializer dvddeserializer;
+
+  static {
+    initializeWellKnownSerializers();
+  }
 
   /**
    * For backward compatibility we must swizzle the package of some classes that had to be moved
@@ -282,7 +347,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     serializationFilter = defaultSerializationFilter;
   }
 
-
   /**
    * {@link DistributedSystemService}s that need to acceptlist Serializable objects can use this to
    * read them from a file and then return them via
@@ -312,25 +376,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
 
   }
 
-
-  /**
-   * Any time new serialization format is added then a new enum needs to be added here.
-   *
-   * @since GemFire 6.6.2
-   */
-  private enum SERIALIZATION_VERSION {
-    vINVALID,
-    // includes 6.6.0.x and 6.6.1.x. Note that no serialization changes were made in 6.6 until 6.6.2
-    v660,
-    // 6.6.2.x or later NOTE if you add a new constant make sure and update "latestVersion".
-    v662
-  }
-
-  /**
-   * Change this constant to be the last one in SERIALIZATION_VERSION
-   */
-  private static final SERIALIZATION_VERSION latestVersion = SERIALIZATION_VERSION.v662;
-
   private static SERIALIZATION_VERSION calculateSerializationVersion() {
     if (serializationVersionTxt == null || serializationVersionTxt.isEmpty()) {
       return latestVersion;
@@ -344,8 +389,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-  private static final SERIALIZATION_VERSION serializationVersion = calculateSerializationVersion();
-
   public static boolean is662SerializationEnabled() {
     return serializationVersion.ordinal() >= SERIALIZATION_VERSION.v662.ordinal();
   }
@@ -359,10 +402,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-  static {
-    initializeWellKnownSerializers();
-  }
-
   private static void initializeWellKnownSerializers() {
     // ArrayBlockingQueue does not have zero-arg constructor
     // LinkedBlockingQueue does have zero-arg constructor but no way to get capacity
@@ -770,42 +809,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
   }
 
   /**
-   * Maps the id of a serializer to its {@code DataSerializer}.
-   */
-  private static final ConcurrentMap/* <Integer, DataSerializer|Marker> */ idsToSerializers =
-      new ConcurrentHashMap();
-
-  /**
-   * Contains the classnames of the data serializers (and not the supported classes) not yet loaded
-   * into the vm as keys and their corresponding holder instances as values.
-   */
-  private static final ConcurrentHashMap<String, SerializerAttributesHolder> dsClassesToHolders =
-      new ConcurrentHashMap<>();
-
-  /**
-   * Contains the id of the data serializers not yet loaded into the vm as keys and their
-   * corresponding holder instances as values.
-   */
-  private static final ConcurrentHashMap<Integer, SerializerAttributesHolder> idsToHolders =
-      new ConcurrentHashMap<>();
-
-  /**
-   * Contains the classnames of supported classes as keys and their corresponding
-   * SerializerAttributesHolder instances as values. This applies only to the data serializers which
-   * have not been loaded into the vm.
-   */
-  private static final ConcurrentHashMap<String, SerializerAttributesHolder> supportedClassesToHolders =
-      new ConcurrentHashMap<>();
-
-  /**
-   * {@code RegistrationListener}s that receive callbacks when {@code DataSerializer}s and {@code
-   * Instantiator}s are registered. Note: copy-on-write access used for this set
-   */
-  private static volatile Set listeners = new HashSet();
-
-  private static final Object listenersSync = new Object();
-
-  /**
    * Convert the given unsigned byte to an int. The returned value will be in the range [0..255]
    * inclusive
    */
@@ -813,14 +816,14 @@ public abstract class InternalDataSerializer extends DataSerializer {
     return ub & 0xFF;
   }
 
-  public static void setOldClientSupportService(final OldClientSupportService svc) {
-    oldClientSupportService = svc;
-  }
-
   public static OldClientSupportService getOldClientSupportService() {
     return oldClientSupportService;
   }
 
+  public static void setOldClientSupportService(final OldClientSupportService svc) {
+    oldClientSupportService = svc;
+  }
+
   /**
    * Instantiates an instance of {@code DataSerializer}
    *
@@ -1098,53 +1101,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-  /**
-   * A SerializerAttributesHolder holds information required to load a DataSerializer and exists to
-   * allow client/server connections to be created more quickly than they would if the
-   * DataSerializer information downloaded from the server were used to immediately load the
-   * corresponding classes.
-   */
-  public static class SerializerAttributesHolder {
-    private String className = "";
-    private EventID eventId = null;
-    private ClientProxyMembershipID proxyId = null;
-    private int id = 0;
-
-    SerializerAttributesHolder() {}
-
-    SerializerAttributesHolder(String name, EventID event, ClientProxyMembershipID proxy, int id) {
-      this.className = name;
-      this.eventId = event;
-      this.proxyId = proxy;
-      this.id = id;
-    }
-
-    /**
-     * @return String the classname of the data serializer this instance represents.
-     */
-    public String getClassName() {
-      return this.className;
-    }
-
-    public EventID getEventId() {
-      return this.eventId;
-    }
-
-    public ClientProxyMembershipID getProxyId() {
-      return this.proxyId;
-    }
-
-    public int getId() {
-      return this.id;
-    }
-
-    @Override
-    public String toString() {
-      return "SerializerAttributesHolder[name=" + this.className + ",id=" + this.id + ",eventId="
-          + this.eventId + ']';
-    }
-  }
-
   private static void sendRegistrationMessageToServers(DataSerializer dataSerializer) {
     PoolManagerImpl.allPoolsRegisterDataSerializers(dataSerializer);
   }
@@ -1685,7 +1641,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-
   public static boolean autoSerialized(Object o, DataOutput out) throws IOException {
     AutoSerializableManager asm = TypeRegistry.getAutoSerializableManager();
     if (asm != null) {
@@ -1964,7 +1919,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-
   /**
    * Writes the type code for a primitive type Class to {@code DataOutput}.
    */
@@ -1997,44 +1951,35 @@ public abstract class InternalDataSerializer extends DataSerializer {
   }
 
   public static Class decodePrimitiveClass(byte typeCode) {
-    if (typeCode == DSCODE.BOOLEAN_TYPE.toByte()) {
-      return Boolean.TYPE;
-    }
-    if (typeCode == DSCODE.CHARACTER_TYPE.toByte()) {
-      return Character.TYPE;
-    }
-    if (typeCode == DSCODE.BYTE_TYPE.toByte()) {
-      return Byte.TYPE;
-    }
-    if (typeCode == DSCODE.SHORT_TYPE.toByte()) {
-      return Short.TYPE;
-    }
-    if (typeCode == DSCODE.INTEGER_TYPE.toByte()) {
-      return Integer.TYPE;
-    }
-    if (typeCode == DSCODE.LONG_TYPE.toByte()) {
-      return Long.TYPE;
-    }
-    if (typeCode == DSCODE.FLOAT_TYPE.toByte()) {
-      return Float.TYPE;
-    }
-    if (typeCode == DSCODE.DOUBLE_TYPE.toByte()) {
-      return Double.TYPE;
-    }
-    if (typeCode == DSCODE.VOID_TYPE.toByte()) {
-      return Void.TYPE;
-    }
-    if (typeCode == DSCODE.NULL.toByte()) {
-      return null;
+    DSCODE dscode = DscodeHelper.toDSCODE(typeCode);
+    switch (dscode) {
+      case BOOLEAN_TYPE:
+        return Boolean.TYPE;
+      case CHARACTER_TYPE:
+        return Character.TYPE;
+      case BYTE_TYPE:
+        return Byte.TYPE;
+      case SHORT_TYPE:
+        return Short.TYPE;
+      case INTEGER_TYPE:
+        return Integer.TYPE;
+      case LONG_TYPE:
+        return Long.TYPE;
+      case FLOAT_TYPE:
+        return Float.TYPE;
+      case DOUBLE_TYPE:
+        return Double.TYPE;
+      case VOID_TYPE:
+        return Void.TYPE;
+      case NULL:
+        return null;
+      default:
+        throw new InternalGemFireError(
+            LocalizedStrings.InternalDataSerializer_UNEXPECTED_TYPECODE_0
+                .toLocalizedString(typeCode));
     }
-    throw new InternalGemFireError(
-        LocalizedStrings.InternalDataSerializer_UNEXPECTED_TYPECODE_0.toLocalizedString(typeCode));
-  }
 
-  private static final byte TIME_UNIT_NANOSECONDS = -1;
-  private static final byte TIME_UNIT_MICROSECONDS = -2;
-  private static final byte TIME_UNIT_MILLISECONDS = -3;
-  private static final byte TIME_UNIT_SECONDS = -4;
+  }
 
   /**
    * Reads a {@code TimeUnit} from a {@code DataInput}.
@@ -2147,10 +2092,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     return result;
   }
 
-  private static final ConcurrentMap dsfidToClassMap =
-      logger.isTraceEnabled(LogMarker.SERIALIZER_WRITE_DSFID_VERBOSE) ? new ConcurrentHashMap()
-          : null;
-
   public static void writeUserDataSerializableHeader(int classId, DataOutput out)
       throws IOException {
     if (classId <= Byte.MAX_VALUE && classId >= Byte.MIN_VALUE) {
@@ -2423,7 +2364,7 @@ public abstract class InternalDataSerializer extends DataSerializer {
       boolean invoked = false;
       Version v = InternalDataSerializer.getVersionForDataStreamOrNull(out);
 
-      if (v != null && v != Version.CURRENT) {
+      if (Version.CURRENT != v && v != null) {
         // get versions where DataOutput was upgraded
         Version[] versions = null;
         if (ds instanceof SerializationVersions) {
@@ -2432,7 +2373,7 @@ public abstract class InternalDataSerializer extends DataSerializer {
         }
         // check if the version of the peer or diskstore is different and
         // there has been a change in the message
-        if (versions != null && versions.length > 0) {
+        if (versions != null) {
           for (Version version : versions) {
             // if peer version is less than the greatest upgraded version
             if (v.compareTo(version) < 0) {
@@ -2496,7 +2437,7 @@ public abstract class InternalDataSerializer extends DataSerializer {
     try {
       boolean invoked = false;
       Version v = InternalDataSerializer.getVersionForDataStreamOrNull(in);
-      if (v != null && v != Version.CURRENT) {
+      if (Version.CURRENT != v && v != null) {
         // get versions where DataOutput was upgraded
         Version[] versions = null;
         if (ds instanceof SerializationVersions) {
@@ -2505,7 +2446,7 @@ public abstract class InternalDataSerializer extends DataSerializer {
         }
         // check if the version of the peer or diskstore is different and
         // there has been a change in the message
-        if (versions != null && versions.length > 0) {
+        if (versions != null) {
           for (Version version : versions) {
             // if peer version is less than the greatest upgraded version
             if (v.compareTo(version) < 0) {
@@ -2535,7 +2476,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-
   private static Object readDataSerializable(final DataInput in)
       throws IOException, ClassNotFoundException {
     Class c = readClass(in);
@@ -2643,25 +2583,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-  // array is null
-  public static final byte NULL_ARRAY = -1;
-
-  /**
-   * array len encoded as unsigned short in next 2 bytes
-   *
-   * @since GemFire 5.7
-   */
-  private static final byte SHORT_ARRAY_LEN = -2;
-
-  /**
-   * array len encoded as int in next 4 bytes
-   *
-   * @since GemFire 5.7
-   */
-  public static final byte INT_ARRAY_LEN = -3;
-
-  private static final int MAX_BYTE_ARRAY_LEN = (byte) -4 & 0xFF;
-
   public static void writeArrayLength(int len, DataOutput out) throws IOException {
     if (len == -1) {
       out.writeByte(NULL_ARRAY);
@@ -2684,9 +2605,9 @@ public abstract class InternalDataSerializer extends DataSerializer {
       int result = ubyteToInt(code);
       if (result > MAX_BYTE_ARRAY_LEN) {
         if (code == SHORT_ARRAY_LEN) {
-          result = in.readUnsignedShort();
+          return in.readUnsignedShort();
         } else if (code == INT_ARRAY_LEN) {
-          result = in.readInt();
+          return in.readInt();
         } else {
           throw new IllegalStateException("unexpected array length code=" + code);
         }
@@ -2695,27 +2616,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-  /**
-   * Serializes a list of Integers. The argument may be null. Deserialize with
-   * readListOfIntegers().
-   *
-   * TODO: writeListOfIntegers is unused
-   */
-  public void writeListOfIntegers(List<Integer> list, DataOutput out) throws IOException {
-    int size;
-    if (list == null) {
-      size = -1;
-    } else {
-      size = list.size();
-    }
-    InternalDataSerializer.writeArrayLength(size, out);
-    if (size > 0) {
-      for (int i = 0; i < size; i++) {
-        out.writeInt(list.get(i));
-      }
-    }
-  }
-
   private static Object readDSFID(final DataInput in, DSCODE dscode)
       throws IOException, ClassNotFoundException {
     if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
@@ -2769,58 +2669,58 @@ public abstract class InternalDataSerializer extends DataSerializer {
    */
   private static String readString(DataInput in, DSCODE dscode) throws IOException {
     switch (dscode) {
-      case STRING_BYTES: {
-        int len = in.readUnsignedShort();
-        if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
-          logger.trace(LogMarker.SERIALIZER_VERBOSE, "Reading STRING_BYTES of len={}", len);
-        }
-        byte[] buf = new byte[len];
-        in.readFully(buf, 0, len);
-        return new String(buf, 0); // intentionally using deprecated constructor
-      }
-      case STRING: {
-        if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
-          logger.trace(LogMarker.SERIALIZER_VERBOSE, "Reading utf STRING");
-        }
-        return in.readUTF();
-      }
+      case STRING_BYTES:
+        return readStringBytesFromDataInput(in, in.readUnsignedShort());
+      case STRING:
+        return readStringUTFFromDataInput(in);
       case NULL_STRING: {
         if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
           logger.trace(LogMarker.SERIALIZER_VERBOSE, "Reading NULL_STRING");
         }
         return null;
       }
-      case HUGE_STRING_BYTES: {
-        int len = in.readInt();
-        if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
-          logger.trace(LogMarker.SERIALIZER_VERBOSE, "Reading HUGE_STRING_BYTES of len={}", len);
-        }
-        byte[] buf = new byte[len];
-        in.readFully(buf, 0, len);
-        return new String(buf, 0); // intentionally using deprecated constructor
-      }
-      case HUGE_STRING: {
-        int len = in.readInt();
-        if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
-          logger.trace(LogMarker.SERIALIZER_VERBOSE, "Reading HUGE_STRING of len={}", len);
-        }
-        char[] buf = new char[len];
-        for (int i = 0; i < len; i++) {
-          buf[i] = in.readChar();
-        }
-        return new String(buf);
-      }
+      case HUGE_STRING_BYTES:
+        return readStringBytesFromDataInput(in, in.readInt());
+      case HUGE_STRING:
+        return readHugeStringFromDataInput(in);
       default:
         throw new IOException("Unknown String header " + dscode);
     }
   }
 
+  private static String readHugeStringFromDataInput(DataInput in) throws IOException {
+    int len = in.readInt();
+    if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
+      logger.trace(LogMarker.SERIALIZER_VERBOSE, "Reading HUGE_STRING of len={}", len);
+    }
+    char[] buf = new char[len];
+    for (int i = 0; i < len; i++) {
+      buf[i] = in.readChar();
+    }
+    return new String(buf);
+  }
+
+  private static String readStringUTFFromDataInput(DataInput in) throws IOException {
+    if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
+      logger.trace(LogMarker.SERIALIZER_VERBOSE, "Reading utf STRING");
+    }
+    return in.readUTF();
+  }
+
+  private static String readStringBytesFromDataInput(DataInput dataInput, int len)
+      throws IOException {
+    if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
+      logger.trace(LogMarker.SERIALIZER_VERBOSE, "Reading STRING_BYTES of len={}", len);
+    }
+    byte[] buf = new byte[len];
+    dataInput.readFully(buf, 0, len);
+    return new String(buf, 0); // intentionally using deprecated constructor
+  }
+
   public static String readString(DataInput in, byte header) throws IOException {
     return readString(in, DscodeHelper.toDSCODE(header));
   }
 
-  private static DataSerializer dvddeserializer;
-
   // TODO: registerDVDDeserializer is unused
   public static void registerDVDDeserializer(DataSerializer dvddeslzr) {
     dvddeserializer = dvddeslzr;
@@ -2872,13 +2772,13 @@ public abstract class InternalDataSerializer extends DataSerializer {
       case NULL_STRING:
         return null;
       case STRING:
-        return readString(in, headerDSCode);
+        return readStringUTFFromDataInput(in);
       case HUGE_STRING:
-        return readString(in, headerDSCode);
+        return readHugeStringFromDataInput(in);
       case STRING_BYTES:
-        return readString(in, headerDSCode);
+        return readStringBytesFromDataInput(in, in.readUnsignedShort());
       case HUGE_STRING_BYTES:
-        return readString(in, headerDSCode);
+        return readStringBytesFromDataInput(in, in.readInt());
       case CLASS:
         return readClass(in);
       case DATE:
@@ -3036,7 +2936,7 @@ public abstract class InternalDataSerializer extends DataSerializer {
       serializationFilter.setFilterOn((ObjectInputStream) ois);
       if (stream instanceof VersionedDataStream) {
         Version v = ((VersionedDataStream) stream).getVersion();
-        if (v != null && v != Version.CURRENT) {
+        if (Version.CURRENT != v && v != null) {
           ois = new VersionedObjectInput(ois, v);
         }
       }
@@ -3088,8 +2988,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-  private static final ThreadLocal<Boolean> pdxSerializationInProgress = new ThreadLocal<>();
-
   public static boolean isPdxSerializationInProgress() {
     Boolean v = pdxSerializationInProgress.get();
     return v != null && v;
@@ -3288,149 +3186,458 @@ public abstract class InternalDataSerializer extends DataSerializer {
     return supportedClassesToHolders;
   }
 
+  public static void writeObjectArray(Object[] array, DataOutput out, boolean ensureCompatibility)
+      throws IOException {
+    InternalDataSerializer.checkOut(out);
+    int length = -1;
+    if (array != null) {
+      length = array.length;
+    }
+    InternalDataSerializer.writeArrayLength(length, out);
+    if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
+      logger.trace(LogMarker.SERIALIZER_VERBOSE, "Writing Object array of length {}", length);
+    }
+    if (length >= 0) {
+      writeClass(array.getClass().getComponentType(), out);
+      for (int i = 0; i < length; i++) {
+        basicWriteObject(array[i], out, ensureCompatibility);
+      }
+    }
+  }
+
   /**
-   * A marker object for {@code DataSerializer}s that have not been registered. Using this marker
-   * object allows us to asynchronously send {@code DataSerializer} registration updates. If the
-   * serialized bytes arrive at a VM before the registration message does, the deserializer will
-   * wait an amount of time for the registration message to arrive.
+   * Write a variable length long the old way (pre 7.0). Use this only in contexts where you might
+   * need to communicate with pre 7.0 members or files.
    */
-  abstract static class Marker {
-    /**
-     * The DataSerializer that is filled in upon registration
-     */
-    protected DataSerializer serializer = null;
-
-    /**
-     * set to true once setSerializer is called.
-     */
-    boolean hasBeenSet = false;
-
-    abstract DataSerializer getSerializer();
+  public static void writeVLOld(long data, DataOutput out) throws IOException {
+    if (data < 0) {
+      Assert.fail("Data expected to be >=0 is " + data);
+    }
+    if (data <= MAX_BYTE_VL) {
+      out.writeByte((byte) data);
+    } else if (data <= 0x7FFF) {
+      // set the sign bit to indicate a short
+      out.write(((int) data >>> 8 | 0x80) & 0xFF);
+      out.write((int) data >>> 0 & 0xFF);
+    } else if (data <= Integer.MAX_VALUE) {
+      out.writeByte(INT_VL);
+      out.writeInt((int) data);
+    } else {
+      out.writeByte(LONG_VL);
+      out.writeLong(data);
+    }
+  }
 
-    /**
-     * Sets the serializer associated with this marker. It will notify any threads that are waiting
-     * for the serializer to be registered.
-     */
-    void setSerializer(DataSerializer serializer) {
-      synchronized (this) {
-        this.hasBeenSet = true;
-        this.serializer = serializer;
-        this.notifyAll();
-      }
+  /**
+   * Write a variable length long the old way (pre 7.0). Use this only in contexts where you might
+   * need to communicate with pre 7.0 members or files.
+   */
+  public static long readVLOld(DataInput in) throws IOException {
+    byte code = in.readByte();
+    long result;
+    if (code < 0) {
+      // mask off sign bit
+      result = code & 0x7F;
+      result <<= 8;
+      result |= in.readByte() & 0xFF;
+    } else if (code <= MAX_BYTE_VL) {
+      result = code;
+    } else if (code == INT_VL) {
+      result = in.readInt();
+    } else if (code == LONG_VL) {
+      result = in.readLong();
+    } else {
+      throw new IllegalStateException("unexpected variable length code=" + code);
     }
+    return result;
   }
 
   /**
-   * A marker object for {@code DataSerializer}s that have not been registered. Using this marker
-   * object allows us to asynchronously send {@code DataSerializer} registration updates. If the
-   * serialized bytes arrive at a VM before the registration message does, the deserializer will
-   * wait an amount of time for the registration message to arrive. Made public for unit test
-   * access.
+   * Encode a long as a variable length array.
    *
-   * @since GemFire 5.7
+   * This method is appropriate for unsigned integers. For signed integers, negative values will
+   * always consume 10 bytes, so it is recommended to use writeSignedVL instead.
+   *
+   * This is taken from the varint encoding in protobufs (BSD licensed). See
+   * https://developers.google.com/protocol-buffers/docs/encoding
    */
-  public static class GetMarker extends Marker {
-    /**
-     * Number of milliseconds to wait. Also used by InternalInstantiator. Note that some tests set
-     * this to a small amount to speed up failures. Made public for unit test access.
-     */
-    public static int WAIT_MS = Integer.getInteger(
-        DistributionConfig.GEMFIRE_PREFIX + "InternalDataSerializer.WAIT_MS", 60 * 1000);
-
-    /**
-     * Returns the serializer associated with this marker. If the serializer has not been registered
-     * yet, then this method will wait until the serializer is registered. If this method has to
-     * wait for too long, then {@code null} is returned.
-     */
-    @Override
-    DataSerializer getSerializer() {
-      synchronized (this) {
-        boolean firstTime = true;
-        long endTime = 0;
-        while (!this.hasBeenSet) {
-          if (firstTime) {
-            firstTime = false;
-            endTime = System.currentTimeMillis() + WAIT_MS;
-          }
-          try {
-            long remainingMs = endTime - System.currentTimeMillis();
-            if (remainingMs > 0) {
-              this.wait(remainingMs); // spurious wakeup ok
-            } else {
-              // timed out call setSerializer just to make sure that anyone else
-              // also waiting on this marker times out also
-              setSerializer(null);
-              break;
-            }
-          } catch (InterruptedException ignored) {
-            Thread.currentThread().interrupt();
-            // Just return null, let it fail
-            return null;
-          }
-        }
-        return this.serializer;
+  public static void writeUnsignedVL(long data, DataOutput out) throws IOException {
+    while (true) {
+      if ((data & ~0x7FL) == 0) {
+        out.writeByte((int) data);
+        return;
+      } else {
+        out.writeByte((int) data & 0x7F | 0x80);
+        data >>>= 7;
       }
     }
   }
 
   /**
-   * A marker object for {@code DataSerializer}s that is in the process of being registered. It is
-   * possible for getSerializer to return {@code null}
+   * Decode a long as a variable length array.
    *
-   * @since GemFire 5.7
+   * This is taken from the varint encoding in protobufs (BSD licensed). See
+   * https://developers.google.com/protocol-buffers/docs/encoding
    */
-  static class InitMarker extends Marker {
-    /*
-     * Returns the serializer associated with this marker. If the serializer has not been registered
-     * yet, then this method will wait until the serializer is registered. If this method has to
-     * wait for too long, then {@code null} is returned.
-     */
-
-    /**
-     * Returns the serializer associated with this marker. Waits forever (unless interrupted) for it
-     * to be initialized. Returns null if this Marker failed to initialize.
-     */
-    @Override
-    DataSerializer getSerializer() {
-      synchronized (this) {
-        while (!this.hasBeenSet) {
-          try {
-            this.wait(); // spurious wakeup ok
-          } catch (InterruptedException ignored) {
-            Thread.currentThread().interrupt();
-            // Just return null, let it fail
-            return null;
-          }
-        }
-        return this.serializer;
+  public static long readUnsignedVL(DataInput in) throws IOException {
+    int shift = 0;
+    long result = 0;
+    while (shift < 64) {
+      final byte b = in.readByte();
+      result |= (long) (b & 0x7F) << shift;
+      if ((b & 0x80) == 0) {
+        return result;
       }
+      shift += 7;
     }
+    throw new GemFireIOException("Malformed variable length integer");
   }
 
   /**
-   * A distribution message that alerts other members of the distributed cache of a new {@code
-   * DataSerializer} being registered.
-   */
-  public static class RegistrationMessage extends SerialDistributionMessage {
+   * Encode a signed long as a variable length array.
+   *
+   * This method is appropriate for signed integers. It uses zig zag encoding to so that negative
+   * numbers will be represented more compactly. For unsigned values, writeUnsignedVL will be more
+   * efficient.
+   */
+  public static void writeSignedVL(long data, DataOutput out) throws IOException {
+    writeUnsignedVL(encodeZigZag64(data), out);
+  }
+
+  /**
+   * Decode a signed long as a variable length array.
+   *
+   * This method is appropriate for signed integers. It uses zig zag encoding to so that negative
+   * numbers will be represented more compactly. For unsigned values, writeUnsignedVL will be more
+   * efficient.
+   */
+  public static long readSignedVL(DataInput in) throws IOException {
+    return decodeZigZag64(readUnsignedVL(in));
+  }
+
+  /**
+   * Decode a ZigZag-encoded 64-bit value. ZigZag encodes signed integers into values that can be
+   * efficiently encoded with varint. (Otherwise, negative values must be sign-extended to 64 bits
+   * to be varint encoded, thus always taking 10 bytes on the wire.)
+   *
+   * @param n An unsigned 64-bit integer, stored in a signed int because Java has no explicit
+   *        unsigned support.
+   * @return A signed 64-bit integer.
+   *
+   *         This is taken from the varint encoding in protobufs (BSD licensed). See
+   *         https://developers.google.com/protocol-buffers/docs/encoding
+   */
+  private static long decodeZigZag64(final long n) {
+    return n >>> 1 ^ -(n & 1);
+  }
+
+  /**
+   * Encode a ZigZag-encoded 64-bit value. ZigZag encodes signed integers into values that can be
+   * efficiently encoded with varint. (Otherwise, negative values must be sign-extended to 64 bits
+   * to be varint encoded, thus always taking 10 bytes on the wire.)
+   *
+   * @param n A signed 64-bit integer.
+   * @return An unsigned 64-bit integer, stored in a signed int because Java has no explicit
+   *         unsigned support.
+   *
+   *         This is taken from the varint encoding in protobufs (BSD licensed). See
+   *         https://developers.google.com/protocol-buffers/docs/encoding
+   */
+  private static long encodeZigZag64(final long n) {
+    // Note: the right-shift must be arithmetic
+    return n << 1 ^ n >> 63;
+  }
+
+  /* test only method */
+  public static int calculateBytesForTSandDSID(int dsid) {
+    HeapDataOutputStream out = new HeapDataOutputStream(4 + 8, Version.CURRENT);
+    long now = System.currentTimeMillis();
+    try {
+      writeUnsignedVL(now, out);
+      writeUnsignedVL(InternalDataSerializer.encodeZigZag64(dsid), out);
+    } catch (IOException ignored) {
+      return 0;
+    }
+    return out.size();
+  }
+
+  public static Class<?> getCachedClass(String p_className) throws ClassNotFoundException {
+    String className = processIncomingClassName(p_className);
+    if (LOAD_CLASS_EACH_TIME) {
+      return ClassPathLoader.getLatest().forName(className);
+    } else {
+      Class<?> result = getExistingCachedClass(className);
+      if (result == null) {
+        // Do the forName call outside the sync to fix bug 46172
+        result = ClassPathLoader.getLatest().forName(className);
+        synchronized (cacheAccessLock) {
+          Class<?> cachedClass = getExistingCachedClass(className);
+          if (cachedClass == null) {
+            classCache.put(className, new WeakReference<>(result));
+          } else {
+            result = cachedClass;
+          }
+        }
+      }
+      return result;
+    }
+  }
+
+  private static Class<?> getExistingCachedClass(String className) {
+    WeakReference<Class<?>> wr = classCache.get(className);
+    Class<?> result = null;
+    if (wr != null) {
+      result = wr.get();
+    }
+    return result;
+  }
+
+  public static void flushClassCache() {
+    if (classCache != null) {
+      // Not locking classCache during clear as doing so causes a deadlock in the DeployedJar
+      classCache.clear();
+    }
+  }
+
+  /**
+   * Serializes a list of Integers. The argument may be null. Deserialize with
+   * readListOfIntegers().
+   *
+   * TODO: writeListOfIntegers is unused
+   */
+  public void writeListOfIntegers(List<Integer> list, DataOutput out) throws IOException {
+    int size = -1;
+    if (list != null) {
+      size = list.size();
+    }
+    InternalDataSerializer.writeArrayLength(size, out);
+    for (int i = 0; i < size; i++) {
+      out.writeInt(list.get(i));
+    }
+  }
+
+  /**
+   * Any time new serialization format is added then a new enum needs to be added here.
+   *
+   * @since GemFire 6.6.2
+   */
+  private enum SERIALIZATION_VERSION {
+    vINVALID,
+    // includes 6.6.0.x and 6.6.1.x. Note that no serialization changes were made in 6.6 until 6.6.2
+    v660,
+    // 6.6.2.x or later NOTE if you add a new constant make sure and update "latestVersion".
+    v662
+  }
+
+  /**
+   * A listener whose listener methods are invoked when {@link DataSerializer}s and {@link
+   * Instantiator}s are registered. This is part of the fix for bug 31422.
+   *
+   * @see InternalDataSerializer#addRegistrationListener
+   * @see InternalDataSerializer#removeRegistrationListener
+   */
+  public interface RegistrationListener {
+
     /**
-     * The id of the {@code DataSerializer} that was registered since 5.7 an int instead of a byte
+     * Invoked when a new {@code Instantiator} is {@linkplain Instantiator#register(Instantiator)
+     * registered}.
      */
-    private int id;
+    void newInstantiator(Instantiator instantiator);
 
     /**
-     * The eventId of the {@code DataSerializer} that was registered
+     * Invoked when a new {@code DataSerializer} is {@linkplain DataSerializer#register(Class)
+     * registered}.
      */
-    protected EventID eventId;
+    void newDataSerializer(DataSerializer ds);
+  }
+
+  /**
+   * A SerializerAttributesHolder holds information required to load a DataSerializer and exists to
+   * allow client/server connections to be created more quickly than they would if the
+   * DataSerializer information downloaded from the server were used to immediately load the
+   * corresponding classes.
+   */
+  public static class SerializerAttributesHolder {
+    private String className = "";
+    private EventID eventId = null;
+    private ClientProxyMembershipID proxyId = null;
+    private int id = 0;
+
+    SerializerAttributesHolder() {}
+
+    SerializerAttributesHolder(String name, EventID event, ClientProxyMembershipID proxy, int id) {
+      this.className = name;
+      this.eventId = event;
+      this.proxyId = proxy;
+      this.id = id;
+    }
 
     /**
-     * The name of the {@code DataSerializer} class
+     * @return String the classname of the data serializer this instance represents.
      */
-    private String className;
+    public String getClassName() {
+      return this.className;
+    }
+
+    public EventID getEventId() {
+      return this.eventId;
+    }
+
+    public ClientProxyMembershipID getProxyId() {
+      return this.proxyId;
+    }
+
+    public int getId() {
+      return this.id;
+    }
+
+    @Override
+    public String toString() {
+      return "SerializerAttributesHolder[name=" + this.className + ",id=" + this.id + ",eventId="
+          + this.eventId + ']';
+    }
+  }
+
+  /**
+   * A marker object for {@code DataSerializer}s that have not been registered. Using this marker
+   * object allows us to asynchronously send {@code DataSerializer} registration updates. If the
+   * serialized bytes arrive at a VM before the registration message does, the deserializer will
+   * wait an amount of time for the registration message to arrive.
+   */
+  abstract static class Marker {
+    /**
+     * The DataSerializer that is filled in upon registration
+     */
+    protected DataSerializer serializer = null;
+
+    /**
+     * set to true once setSerializer is called.
+     */
+    boolean hasBeenSet = false;
+
+    abstract DataSerializer getSerializer();
+
+    /**
+     * Sets the serializer associated with this marker. It will notify any threads that are waiting
+     * for the serializer to be registered.
+     */
+    void setSerializer(DataSerializer serializer) {
+      synchronized (this) {
+        this.hasBeenSet = true;
+        this.serializer = serializer;
+        this.notifyAll();
+      }
+    }
+  }
+
+  /**
+   * A marker object for {@code DataSerializer}s that have not been registered. Using this marker
+   * object allows us to asynchronously send {@code DataSerializer} registration updates. If the
+   * serialized bytes arrive at a VM before the registration message does, the deserializer will
+   * wait an amount of time for the registration message to arrive. Made public for unit test
+   * access.
+   *
+   * @since GemFire 5.7
+   */
+  public static class GetMarker extends Marker {
+    /**
+     * Number of milliseconds to wait. Also used by InternalInstantiator. Note that some tests set
+     * this to a small amount to speed up failures. Made public for unit test access.
+     */
+    public static int WAIT_MS = Integer.getInteger(
+        DistributionConfig.GEMFIRE_PREFIX + "InternalDataSerializer.WAIT_MS", 60 * 1000);
+
+    /**
+     * Returns the serializer associated with this marker. If the serializer has not been registered
+     * yet, then this method will wait until the serializer is registered. If this method has to
+     * wait for too long, then {@code null} is returned.
+     */
+    @Override
+    DataSerializer getSerializer() {
+      synchronized (this) {
+        boolean firstTime = true;
+        long endTime = 0;
+        while (!this.hasBeenSet) {
+          if (firstTime) {
+            firstTime = false;
+            endTime = System.currentTimeMillis() + WAIT_MS;
+          }
+          try {
+            long remainingMs = endTime - System.currentTimeMillis();
+            if (remainingMs > 0) {
+              this.wait(remainingMs); // spurious wakeup ok
+            } else {
+              // timed out call setSerializer just to make sure that anyone else
+              // also waiting on this marker times out also
+              setSerializer(null);
+              break;
+            }
+          } catch (InterruptedException ignored) {
+            Thread.currentThread().interrupt();
+            // Just return null, let it fail
+            return null;
+          }
+        }
+        return this.serializer;
+      }
+    }
+  }
 
+  /**
+   * A marker object for {@code DataSerializer}s that is in the process of being registered. It is
+   * possible for getSerializer to return {@code null}
+   *
+   * @since GemFire 5.7
+   */
+  static class InitMarker extends Marker {
+    /*
+     * Returns the serializer associated with this marker. If the serializer has not been registered
+     * yet, then this method will wait until the serializer is registered. If this method has to
+     * wait for too long, then {@code null} is returned.
+     */
+
+    /**
+     * Returns the serializer associated with this marker. Waits forever (unless interrupted) for it
+     * to be initialized. Returns null if this Marker failed to initialize.
+     */
+    @Override
+    DataSerializer getSerializer() {
+      synchronized (this) {
+        while (!this.hasBeenSet) {
+          try {
+            this.wait(); // spurious wakeup ok
+          } catch (InterruptedException ignored) {
+            Thread.currentThread().interrupt();
+            // Just return null, let it fail
+            return null;
+          }
+        }
+        return this.serializer;
+      }
+    }
+  }
+
+  /**
+   * A distribution message that alerts other members of the distributed cache of a new {@code
+   * DataSerializer} being registered.
+   */
+  public static class RegistrationMessage extends SerialDistributionMessage {
+    /**
+     * The versions in which this message was modified
+     */
+    private static final Version[] dsfidVersions = new Version[] {};
+    /**
+     * The eventId of the {@code DataSerializer} that was registered
+     */
+    protected EventID eventId;
+    /**
+     * The id of the {@code DataSerializer} that was registered since 5.7 an int instead of a byte
+     */
+    private int id;
     /**
-     * The versions in which this message was modified
+     * The name of the {@code DataSerializer} class
      */
-    private static final Version[] dsfidVersions = new Version[] {};
+    private String className;
 
     /**
      * Constructor for {@code DataSerializable}
@@ -3553,28 +3760,6 @@ public abstract class InternalDataSerializer extends DataSerializer {
   }
 
   /**
-   * A listener whose listener methods are invoked when {@link DataSerializer}s and {@link
-   * Instantiator}s are registered. This is part of the fix for bug 31422.
-   *
-   * @see InternalDataSerializer#addRegistrationListener
-   * @see InternalDataSerializer#removeRegistrationListener
-   */
-  public interface RegistrationListener {
-
-    /**
-     * Invoked when a new {@code Instantiator} is {@linkplain Instantiator#register(Instantiator)
-     * registered}.
-     */
-    void newInstantiator(Instantiator instantiator);
-
-    /**
-     * Invoked when a new {@code DataSerializer} is {@linkplain DataSerializer#register(Class)
-     * registered}.
-     */
-    void newDataSerializer(DataSerializer ds);
-  }
-
-  /**
    * An {@code ObjectInputStream} whose {@link #resolveClass} method loads classes from the current
    * context class loader.
    */
@@ -3677,235 +3862,4 @@ public abstract class InternalDataSerializer extends DataSerializer {
   protected abstract static class WellKnownPdxDS extends WellKnownDS {
     // subclasses need to implement toData
   }
-
-  public static void writeObjectArray(Object[] array, DataOutput out, boolean ensureCompatibility)
-      throws IOException {
-    InternalDataSerializer.checkOut(out);
-    int length;
-    if (array == null) {
-      length = -1;
-    } else {
-      length = array.length;
-    }
-    InternalDataSerializer.writeArrayLength(length, out);
-    if (logger.isTraceEnabled(LogMarker.SERIALIZER_VERBOSE)) {
-      logger.trace(LogMarker.SERIALIZER_VERBOSE, "Writing Object array of length {}", length);
-    }
-    if (length >= 0) {
-      writeClass(array.getClass().getComponentType(), out);
-      for (int i = 0; i < length; i++) {
-        basicWriteObject(array[i], out, ensureCompatibility);
-      }
-    }
-  }
-
-  // Variable Length long encoded as int in next 4 bytes
-  private static final byte INT_VL = 126;
-
-  // Variable Length long encoded as long in next 8 bytes
-  private static final byte LONG_VL = 127;
-
-  private static final int MAX_BYTE_VL = 125;
-
-  /**
-   * Write a variable length long the old way (pre 7.0). Use this only in contexts where you might
-   * need to communicate with pre 7.0 members or files.
-   */
-  public static void writeVLOld(long data, DataOutput out) throws IOException {
-    if (data < 0) {
-      Assert.fail("Data expected to be >=0 is " + data);
-    }
-    if (data <= MAX_BYTE_VL) {
-      out.writeByte((byte) data);
-    } else if (data <= 0x7FFF) {
-      // set the sign bit to indicate a short
-      out.write(((int) data >>> 8 | 0x80) & 0xFF);
-      out.write((int) data >>> 0 & 0xFF);
-    } else if (data <= Integer.MAX_VALUE) {
-      out.writeByte(INT_VL);
-      out.writeInt((int) data);
-    } else {
-      out.writeByte(LONG_VL);
-      out.writeLong(data);
-    }
-  }
-
-  /**
-   * Write a variable length long the old way (pre 7.0). Use this only in contexts where you might
-   * need to communicate with pre 7.0 members or files.
-   */
-  public static long readVLOld(DataInput in) throws IOException {
-    byte code = in.readByte();
-    long result;
-    if (code < 0) {
-      // mask off sign bit
-      result = code & 0x7F;
-      result <<= 8;
-      result |= in.readByte() & 0xFF;
-    } else if (code <= MAX_BYTE_VL) {
-      result = code;
-    } else if (code == INT_VL) {
-      result = in.readInt();
-    } else if (code == LONG_VL) {
-      result = in.readLong();
-    } else {
-      throw new IllegalStateException("unexpected variable length code=" + code);
-    }
-    return result;
-  }
-
-  /**
-   * Encode a long as a variable length array.
-   *
-   * This method is appropriate for unsigned integers. For signed integers, negative values will
-   * always consume 10 bytes, so it is recommended to use writeSignedVL instead.
-   *
-   * This is taken from the varint encoding in protobufs (BSD licensed). See
-   * https://developers.google.com/protocol-buffers/docs/encoding
-   */
-  public static void writeUnsignedVL(long data, DataOutput out) throws IOException {
-    while (true) {
-      if ((data & ~0x7FL) == 0) {
-        out.writeByte((int) data);
-        return;
-      } else {
-        out.writeByte((int) data & 0x7F | 0x80);
-        data >>>= 7;
-      }
-    }
-  }
-
-  /**
-   * Decode a long as a variable length array.
-   *
-   * This is taken from the varint encoding in protobufs (BSD licensed). See
-   * https://developers.google.com/protocol-buffers/docs/encoding
-   */
-  public static long readUnsignedVL(DataInput in) throws IOException {
-    int shift = 0;
-    long result = 0;
-    while (shift < 64) {
-      final byte b = in.readByte();
-      result |= (long) (b & 0x7F) << shift;
-      if ((b & 0x80) == 0) {
-        return result;
-      }
-      shift += 7;
-    }
-    throw new GemFireIOException("Malformed variable length integer");
-  }
-
-  /**
-   * Encode a signed long as a variable length array.
-   *
-   * This method is appropriate for signed integers. It uses zig zag encoding to so that negative
-   * numbers will be represented more compactly. For unsigned values, writeUnsignedVL will be more
-   * efficient.
-   */
-  public static void writeSignedVL(long data, DataOutput out) throws IOException {
-    writeUnsignedVL(encodeZigZag64(data), out);
-  }
-
-  /**
-   * Decode a signed long as a variable length array.
-   *
-   * This method is appropriate for signed integers. It uses zig zag encoding to so that negative
-   * numbers will be represented more compactly. For unsigned values, writeUnsignedVL will be more
-   * efficient.
-   */
-  public static long readSignedVL(DataInput in) throws IOException {
-    return decodeZigZag64(readUnsignedVL(in));
-  }
-
-  /**
-   * Decode a ZigZag-encoded 64-bit value. ZigZag encodes signed integers into values that can be
-   * efficiently encoded with varint. (Otherwise, negative values must be sign-extended to 64 bits
-   * to be varint encoded, thus always taking 10 bytes on the wire.)
-   *
-   * @param n An unsigned 64-bit integer, stored in a signed int because Java has no explicit
-   *        unsigned support.
-   * @return A signed 64-bit integer.
-   *
-   *         This is taken from the varint encoding in protobufs (BSD licensed). See
-   *         https://developers.google.com/protocol-buffers/docs/encoding
-   */
-  private static long decodeZigZag64(final long n) {
-    return n >>> 1 ^ -(n & 1);
-  }
-
-  /**
-   * Encode a ZigZag-encoded 64-bit value. ZigZag encodes signed integers into values that can be
-   * efficiently encoded with varint. (Otherwise, negative values must be sign-extended to 64 bits
-   * to be varint encoded, thus always taking 10 bytes on the wire.)
-   *
-   * @param n A signed 64-bit integer.
-   * @return An unsigned 64-bit integer, stored in a signed int because Java has no explicit
-   *         unsigned support.
-   *
-   *         This is taken from the varint encoding in protobufs (BSD licensed). See
-   *         https://developers.google.com/protocol-buffers/docs/encoding
-   */
-  private static long encodeZigZag64(final long n) {
-    // Note: the right-shift must be arithmetic
-    return n << 1 ^ n >> 63;
-  }
-
-  /* test only method */
-  public static int calculateBytesForTSandDSID(int dsid) {
-    HeapDataOutputStream out = new HeapDataOutputStream(4 + 8, Version.CURRENT);
-    long now = System.currentTimeMillis();
-    try {
-      writeUnsignedVL(now, out);
-      writeUnsignedVL(InternalDataSerializer.encodeZigZag64(dsid), out);
-    } catch (IOException ignored) {
-      return 0;
-    }
-    return out.size();
-  }
-
-  public static final boolean LOAD_CLASS_EACH_TIME =
-      Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "loadClassOnEveryDeserialization");
-
-  private static final CopyOnWriteHashMap<String, WeakReference<Class<?>>> classCache =
-      LOAD_CLASS_EACH_TIME ? null : new CopyOnWriteHashMap<>();
-
-  private static final Object cacheAccessLock = new Object();
-
-  public static Class<?> getCachedClass(String p_className) throws ClassNotFoundException {
-    String className = processIncomingClassName(p_className);
-    if (LOAD_CLASS_EACH_TIME) {
-      return ClassPathLoader.getLatest().forName(className);
-    } else {
-      Class<?> result = getExistingCachedClass(className);
-      if (result == null) {
-        // Do the forName call outside the sync to fix bug 46172
-        result = ClassPathLoader.getLatest().forName(className);
-        synchronized (cacheAccessLock) {
-          Class<?> cachedClass = getExistingCachedClass(className);
-          if (cachedClass == null) {
-            classCache.put(className, new WeakReference<>(result));
-          } else {
-            result = cachedClass;
-          }
-        }
-      }
-      return result;
-    }
-  }
-
-  private static Class<?> getExistingCachedClass(String className) {
-    WeakReference<Class<?>> wr = classCache.get(className);
-    Class<?> result = null;
-    if (wr != null) {
-      result = wr.get();
-    }
-    return result;
-  }
-
-  public static void flushClassCache() {
-    if (classCache != null) {
-      // Not locking classCache during clear as doing so causes a deadlock in the DeployedJar
-      classCache.clear();
-    }
-  }
 }
diff --git a/geode-core/src/test/java/org/apache/geode/internal/DSCODETest.java b/geode-core/src/test/java/org/apache/geode/internal/DSCODETest.java
index d84b1ce..d4007bb 100644
--- a/geode-core/src/test/java/org/apache/geode/internal/DSCODETest.java
+++ b/geode-core/src/test/java/org/apache/geode/internal/DSCODETest.java
@@ -42,7 +42,8 @@ public class DSCODETest {
 
   @Test
   public void testGetEnumFromByte() {
-    Arrays.stream(DSCODE.values()).filter(dscode -> dscode != DSCODE.RESERVED_FOR_FUTURE_USE && dscode != DSCODE.ILLEGAL)
+    Arrays.stream(DSCODE.values())
+        .filter(dscode -> dscode != DSCODE.RESERVED_FOR_FUTURE_USE && dscode != DSCODE.ILLEGAL)
         .forEach(dscode -> Assert.assertEquals(dscode, DscodeHelper.toDSCODE(dscode.toByte())));
   }
 }