You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by go...@apache.org on 2018/05/14 23:56:01 UTC

[geode] 06/07: GEODE-5198 NPE in DataSerializer registration when forming a client/server connection during handshake

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

gosullivan pushed a commit to branch support/9.5
in repository https://gitbox.apache.org/repos/asf/geode.git

commit ef5ebbd2e1f3ae8dadef11ffc9b12443736df7de
Author: Bruce Schuchardt <bs...@pivotal.io>
AuthorDate: Fri May 11 09:03:23 2018 -0700

    GEODE-5198 NPE in DataSerializer registration when forming a client/server connection during handshake
    
    If a holder can't be found do not record supported classes for it.
    Absense of the holder, which had just been inserted into the
    idsToHolders collection, means that another thread has resolved the
    holder into an actual DataSerializer class and has removed the holder
    and its supported classes.
    
    (cherry picked from commit a60b8d4237f0d27b9b31af304ae6ec3bfcdc077c)
---
 .../client/internal/ClientSideHandshakeImpl.java   |   2 +-
 .../geode/internal/InternalDataSerializer.java     |  32 +-
 .../deadlock/GemFireDeadlockDetectorDUnitTest.java |   2 -
 .../gms/fd/GMSHealthMonitorJUnitTest.java          |   1 -
 .../gms/membership/GMSJoinLeaveJUnitTest.java      |   1 -
 .../internal/DataSerializerHolderJUnitTest.java    | 511 +++++++++++++++++++++
 ...niversalMembershipListenerAdapterDUnitTest.java |   2 -
 .../org/apache/geode/redis/RedisDistDUnitTest.java |   8 -
 8 files changed, 538 insertions(+), 21 deletions(-)

diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientSideHandshakeImpl.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientSideHandshakeImpl.java
index 0f4aa4e..e374549 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientSideHandshakeImpl.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ClientSideHandshakeImpl.java
@@ -343,7 +343,7 @@ public class ClientSideHandshakeImpl extends Handshake implements ClientSideHand
         Integer id = (Integer) dataSerializer.getKey();
         InternalDataSerializer.register((String) dataSerializer.getValue(), false, null, null, id);
       }
