You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by jb...@apache.org on 2021/08/17 18:17:39 UTC

[geode] 07/18: GEODE-6588: Cleanup VersionedObjectList

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

jbarrett pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git

commit 8e097c3c94eca6f917dd94f9e218905132a7fe6b
Author: Jacob Barrett <jb...@pivotal.io>
AuthorDate: Thu May 20 15:06:34 2021 -0700

    GEODE-6588: Cleanup VersionedObjectList
---
 .../cache/tier/sockets/ObjectPartList.java         | 117 ++++++-----
 .../cache/tier/sockets/VersionedObjectList.java    | 220 +++++++++------------
 2 files changed, 155 insertions(+), 182 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ObjectPartList.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ObjectPartList.java
index 9a710e3..fe5945b 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ObjectPartList.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/ObjectPartList.java
@@ -22,9 +22,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.logging.log4j.Logger;
-
 import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
 import org.apache.geode.internal.cache.versions.VersionTag;
 import org.apache.geode.internal.offheap.OffHeapHelper;
 import org.apache.geode.internal.offheap.Releasable;
@@ -32,7 +31,6 @@ import org.apache.geode.internal.serialization.DataSerializableFixedID;
 import org.apache.geode.internal.serialization.DeserializationContext;
 import org.apache.geode.internal.serialization.KnownVersion;
 import org.apache.geode.internal.serialization.SerializationContext;
