You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2021/10/05 21:05:54 UTC

[GitHub] [geode] kirklund commented on a change in pull request #6891: GEODE-9627: Add service provider interface to register DataSerializableFixedIDs

kirklund commented on a change in pull request #6891:
URL: https://github.com/apache/geode/pull/6891#discussion_r722636400



##########
File path: geode-serialization/src/main/java/org/apache/geode/internal/serialization/DataSerializableFixedIdRegistrar.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.serialization;
+
+public interface DataSerializableFixedIdRegistrar {
+
+  /**
+   * Register the constructor for a fixed ID class. Use this to register your
+   * DataSerializableFixedID classes so that deserialization knows how to instantiate them.
+   */
+  void register(int dsfid, Class<? extends DataSerializableFixedID> dsfidClass);

Review comment:
       How naming the parameters `fixedId` and `fixedIdClass` to avoid abbreviations?

##########
File path: geode-serialization/src/main/java/org/apache/geode/internal/serialization/DataSerializableFixedIdLoader.java
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.serialization;
+
+
+/**
+ * Service loader interface used to register Data Serializable Fixed IDs. Geode
+ * {@code CacheService}s should implement this SPI instead of explicitly registering DSFIDs during
+ * service initialization.
+ */
+public interface DataSerializableFixedIdLoader {
+
+  void register(DataSerializableFixedIdRegistrar serializer);

Review comment:
       I think you should rename the parameter to `registrar` here and in all implementations. If you use IntelliJ "Refactor > Change Signature", it'll rename the parameter in all of the implementation as well.

##########
File path: geode-serialization/src/main/java/org/apache/geode/internal/serialization/internal/DSFIDSerializerImpl.java
##########
@@ -336,7 +336,7 @@ public void invokeFromData(Object ds, DataInput in)
 
 
   @Override
-  public void registerDSFID(int dsfid, Class<?> dsfidClass) {
+  public void register(int dsfid, Class<? extends DataSerializableFixedID> dsfidClass) {

Review comment:
       I think you should create a new unit test DSFIDSerializerImplTest. Just add test(s) for this method and focus on covering the if-else block where you added the most code.

##########
File path: geode-lucene/src/main/resources/META-INF/services/org.apache.geode.internal.serialization.DataSerializableFixedIdLoader
##########
@@ -0,0 +1 @@
+org.apache.geode.cache.lucene.internal.LuceneServiceImpl

Review comment:
       This service file should have an Apache license header.

##########
File path: geode-wan/src/main/java/org/apache/geode/cache/wan/internal/client/locator/WANFactoryImpl.java
##########
@@ -27,37 +27,16 @@
 import org.apache.geode.cache.wan.internal.GatewayReceiverFactoryImpl;
 import org.apache.geode.cache.wan.internal.GatewaySenderFactoryImpl;
 import org.apache.geode.distributed.internal.WanLocatorDiscoverer;
-import org.apache.geode.internal.InternalDataSerializer;
 import org.apache.geode.internal.cache.InternalCache;
 import org.apache.geode.internal.cache.wan.spi.WANFactory;
 import org.apache.geode.internal.serialization.DataSerializableFixedID;
+import org.apache.geode.internal.serialization.DataSerializableFixedIdLoader;
+import org.apache.geode.internal.serialization.DataSerializableFixedIdRegistrar;
 
-public class WANFactoryImpl implements WANFactory {
+public class WANFactoryImpl implements WANFactory, DataSerializableFixedIdLoader {
 
   @Override
-  public void initialize() {
-    InternalDataSerializer.getDSFIDSerializer().registerDSFID(
-        DataSerializableFixedID.REMOTE_LOCATOR_JOIN_REQUEST,
-        RemoteLocatorJoinRequest.class);
-    InternalDataSerializer.getDSFIDSerializer().registerDSFID(
-        DataSerializableFixedID.REMOTE_LOCATOR_JOIN_RESPONSE,
-        RemoteLocatorJoinResponse.class);
-    InternalDataSerializer.getDSFIDSerializer().registerDSFID(
-        DataSerializableFixedID.REMOTE_LOCATOR_REQUEST,
-        RemoteLocatorRequest.class);
-    InternalDataSerializer.getDSFIDSerializer().registerDSFID(
-        DataSerializableFixedID.LOCATOR_JOIN_MESSAGE,
-        LocatorJoinMessage.class);
-    InternalDataSerializer.getDSFIDSerializer().registerDSFID(
-        DataSerializableFixedID.REMOTE_LOCATOR_PING_REQUEST,
-        RemoteLocatorPingRequest.class);
-    InternalDataSerializer.getDSFIDSerializer().registerDSFID(
-        DataSerializableFixedID.REMOTE_LOCATOR_PING_RESPONSE,
-        RemoteLocatorPingResponse.class);
-    InternalDataSerializer.getDSFIDSerializer().registerDSFID(
-        DataSerializableFixedID.REMOTE_LOCATOR_RESPONSE,
-        RemoteLocatorResponse.class);
-  }
+  public void initialize() {}

Review comment:
       [comment] You might consider giving `initialize` an empty default in the interface.

##########
File path: geode-serialization/src/main/java/org/apache/geode/internal/serialization/DataSerializableFixedIdLoader.java
##########
@@ -0,0 +1,28 @@
+/*
+ * 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.serialization;
+
+
+/**
+ * Service loader interface used to register Data Serializable Fixed IDs. Geode
+ * {@code CacheService}s should implement this SPI instead of explicitly registering DSFIDs during
+ * service initialization.
+ */
+public interface DataSerializableFixedIdLoader {

Review comment:
       You should annotate this with `@FunctionalInterface`.

##########
File path: geode-serialization/src/main/java/org/apache/geode/internal/serialization/DSFIDSerializer.java
##########
@@ -27,7 +27,7 @@
  * <p>
  * Use DSFIDSerializerFactory to construct a serialization service.
  */
-public interface DSFIDSerializer {
+public interface DSFIDSerializer extends DataSerializableFixedIdRegistrar {

Review comment:
       Now that this interface only has one method defined, you should annotate it with `@FunctionalInterface`.

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java
##########
@@ -138,6 +138,15 @@
   @Rule
   public SerializableTestName testName = new SerializableTestName();
 
+  @BeforeClass
+  public static void setup() {
+    DUnitLauncher.launchIfNeeded(NUM_SERVERS + 1);
+    invokeInEveryVM(
+        () -> InternalDataSerializer.getDSFIDSerializer().register(
+            SerialAckedMessageWithBigReply.DSFID,
+            SerialAckedMessageWithBigReply.class));
+  }
+

Review comment:
       This is redundant with the `DistributedRule`. I would change this and the rule on line 131 to:
   ```
   @Rule
   public DistributedRule distributedRule = new DistributedRule(NUM_SERVERS + 1);
   
   @Before
   public void setUp() {
     invokeInEveryVM(() -> InternalDataSerializer.getDSFIDSerializer().register(
             SerialAckedMessageWithBigReply.DSFID, 
             SerialAckedMessageWithBigReply.class));
   }
   ```

##########
File path: geode-serialization/src/main/java/org/apache/geode/internal/serialization/DataSerializableFixedIdRegistrar.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.serialization;
+
+public interface DataSerializableFixedIdRegistrar {

Review comment:
       You should annotate this with `@FunctionalInterface`.

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/internal/BackwardCompatibilitySerializationDUnitTest.java
##########
@@ -249,6 +248,8 @@ private void resetFlags() {
     private static final KnownVersion[] dsfidVersions =
         new KnownVersion[] {KnownVersion.GEODE_1_1_0, KnownVersion.GEODE_1_5_0};
 
+    static final int TEST_MESSAGE_DSFID = 12345;
+

Review comment:
       I would make this `private` unless another class in this package needs to reference it.

##########
File path: geode-core/src/distributedTest/java/org/apache/geode/internal/BackwardCompatibilitySerializationDUnitTest.java
##########
@@ -68,21 +67,19 @@ public BackwardCompatibilitySerializationDUnitTest() {
     super();
   }
 
+  static {
+    InternalDataSerializer.getDSFIDSerializer()
+        .register(TestMessage.TEST_MESSAGE_DSFID, TestMessage.class);
+  }
+

Review comment:
       Since this is a test, I recommend moving it to `postSetUp`, a new `@Before` method, or a new `@BeforeClass` method.
   
   I think we should avoid static initializers when possible -- if the code fails, the class fails to load and a NoClassDefFoundError is thrown which is hard to debug.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/InternalDataSerializer.java
##########
@@ -329,6 +331,17 @@ public void invokeFromData(Object ds, DataInput in)
     }).create();
     initializeWellKnownSerializers();
     dsfidFactory = new DSFIDFactory(dsfidSerializer);
+
+    ServiceLoader<DataSerializableFixedIdLoader> loaders = ServiceLoader.load(
+        DataSerializableFixedIdLoader.class);
+    for (DataSerializableFixedIdLoader loader : loaders) {
+      try {
+        loader.register(dsfidSerializer);
+      } catch (Exception ex) {
+        logger.warn("Data serializable fixed ID loader '" + loader.getClass().getName() +
+            "' failed", ex);

Review comment:
       Log4j2 preferred syntax is:
   ```
   logger.warn("Data serializable fixed ID loader '{}' failed", loader.getClass().getName(), ex);
   ```
   I would change it mostly to avoid having others copy the concatenation syntax.

##########
File path: geode-apis-compatible-with-redis/src/main/resources/META-INF/services/org.apache.geode.internal.serialization.DataSerializableFixedIdLoader
##########
@@ -0,0 +1 @@
+org.apache.geode.redis.internal.GeodeRedisService

Review comment:
       This service file should have an Apache license header.

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java
##########
@@ -246,28 +255,19 @@ public void createEntryWithBigMessage() {
 
   @Test
   public void receiveBigResponse() {
-    invokeInEveryVM(
-        () -> InternalDataSerializer.getDSFIDSerializer().registerDSFID(SERIAL_ACKED_MESSAGE,
-            SerialAckedMessageWithBigReply.class));
-    try {
-      int locatorPort = createLocator(getVM(0));
-      for (int i = 1; i <= NUM_SERVERS; i++) {
-        createCacheAndRegion(getVM(i), locatorPort);
-      }
-      DistributedMember vm2ID =
-          getVM(2).invoke(() -> cache.getDistributedSystem().getDistributedMember());
-      getVM(1).invoke("receive a large direct-reply message", () -> {
-        SerialAckedMessageWithBigReply messageWithBigReply = new SerialAckedMessageWithBigReply();
-        await().until(() -> {
-          messageWithBigReply.send(Collections.singleton(vm2ID));
-          return true;
-        });
-      });
-    } finally {
-      invokeInEveryVM(
-          () -> InternalDataSerializer.getDSFIDSerializer().registerDSFID(SERIAL_ACKED_MESSAGE,
-              SerialAckedMessage.class));
+    int locatorPort = createLocator(getVM(0));
+    for (int i = 1; i <= NUM_SERVERS; i++) {
+      createCacheAndRegion(getVM(i), locatorPort);
     }
+    DistributedMember vm2ID =
+        getVM(2).invoke(() -> cache.getDistributedSystem().getDistributedMember());
+    getVM(1).invoke("receive a large direct-reply message", () -> {
+      SerialAckedMessageWithBigReply messageWithBigReply = new SerialAckedMessageWithBigReply();
+      await().until(() -> {
+        messageWithBigReply.send(Collections.singleton(vm2ID));
+        return true;
+      });

Review comment:
       Looks like this code was pre-existing and just moved, but it stands out as a very weird usage of `await`. Does it need to be in an `await`? Seems like it should either be a simple call without `await` or a `Mockito.verify` done on a spy of `messageWithBigReply`. 
   
   If the `await` is supposed to retry the `send` multiple times, then what causes the `send` to not work the first time? Does it throw? If it throws, it would need some form of `ignoreException` clause added to it.

##########
File path: geode-core/src/upgradeTest/java/org/apache/geode/ClusterCommunicationsDUnitTest.java
##########
@@ -246,28 +255,19 @@ public void createEntryWithBigMessage() {
 
   @Test
   public void receiveBigResponse() {
-    invokeInEveryVM(
-        () -> InternalDataSerializer.getDSFIDSerializer().registerDSFID(SERIAL_ACKED_MESSAGE,
-            SerialAckedMessageWithBigReply.class));
-    try {
-      int locatorPort = createLocator(getVM(0));
-      for (int i = 1; i <= NUM_SERVERS; i++) {
-        createCacheAndRegion(getVM(i), locatorPort);
-      }
-      DistributedMember vm2ID =
-          getVM(2).invoke(() -> cache.getDistributedSystem().getDistributedMember());
-      getVM(1).invoke("receive a large direct-reply message", () -> {
-        SerialAckedMessageWithBigReply messageWithBigReply = new SerialAckedMessageWithBigReply();
-        await().until(() -> {
-          messageWithBigReply.send(Collections.singleton(vm2ID));
-          return true;
-        });
-      });
-    } finally {
-      invokeInEveryVM(
-          () -> InternalDataSerializer.getDSFIDSerializer().registerDSFID(SERIAL_ACKED_MESSAGE,
-              SerialAckedMessage.class));
+    int locatorPort = createLocator(getVM(0));
+    for (int i = 1; i <= NUM_SERVERS; i++) {
+      createCacheAndRegion(getVM(i), locatorPort);
     }
+    DistributedMember vm2ID =

Review comment:
       [comment] How about `memberVM2` or `memberInVM2` instead of `vm2ID`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org