-      HashMap<Integer, ArrayList<String>> dsToSupportedClassNames = DataSerializer.readHashMap(dis);
+      Map<Integer, List<String>> dsToSupportedClassNames = DataSerializer.readHashMap(dis);
       InternalDataSerializer.updateSupportedClassesMap(dsToSupportedClassNames);
 
       // the server's ping interval is only sent to subscription feeds so we can't read it as
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 334c058..7ab6284 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
@@ -828,7 +828,7 @@ public abstract class InternalDataSerializer extends DataSerializer {
    *
    * @see DataSerializer#register(Class)
    */
-  private static DataSerializer newInstance(Class c) {
+  static DataSerializer newInstance(Class c) {
     if (!DataSerializer.class.isAssignableFrom(c)) {
       throw new IllegalArgumentException(
           LocalizedStrings.DataSerializer_0_DOES_NOT_EXTEND_DATASERIALIZER
@@ -1019,7 +1019,7 @@ public abstract class InternalDataSerializer extends DataSerializer {
 
   /**
    * Marks a {@code DataSerializer} className for registration with the data serialization
-   * framework. Does not necessarily load the classes into this VM.
+   * framework if and when it is needed. Does not necessarily load the classes into this VM.
    *
    * @param className Name of the DataSerializer class.
    * @param distribute If true, distribute this data serializer.
@@ -1073,18 +1073,38 @@ public abstract class InternalDataSerializer extends DataSerializer {
     }
   }
 
-  public static void updateSupportedClassesMap(HashMap<Integer, ArrayList<String>> map) {
-    for (Entry<Integer, ArrayList<String>> e : map.entrySet()) {
+  /**
+   * During client/server handshakes the server may send a collection of DataSerializers and
+   * the classes they support. The DataSerializers are registered as "holders" to avoid loading the
+   * actual classes until they're needed. This method registers the names of classes supported
+   * by the DataSerializers
+   *
+   * @param map The classes returned by DataSerializer.supportedClasses()
+   */
+  public static void updateSupportedClassesMap(Map<Integer, List<String>> map) {
+    for (Entry<Integer, List<String>> e : map.entrySet()) {
       for (String supportedClassName : e.getValue()) {
-        supportedClassesToHolders.putIfAbsent(supportedClassName, idsToHolders.get(e.getKey()));
+        SerializerAttributesHolder serializerAttributesHolder = idsToHolders.get(e.getKey());
+        if (serializerAttributesHolder != null) {
+          supportedClassesToHolders.putIfAbsent(supportedClassName, serializerAttributesHolder);
+        }
       }
     }
   }
 
   public static void updateSupportedClassesMap(String dsClassName, String supportedClassName) {
-    supportedClassesToHolders.putIfAbsent(supportedClassName, dsClassesToHolders.get(dsClassName));
+    SerializerAttributesHolder holder = dsClassesToHolders.get(dsClassName);
+    if (holder != null) {
+      supportedClassesToHolders.putIfAbsent(supportedClassName, holder);
+    }
   }
 
+  /**
+   * 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;
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/deadlock/GemFireDeadlockDetectorDUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/deadlock/GemFireDeadlockDetectorDUnitTest.java
index a6f2a4a..41e89b6 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/deadlock/GemFireDeadlockDetectorDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/deadlock/GemFireDeadlockDetectorDUnitTest.java
@@ -108,8 +108,6 @@ public class GemFireDeadlockDetectorDUnitTest extends JUnit4CacheTestCase {
 
   private static final Lock lock = new ReentrantLock();
 
-  // @Category(FlakyTest.class) // GEODE-516 & GEODE-576: async actions, thread sleeps, time
-  // sensitive
   @Test
   public void testDistributedDeadlockWithFunction() throws Throwable {
     Host host = Host.getHost(0);
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
index f82abd9..52490fa 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/fd/GMSHealthMonitorJUnitTest.java
@@ -173,7 +173,6 @@ public class GMSHealthMonitorJUnitTest {
     assertEquals(mockMembers.get(myAddressIndex + 1), gmsHealthMonitor.getNextNeighbor());
   }
 
-  // @Category(FlakyTest.class) // GEODE-2073
   @Test
   public void testHMNextNeighborAfterTimeout() throws Exception {
     System.out.println("testHMNextNeighborAfterTimeout starting");
diff --git a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
index 0443b32..8fd8764 100644
--- a/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/distributed/internal/membership/gms/membership/GMSJoinLeaveJUnitTest.java
@@ -513,7 +513,6 @@ public class GMSJoinLeaveJUnitTest {
         view.getCrashedMembers().contains(mockMembers[0]));
   }
 
-  // @Category(FlakyTest.class) // GEODE-2074: timed out waiting for view #7
   @Test
   public void testDuplicateJoinRequestDoesNotCauseNewView() throws Exception {
     initMocks();
diff --git a/geode-core/src/test/java/org/apache/geode/internal/DataSerializerHolderJUnitTest.java b/geode-core/src/test/java/org/apache/geode/internal/DataSerializerHolderJUnitTest.java
new file mode 100644
index 0000000..5d45344
--- /dev/null
+++ b/geode-core/src/test/java/org/apache/geode/internal/DataSerializerHolderJUnitTest.java
@@ -0,0 +1,511 @@
+/*
+ * 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.internal;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import junit.framework.TestCase;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class DataSerializerHolderJUnitTest extends TestCase {
+
+  @After
+  public void tearDown() {
+    InternalDataSerializer.reinitialize();
+  }
+
+  @Test
+  public void testHandshakeDatSerializerRegistrationDoesNotHitNPE() throws Throwable {
+    // a thread performing a handshake from the client side may receive a list of
+    // DataSerializer class names. It registers these with InternalDataSerializer to
+    // create placeholders for later lazy loading of the classes
+    Class[] serializers = new Class[] {DataSerializer1.class, DataSerializer2.class,
+        DataSerializer3.class, DataSerializer4.class, DataSerializer5.class, DataSerializer6.class,
+        DataSerializer7.class, DataSerializer8.class, DataSerializer9.class, DataSerializer10.class,
+        DataSerializer11.class, DataSerializer12.class, DataSerializer13.class};
+    for (int index = 0; index < serializers.length; index++) {
+      int id = InternalDataSerializer.newInstance(serializers[index]).getId();
+      InternalDataSerializer.register(serializers[index].getName(), false, null, null, id);
+    }
+
+    // The thread will then register classes handled by the DataSerializers, but if
+    // getSerializers() or a similar method is invoked by some other thread first the
+    // placeholders will be wiped out, causing an NPE when registering the handled
+    // classes. The NPE is caused by the placeholder being null in updateSupportedClassesMap().
+    // Here we avoid creating a multithreaded test by invoking getSerializers() in-line
+    InternalDataSerializer.getSerializers();
+
+
+    // Now we perform the second step in the handshake code of registering the classes
+    // handled by the DataSerializers. Without the bugfix this causes an NPE
+    Map<Integer, List<String>> supportedClasses = new HashMap<>();
+    for (int index = 0; index < serializers.length; index++) {
+      DataSerializer serializer = InternalDataSerializer.newInstance(serializers[index]);
+      List<String> classes = Arrays.<Class>asList(serializer.getSupportedClasses()).stream()
+          .map((clazz) -> clazz.getName()).collect(Collectors.toList());
+      supportedClasses.put(serializer.getId(), classes);
+    }
+    InternalDataSerializer.updateSupportedClassesMap(supportedClasses);
+  }
+
+  public static class DataSerializer1 extends DataSerializer {
+    int tempField = 5;
+
+    public DataSerializer1() {
+
+    }
+
+    @Override
+    public int getId() {
+      return 1;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {MyClass1.class, MyClass2.class, MyClass3.class, MyClass4.class,
+          MyClass5.class, MyClass6.class, MyClass7.class, MyClass8.class, MyClass9.class,
+          MyClass10.class};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      return true;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer2 extends DataSerializer {
+    int tempField = 15;
+
+    public DataSerializer2() {}
+
+    @Override
+    public int getId() {
+      return 2;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {MyClass11.class, MyClass12.class, MyClass13.class, MyClass14.class,
+          MyClass15.class, MyClass16.class, MyClass17.class, MyClass18.class, MyClass19.class,
+          MyClass20.class};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      return true;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer3 extends DataSerializer {
+    int tempField = 25;
+
+    public DataSerializer3() {}
+
+    @Override
+    public int getId() {
+      return 3;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer4 extends DataSerializer {
+    int tempField = 5;
+
+    public DataSerializer4() {
+
+    }
+
+    @Override
+    public int getId() {
+      return 4;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer5 extends DataSerializer {
+    int tempField = 15;
+
+    public DataSerializer5() {}
+
+    @Override
+    public int getId() {
+      return 5;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer6 extends DataSerializer {
+    int tempField = 25;
+
+    public DataSerializer6() {}
+
+    @Override
+    public int getId() {
+      return 6;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer7 extends DataSerializer {
+    int tempField = 5;
+
+    public DataSerializer7() {
+
+    }
+
+    @Override
+    public int getId() {
+      return 7;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer8 extends DataSerializer {
+    int tempField = 15;
+
+    public DataSerializer8() {}
+
+    @Override
+    public int getId() {
+      return 8;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer9 extends DataSerializer {
+    int tempField = 25;
+
+    public DataSerializer9() {}
+
+    @Override
+    public int getId() {
+      return 9;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer10 extends DataSerializer {
+    int tempField = 5;
+
+    public DataSerializer10() {
+
+    }
+
+    @Override
+    public int getId() {
+      return 10;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer11 extends DataSerializer {
+    int tempField = 15;
+
+    public DataSerializer11() {}
+
+    @Override
+    public int getId() {
+      return 11;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer12 extends DataSerializer {
+    int tempField = 25;
+
+    public DataSerializer12() {}
+
+    @Override
+    public int getId() {
+      return 12;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class DataSerializer13 extends DataSerializer {
+    int tempField = 25;
+
+    public DataSerializer13() {}
+
+    @Override
+    public int getId() {
+      return 19;
+    }
+
+    @Override
+    public Class[] getSupportedClasses() {
+      return new Class[] {this.getClass()};
+    }
+
+    @Override
+    public boolean toData(Object o, DataOutput out) throws IOException {
+      out.write(tempField);
+      return false;
+    }
+
+    @Override
+    public Object fromData(DataInput in) throws IOException, ClassNotFoundException {
+      readInteger(in);
+      return null;
+    }
+  }
+
+  public static class MyClass1 {
+    public MyClass1() {}
+  }
+  public static class MyClass2 {
+    public MyClass2() {}
+  }
+  public static class MyClass3 {
+    public MyClass3() {}
+  }
+  public static class MyClass4 {
+    public MyClass4() {}
+  }
+  public static class MyClass5 {
+    public MyClass5() {}
+  }
+  public static class MyClass6 {
+    public MyClass6() {}
+  }
+  public static class MyClass7 {
+    public MyClass7() {}
+  }
+  public static class MyClass8 {
+    public MyClass8() {}
+  }
+  public static class MyClass9 {
+    public MyClass9() {}
+  }
+  public static class MyClass10 {
+    public MyClass10() {}
+  }
+  public static class MyClass11 {
+    public MyClass11() {}
+  }
+  public static class MyClass12 {
+    public MyClass12() {}
+  }
+  public static class MyClass13 {
+    public MyClass13() {}
+  }
+  public static class MyClass14 {
+    public MyClass14() {}
+  }
+  public static class MyClass15 {
+    public MyClass15() {}
+  }
+  public static class MyClass16 {
+    public MyClass16() {}
+  }
+  public static class MyClass17 {
+    public MyClass17() {}
+  }
+  public static class MyClass18 {
+    public MyClass18() {}
+  }
+  public static class MyClass19 {
+    public MyClass19() {}
+  }
+  public static class MyClass20 {
+    public MyClass20() {}
+  }
+
+}
diff --git a/geode-core/src/test/java/org/apache/geode/management/UniversalMembershipListenerAdapterDUnitTest.java b/geode-core/src/test/java/org/apache/geode/management/UniversalMembershipListenerAdapterDUnitTest.java
index c5069e9..a8fc177 100644
--- a/geode-core/src/test/java/org/apache/geode/management/UniversalMembershipListenerAdapterDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/management/UniversalMembershipListenerAdapterDUnitTest.java
@@ -80,7 +80,6 @@ import org.apache.geode.test.dunit.SerializableRunnable;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.rules.DistributedRestoreSystemProperties;
 import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.categories.FlakyTest;
 
 /**
  * Distributed tests for {@link UniversalMembershipListenerAdapter}.
@@ -1077,7 +1076,6 @@ public class UniversalMembershipListenerAdapterDUnitTest extends ClientServerTes
   /**
    * Tests notification of events for bridge server in system bridge client process.
    */
-  @Category(FlakyTest.class) // GEODE-1879
   @Test
   public void testServerEventsInPeerSystem() throws Exception {
     boolean[] firedSystem = new boolean[3];
diff --git a/geode-core/src/test/java/org/apache/geode/redis/RedisDistDUnitTest.java b/geode-core/src/test/java/org/apache/geode/redis/RedisDistDUnitTest.java
index d6030a9..781d062 100644
--- a/geode-core/src/test/java/org/apache/geode/redis/RedisDistDUnitTest.java
+++ b/geode-core/src/test/java/org/apache/geode/redis/RedisDistDUnitTest.java
@@ -36,7 +36,6 @@ import org.apache.geode.test.dunit.SerializableCallable;
 import org.apache.geode.test.dunit.VM;
 import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
 import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.categories.FlakyTest;
 
 @Category(DistributedTest.class)
 public class RedisDistDUnitTest extends JUnit4DistributedTestCase {
@@ -105,9 +104,6 @@ public class RedisDistDUnitTest extends JUnit4DistributedTestCase {
     disconnectAllFromDS();
   }
 
-  @Category(FlakyTest.class) // GEODE-1092: random ports, failure stack involves TCPTransport
-                             // ConnectionHandler (are we eating BindExceptions somewhere?), uses
-                             // Random, async actions
   @Test
   public void testConcListOps() throws Exception {
     final Jedis jedis1 = new Jedis(localHost, server1Port, JEDIS_TIMEOUT);
@@ -143,8 +139,6 @@ public class RedisDistDUnitTest extends JUnit4DistributedTestCase {
     assertEquals(result1, result2);
   }
 
-  @Category(FlakyTest.class) // GEODE-717: random ports, BindException in failure stack, async
-                             // actions
   @Test
   public void testConcCreateDestroy() throws Exception {
     IgnoredException.addIgnoredException("RegionDestroyedException");
@@ -205,8 +199,6 @@ public class RedisDistDUnitTest extends JUnit4DistributedTestCase {
   /**
    * Just make sure there are no unexpected server crashes
    */
-  @Category(FlakyTest.class) // GEODE-1697
-  @Test
   public void testConcOps() throws Exception {
 
     final int ops = 100;

-- 
To stop receiving notification emails like this one, please contact
gosullivan@apache.org.