-import org.apache.geode.logging.internal.log4j.api.LogService;
 
 /**
  * Encapsulates list containing objects, serialized objects, raw byte arrays, or exceptions. It can
@@ -42,8 +40,6 @@ import org.apache.geode.logging.internal.log4j.api.LogService;
  * @since GemFire 5.7
  */
 public class ObjectPartList implements DataSerializableFixedID, Releasable {
-  private static final Logger logger = LogService.getLogger();
-
   protected static final byte BYTES = 0;
 
   protected static final byte OBJECT = 1;
@@ -56,33 +52,33 @@ public class ObjectPartList implements DataSerializableFixedID, Releasable {
 
   protected boolean hasKeys;
 
-  protected List keys;
+  protected List<Object> keys;
 
-  protected List objects;
+  protected List<Object> objects;
 
   public void addPart(Object key, Object value, byte objectType, VersionTag versionTag) {
-    int size = this.objects.size();
-    int maxSize = this.objectTypeArray.length;
+    int size = objects.size();
+    int maxSize = objectTypeArray.length;
     if (size >= maxSize) {
       throw new IndexOutOfBoundsException("Cannot add object part beyond " + maxSize + " elements");
     }
-    if (this.hasKeys) {
+    if (hasKeys) {
       if (key == null) {
         throw new IllegalArgumentException("Cannot add null key");
       }
-      this.keys.add(key);
+      keys.add(key);
     }
-    this.objectTypeArray[size] = objectType;
-    this.objects.add(value);
+    objectTypeArray[size] = objectType;
+    objects.add(value);
   }
 
   // public methods
 
   public ObjectPartList() {
-    this.objectTypeArray = null;
-    this.hasKeys = false;
-    this.keys = null;
-    this.objects = new ArrayList();
+    objectTypeArray = null;
+    hasKeys = false;
+    keys = null;
+    objects = new ArrayList<>();
   }
 
   public ObjectPartList(int maxSize, boolean hasKeys) {
@@ -90,14 +86,14 @@ public class ObjectPartList implements DataSerializableFixedID, Releasable {
       throw new IllegalArgumentException(
           "Invalid size " + maxSize + " to ObjectPartList constructor");
     }
-    this.objectTypeArray = new byte[maxSize];
+    objectTypeArray = new byte[maxSize];
     this.hasKeys = hasKeys;
     if (hasKeys) {
-      this.keys = new ArrayList();
+      keys = new ArrayList<>();
     } else {
-      this.keys = null;
+      keys = null;
     }
-    this.objects = new ArrayList();
+    objects = new ArrayList<>();
   }
 
   public void addObjectPart(Object key, Object value, boolean isObject, VersionTag versionTag) {
@@ -116,54 +112,55 @@ public class ObjectPartList implements DataSerializableFixedID, Releasable {
 
 
   public void addAll(ObjectPartList other) {
-    if (this.hasKeys) {
+    if (hasKeys) {
       if (other.keys != null) {
-        if (this.keys == null) {
-          this.keys = new ArrayList(other.keys);
+        if (keys == null) {
+          keys = new ArrayList<>(other.keys);
         } else {
-          this.keys.addAll(other.keys);
+          keys.addAll(other.keys);
         }
       }
     } else if (other.hasKeys) {
-      this.hasKeys = true;
-      this.keys = new ArrayList(other.keys);
+      hasKeys = true;
+      keys = new ArrayList<>(other.keys);
     }
-    this.objects.addAll(other.objects);
+    objects.addAll(other.objects);
   }
 
   public List<Object> getKeys() {
-    if (this.keys == null) {
+    if (keys == null) {
       return Collections.emptyList();
     } else {
-      return Collections.unmodifiableList(this.keys);
+      return Collections.unmodifiableList(keys);
     }
   }
 
   /** unprotected access to the keys collection, which may be null */
-  public List getKeysForTest() {
-    return this.keys;
+  @VisibleForTesting
+  List<Object> getKeysForTest() {
+    return keys;
   }
 
-  public List getObjects() {
-    if (this.objects == null) {
+  public List<Object> getObjects() {
+    if (objects == null) {
       return Collections.emptyList();
     } else {
-      return Collections.unmodifiableList(this.objects);
+      return Collections.unmodifiableList(objects);
     }
   }
 
   /** unprotected access to the objects collection, which may be null */
-  public List getObjectsForTest() {
-    return this.objects;
+  public List<Object> getObjectsForTest() {
+    return objects;
   }
 
   public int size() {
     // some lists have only keys and some have only objects, so we need to choose
     // the correct collection to query
-    if (this.hasKeys) {
-      return this.keys.size();
+    if (hasKeys) {
+      return keys.size();
     } else {
-      return this.objects.size();
+      return objects.size();
     }
   }
 
@@ -171,31 +168,31 @@ public class ObjectPartList implements DataSerializableFixedID, Releasable {
     if (maxSize <= 0) {
       throw new IllegalArgumentException("Invalid size " + maxSize + " to ObjectPartList.reinit");
     }
-    this.objectTypeArray = new byte[maxSize];
-    this.objects.clear();
-    this.keys.clear();
+    objectTypeArray = new byte[maxSize];
+    objects.clear();
+    keys.clear();
   }
 
   public void clear() {
     release();
-    this.objects.clear();
-    if (this.keys != null) {
-      this.keys.clear();
+    objects.clear();
+    if (keys != null) {
+      keys.clear();
     }
   }
 
   @Override
   public void toData(DataOutput out,
       SerializationContext context) throws IOException {
-    out.writeBoolean(this.hasKeys);
-    if (this.objectTypeArray != null) {
-      int numObjects = this.objects.size();
+    out.writeBoolean(hasKeys);
+    if (objectTypeArray != null) {
+      int numObjects = objects.size();
       out.writeInt(numObjects);
       for (int index = 0; index < numObjects; ++index) {
-        Object value = this.objects.get(index);
-        byte objectType = this.objectTypeArray[index];
-        if (this.hasKeys) {
-          context.getSerializer().writeObject(this.keys.get(index), out);
+        Object value = objects.get(index);
+        byte objectType = objectTypeArray[index];
+        if (hasKeys) {
+          context.getSerializer().writeObject(keys.get(index), out);
         }
         out.writeBoolean(objectType == EXCEPTION);
         if (objectType == OBJECT && value instanceof byte[]) {
@@ -217,16 +214,16 @@ public class ObjectPartList implements DataSerializableFixedID, Releasable {
   @Override
   public void fromData(DataInput in,
       DeserializationContext context) throws IOException, ClassNotFoundException {
-    this.hasKeys = in.readBoolean();
-    if (this.hasKeys) {
-      this.keys = new ArrayList();
+    hasKeys = in.readBoolean();
+    if (hasKeys) {
+      keys = new ArrayList<>();
     }
     int numObjects = in.readInt();
     if (numObjects > 0) {
       for (int index = 0; index < numObjects; ++index) {
-        if (this.hasKeys) {
+        if (hasKeys) {
           Object key = context.getDeserializer().readObject(in);
-          this.keys.add(key);
+          keys.add(key);
         }
         boolean isException = in.readBoolean();
         Object value;
@@ -238,7 +235,7 @@ public class ObjectPartList implements DataSerializableFixedID, Releasable {
         } else {
           value = context.getDeserializer().readObject(in);
         }
-        this.objects.add(value);
+        objects.add(value);
       }
     }
   }
@@ -255,7 +252,7 @@ public class ObjectPartList implements DataSerializableFixedID, Releasable {
 
   @Override
   public void release() {
-    for (Object v : this.objects) {
+    for (Object v : objects) {
       OffHeapHelper.release(v);
     }
   }
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/VersionedObjectList.java b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/VersionedObjectList.java
index a1f96b2..7495939 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/VersionedObjectList.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/VersionedObjectList.java
@@ -36,8 +36,6 @@ import org.apache.geode.DataSerializer;
 import org.apache.geode.InternalGemFireException;
 import org.apache.geode.annotations.Immutable;
 import org.apache.geode.distributed.DistributedMember;
-import org.apache.geode.distributed.internal.DistributionManager;
-import org.apache.geode.distributed.internal.InternalDistributedSystem;
 import org.apache.geode.distributed.internal.membership.InternalDistributedMember;
 import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.cache.versions.DiskVersionTag;
@@ -83,13 +81,13 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
       logger.trace(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE,
           "VersionedObjectList.addKeyAndVersion({}; {})", key, versionTag);
     }
-    if (this.objects.size() > 0) {
+    if (objects.size() > 0) {
       throw new IllegalStateException("attempt to add key/version to a list containing objects");
     }
-    this.keys.add(key);
-    if (this.regionIsVersioned) {
+    keys.add(key);
+    if (regionIsVersioned) {
       // Assert.assertTrue(versionTag != null);
-      this.versionTags.add(versionTag);
+      versionTags.add(versionTag);
     }
   }
 
@@ -100,19 +98,19 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
           objectType, versionTag);
     }
     super.addPart(key, value, objectType, versionTag);
-    if (this.regionIsVersioned) {
-      int tagsSize = this.versionTags.size();
-      if (keys != null && (tagsSize != this.keys.size() - 1)) {
+    if (regionIsVersioned) {
+      int tagsSize = versionTags.size();
+      if (keys != null && (tagsSize != keys.size() - 1)) {
         // this should not happen - either all or none of the entries should have tags
         throw new InternalGemFireException();
       }
-      if (this.objects != null && (this.objects.size() > 0)
-          && (tagsSize != this.objects.size() - 1)) {
+      if (objects != null && (objects.size() > 0)
+          && (tagsSize != objects.size() - 1)) {
         // this should not happen - either all or none of the entries should have tags
         throw new InternalGemFireException();
       }
       // Assert.assertTrue(versionTag != null);
-      this.versionTags.add(versionTag);
+      versionTags.add(versionTag);
     }
   }
 
@@ -127,7 +125,7 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
 
   public VersionedObjectList() {
     super();
-    this.versionTags = new ArrayList();
+    versionTags = new ArrayList<>();
   }
 
   public VersionedObjectList(boolean serializeValues) {
@@ -143,10 +141,10 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
       boolean serializeValues) {
     super(maxSize, hasKeys);
     if (regionIsVersioned) {
-      this.versionTags = new ArrayList(maxSize);
+      versionTags = new ArrayList<>(maxSize);
       this.regionIsVersioned = true;
     } else {
-      this.versionTags = new ArrayList();
+      versionTags = new ArrayList<>();
     }
     this.serializeValues = serializeValues;
   }
@@ -185,42 +183,42 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
       logger.trace(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE, "VOL.addAll(other={}; this={}", other,
           this);
     }
-    int myTypeArrayLength = this.hasKeys ? this.keys.size() : this.objects.size();
+    int myTypeArrayLength = hasKeys ? keys.size() : objects.size();
     int otherTypeArrayLength = other.hasKeys ? other.keys.size() : other.objects.size();
     super.addAll(other);
     VersionedObjectList vother = (VersionedObjectList) other;
-    this.regionIsVersioned |= vother.regionIsVersioned;
-    this.versionTags.addAll(vother.versionTags);
+    regionIsVersioned |= vother.regionIsVersioned;
+    versionTags.addAll(vother.versionTags);
     if (myTypeArrayLength > 0 || otherTypeArrayLength > 0) {
       int newSize = myTypeArrayLength + otherTypeArrayLength;
-      if (this.objectTypeArray != null) {
-        newSize = Math.max(newSize, this.objectTypeArray.length);
-        if (this.objectTypeArray.length < newSize) { // need more room
-          byte[] temp = this.objectTypeArray;
-          this.objectTypeArray = new byte[newSize];
-          System.arraycopy(temp, 0, this.objectTypeArray, 0, temp.length);
+      if (objectTypeArray != null) {
+        newSize = Math.max(newSize, objectTypeArray.length);
+        if (objectTypeArray.length < newSize) { // need more room
+          byte[] temp = objectTypeArray;
+          objectTypeArray = new byte[newSize];
+          System.arraycopy(temp, 0, objectTypeArray, 0, temp.length);
         }
       } else {
-        this.objectTypeArray = new byte[newSize];
+        objectTypeArray = new byte[newSize];
       }
       if (other.objectTypeArray != null) {
-        System.arraycopy(other.objectTypeArray, 0, this.objectTypeArray, myTypeArrayLength,
+        System.arraycopy(other.objectTypeArray, 0, objectTypeArray, myTypeArrayLength,
             otherTypeArrayLength);
       }
     }
   }
 
   public List<VersionTag> getVersionTags() {
-    return Collections.unmodifiableList(this.versionTags);
+    return Collections.unmodifiableList(versionTags);
   }
 
   public boolean hasVersions() {
-    return this.versionTags.size() > 0;
+    return versionTags.size() > 0;
   }
 
   /** clear the version tags from this list */
   public void clearVersions() {
-    this.versionTags = new ArrayList<VersionTag>(Math.max(50, this.size()));
+    versionTags = new ArrayList<>(Math.max(50, size()));
   }
 
   /**
@@ -230,7 +228,7 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
    * @param tag the version tag to add
    */
   public void addVersion(VersionTag tag) {
-    this.versionTags.add(tag);
+    versionTags.add(tag);
   }
 
   /**
@@ -238,7 +236,7 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
    *
    */
   public void saveVersions(Map<Object, VersionTag> vault) {
-    Iterator it = this.iterator();
+    Iterator it = iterator();
     while (it.hasNext()) {
       Entry e = it.next();
       if (e.getVersionTag() != null || !vault.containsKey(e.getKey())) {
@@ -254,16 +252,16 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
    * @return whether the source region had concurrency checks enabled
    */
   public boolean regionIsVersioned() {
-    return this.regionIsVersioned;
+    return regionIsVersioned;
   }
 
   /**
    * add some versionless keys
    */
   public void addAllKeys(Collection<?> keys) {
-    if (!this.hasKeys) {
-      this.hasKeys = true;
-      this.keys = new ArrayList(keys);
+    if (!hasKeys) {
+      hasKeys = true;
+      this.keys = new ArrayList<>(keys);
     } else {
       this.keys.addAll(keys);
     }
@@ -272,29 +270,25 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
   @Override
   public void reinit(int maxSize) {
     super.reinit(maxSize);
-    this.versionTags.clear();
+    versionTags.clear();
   }
 
   /** sets the keys for this partlist */
-  public VersionedObjectList setKeys(List newKeys) {
-    this.keys = newKeys;
-    this.hasKeys = (this.keys != null);
+  public VersionedObjectList setKeys(List<Object> newKeys) {
+    keys = newKeys;
+    hasKeys = (keys != null);
     return this;
   }
 
   public void clearObjects() {
-    this.objects = Collections.emptyList();
-    this.objectTypeArray = new byte[0];
+    objects = Collections.emptyList();
+    objectTypeArray = new byte[0];
   }
 
   @Override
   public void clear() {
     super.clear();
-    this.versionTags.clear();
-  }
-
-  public void processVersionTags(InternalDistributedMember sender) {
-
+    versionTags.clear();
   }
 
   @Immutable
@@ -305,24 +299,6 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
     return serializationVersions;
   }
 
-  /*
-   * for backward compatibility we need to make sure the IDs in the version tags aren't the partial
-   * IDs sent in serialized tags but the full tags. See bug #50063
-   */
-  private void getCanonicalIDs() {
-    if (this.versionTags != null) {
-      DistributionManager dm =
-          InternalDistributedSystem.getConnectedInstance().getDistributionManager();
-      if (dm != null) {
-        for (VersionTag tag : this.versionTags) {
-          if (tag != null) {
-            tag.setCanonicalIDs(dm);
-          }
-        }
-      }
-    }
-  }
-
   static final byte FLAG_NULL_TAG = 0;
   static final byte FLAG_FULL_TAG = 1;
   static final byte FLAG_TAG_WITH_NEW_ID = 2;
@@ -331,7 +307,7 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
   @Override
   public void toData(DataOutput out,
       SerializationContext context) throws IOException {
-    toData(out, context, 0, this.regionIsVersioned ? this.versionTags.size() : size(), true, true);
+    toData(out, context, 0, regionIsVersioned ? versionTags.size() : size(), true, true);
   }
 
   void toData(DataOutput out, SerializationContext context,
@@ -340,17 +316,17 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
     int flags = 0;
     boolean hasObjects = false;
     boolean hasTags = false;
-    if (sendKeys && this.hasKeys) {
+    if (sendKeys && hasKeys) {
       flags |= 0x01;
     }
-    if (sendObjects && !this.objects.isEmpty()) {
+    if (sendObjects && !objects.isEmpty()) {
       flags |= 0x02;
       hasObjects = true;
     }
-    if (this.versionTags.size() > 0) {
+    if (versionTags.size() > 0) {
       flags |= 0x04;
       hasTags = true;
-      for (VersionTag tag : this.versionTags) {
+      for (VersionTag tag : versionTags) {
         if (tag != null) {
           if (tag instanceof DiskVersionTag) {
             flags |= 0x20;
@@ -359,10 +335,10 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
         }
       }
     }
-    if (this.regionIsVersioned) {
+    if (regionIsVersioned) {
       flags |= 0x08;
     }
-    if (this.serializeValues) {
+    if (serializeValues) {
       flags |= 0x10;
     }
     if (logger.isTraceEnabled(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE)) {
@@ -373,38 +349,38 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
     out.writeByte(flags);
     if (sendKeys && hasKeys) {
       int numToWrite = numEntries;
-      if (numToWrite + startIndex > this.keys.size()) {
-        numToWrite = Math.max(0, this.keys.size() - startIndex);
+      if (numToWrite + startIndex > keys.size()) {
+        numToWrite = Math.max(0, keys.size() - startIndex);
       }
       InternalDataSerializer.writeUnsignedVL(numToWrite, out);
       int index = startIndex;
       for (int i = 0; i < numToWrite; i++, index++) {
-        context.getSerializer().writeObject(this.keys.get(index), out);
+        context.getSerializer().writeObject(keys.get(index), out);
       }
     }
     if (sendObjects && hasObjects) {
       int numToWrite = numEntries;
-      if (numToWrite + startIndex > this.objects.size()) {
-        numToWrite = Math.max(0, this.objects.size() - startIndex);
+      if (numToWrite + startIndex > objects.size()) {
+        numToWrite = Math.max(0, objects.size() - startIndex);
       }
       InternalDataSerializer.writeUnsignedVL(numToWrite, out);
       int idx = 0;
       int index = startIndex;
       for (int i = 0; i < numToWrite; i++, index++) {
-        writeObject(this.objects.get(index), idx++, out, context);
+        writeObject(objects.get(index), idx++, out, context);
       }
     }
     if (hasTags) {
       int numToWrite = numEntries;
-      if (numToWrite + startIndex > this.versionTags.size()) {
-        numToWrite = Math.max(0, this.versionTags.size() - startIndex);
+      if (numToWrite + startIndex > versionTags.size()) {
+        numToWrite = Math.max(0, versionTags.size() - startIndex);
       }
       InternalDataSerializer.writeUnsignedVL(numToWrite, out);
-      Map<VersionSource, Integer> ids = new HashMap<VersionSource, Integer>(numToWrite);
+      Map<VersionSource, Integer> ids = new HashMap<>(numToWrite);
       int idCount = 0;
       int index = startIndex;
       for (int i = 0; i < numToWrite; i++, index++) {
-        VersionTag tag = this.versionTags.get(index);
+        VersionTag tag = versionTags.get(index);
         if (tag == null) {
           out.writeByte(FLAG_NULL_TAG);
         } else {
@@ -416,7 +392,7 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
             Integer idNumber = ids.get(id);
             if (idNumber == null) {
               out.writeByte(FLAG_TAG_WITH_NEW_ID);
-              idNumber = Integer.valueOf(idCount++);
+              idNumber = idCount++;
               ids.put(id, idNumber);
               InternalDataSerializer.invokeToData(tag, out);
             } else {
@@ -437,24 +413,24 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
     final boolean isDebugEnabled_VOL =
         logger.isTraceEnabled(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE);
     int flags = in.readByte();
-    this.hasKeys = (flags & 0x01) == 0x01;
+    hasKeys = (flags & 0x01) == 0x01;
     boolean hasObjects = (flags & 0x02) == 0x02;
     boolean hasTags = (flags & 0x04) == 0x04;
-    this.regionIsVersioned = (flags & 0x08) == 0x08;
-    this.serializeValues = (flags & 0x10) == 0x10;
+    regionIsVersioned = (flags & 0x08) == 0x08;
+    serializeValues = (flags & 0x10) == 0x10;
     boolean persistent = (flags & 0x20) == 0x20;
     if (isDebugEnabled_VOL) {
       logger.trace(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE,
           "deserializing a VersionedObjectList with flags 0x{}", Integer.toHexString(flags));
     }
-    if (this.hasKeys) {
+    if (hasKeys) {
       int size = (int) InternalDataSerializer.readUnsignedVL(in);
-      this.keys = new ArrayList(size);
+      keys = new ArrayList<>(size);
       if (isDebugEnabled_VOL) {
         logger.trace(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE, "reading {} keys", size);
       }
       for (int i = 0; i < size; i++) {
-        this.keys.add(context.getDeserializer().readObject(in));
+        keys.add(context.getDeserializer().readObject(in));
       }
     }
     if (hasObjects) {
@@ -462,58 +438,58 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
       if (isDebugEnabled_VOL) {
         logger.trace(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE, "reading {} objects", size);
       }
-      this.objects = new ArrayList(size);
-      this.objectTypeArray = new byte[size];
+      objects = new ArrayList<>(size);
+      objectTypeArray = new byte[size];
       for (int i = 0; i < size; i++) {
         readObject(i, in, context);
       }
     } else {
-      this.objects = new ArrayList();
+      objects = new ArrayList<>();
     }
     if (hasTags) {
       int size = (int) InternalDataSerializer.readUnsignedVL(in);
       if (isDebugEnabled_VOL) {
         logger.trace(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE, "reading {} version tags", size);
       }
-      this.versionTags = new ArrayList<VersionTag>(size);
-      List<VersionSource> ids = new ArrayList<VersionSource>(size);
+      versionTags = new ArrayList<>(size);
+      List<VersionSource> ids = new ArrayList<>(size);
       for (int i = 0; i < size; i++) {
         byte entryType = in.readByte();
         switch (entryType) {
           case FLAG_NULL_TAG:
-            this.versionTags.add(null);
+            versionTags.add(null);
             break;
           case FLAG_FULL_TAG:
-            this.versionTags.add(VersionTag.create(persistent, in));
+            versionTags.add(VersionTag.create(persistent, in));
             break;
           case FLAG_TAG_WITH_NEW_ID:
             VersionTag tag = VersionTag.create(persistent, in);
             ids.add(tag.getMemberID());
-            this.versionTags.add(tag);
+            versionTags.add(tag);
             break;
           case FLAG_TAG_WITH_NUMBER_ID:
             tag = VersionTag.create(persistent, in);
             int idNumber = (int) InternalDataSerializer.readUnsignedVL(in);
             tag.setMemberID(ids.get(idNumber));
-            this.versionTags.add(tag);
+            versionTags.add(tag);
             break;
         }
       }
     } else {
-      this.versionTags = new ArrayList<VersionTag>();
+      versionTags = new ArrayList<>();
     }
   }
 
   private void writeObject(Object value, int index, DataOutput out,
       SerializationContext context) throws IOException {
-    byte objectType = this.objectTypeArray[index];
+    byte objectType = objectTypeArray[index];
     if (logger.isTraceEnabled(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE)) {
       logger.trace(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE, "writing object {} of type {}: {}",
           index, objectType, value);
     }
     out.writeByte(objectType);
     if (objectType == OBJECT && value instanceof byte[]) {
-      if (this.serializeValues) {
+      if (serializeValues) {
         DataSerializer.writeByteArray((byte[]) value, out);
       } else {
         out.write((byte[]) value);
@@ -524,7 +500,7 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
       // write the exception string for native clients
       DataSerializer.writeString(value.toString(), out);
     } else {
-      if (this.serializeValues) {
+      if (serializeValues) {
         DataSerializer.writeObjectAsByteArray(value, out);
       } else {
         context.getSerializer().writeObject(value, out);
@@ -536,23 +512,23 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
   private void readObject(int index, DataInput in,
       DeserializationContext context) throws IOException, ClassNotFoundException {
     Object value;
-    this.objectTypeArray[index] = in.readByte();
+    objectTypeArray[index] = in.readByte();
     if (logger.isTraceEnabled(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE)) {
       logger.trace(LogMarker.VERSIONED_OBJECT_LIST_VERBOSE, "reading object {} of type {}", index,
           objectTypeArray[index]);
     }
-    boolean isException = this.objectTypeArray[index] == EXCEPTION;
+    boolean isException = objectTypeArray[index] == EXCEPTION;
     if (isException) {
       byte[] exBytes = DataSerializer.readByteArray(in);
       value = CacheServerHelper.deserialize(exBytes);
       // ignore the exception string meant for native clients
       DataSerializer.readString(in);
-    } else if (this.serializeValues) {
+    } else if (serializeValues) {
       value = DataSerializer.readByteArray(in);
     } else {
       value = context.getDeserializer().readObject(in);
     }
-    this.objects.add(value);
+    objects.add(value);
   }
 
   @Override
@@ -578,28 +554,28 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
         .append(keys == null ? "null" : String.valueOf(keys.size())).append("; objects=")
         .append(objects.size()).append("; isobject=")
         .append(objectTypeArray == null ? "null" : objectTypeArray.length).append("; ccEnabled=")
-        .append(this.regionIsVersioned).append("; versionTags=")
-        .append(String.valueOf(versionTags.size())).append(")\n");
-    Iterator entries = this.iterator();
+        .append(regionIsVersioned).append("; versionTags=")
+        .append(versionTags.size()).append(")\n");
+    Iterator entries = iterator();
     while (entries.hasNext()) {
       desc.append(entries.next()).append("\n");
     }
     return desc.toString();
   }
 
-  public Set keySet() {
-    if (!this.hasKeys) {
-      return Collections.EMPTY_SET;
+  public Set<Object> keySet() {
+    if (!hasKeys) {
+      return Collections.emptySet();
     } else {
-      return new HashSet(this.keys);
+      return new HashSet<>(keys);
     }
   }
 
-  public class Entry implements Map.Entry {
+  public class Entry implements Map.Entry<Object, Object> {
     int index;
 
     Entry(int idx) {
-      this.index = idx;
+      index = idx;
     }
 
     @Override
@@ -674,7 +650,7 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
     return new Iterator();
   }
 
-  public class Iterator implements java.util.Iterator {
+  public class Iterator implements java.util.Iterator<Entry> {
     int index = 0;
     int size = size();
 
@@ -719,10 +695,10 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
    */
   public static class Chunker implements DataSerializableFixedID {
     private int index = 0;
-    private VersionedObjectList list;
-    private int chunkSize;
-    private boolean sendKeys;
-    private boolean sendObjects;
+    private final VersionedObjectList list;
+    private final int chunkSize;
+    private final boolean sendKeys;
+    private final boolean sendObjects;
 
     public Chunker(VersionedObjectList list, int chunkSize, boolean sendKeys, boolean sendObjects) {
       this.list = list;
@@ -740,8 +716,8 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
     public void toData(DataOutput out,
         SerializationContext context) throws IOException {
       int startIndex = index;
-      this.index += this.chunkSize;
-      this.list.toData(out, context, startIndex, chunkSize, sendKeys, sendObjects);
+      index += chunkSize;
+      list.toData(out, context, startIndex, chunkSize, sendKeys, sendObjects);
     }
 
     @Override
@@ -752,7 +728,7 @@ public class VersionedObjectList extends ObjectPartList implements Externalizabl
 
     @Override
     public KnownVersion[] getSerializationVersions() {
-      return this.list.getSerializationVersions();
+      return list.getSerializationVersions();
     }
   }