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/12 20:06:14 UTC

[GitHub] [geode] kirklund commented on a change in pull request #6036: GEODE-8856: Persist gateway-sender state

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



##########
File path: geode-core/src/main/java/org/apache/geode/cache/configuration/CacheConfig.java
##########
@@ -2642,6 +2642,8 @@ public void setOverflowDirectory(String value) {
     protected String batchTimeInterval;
     @XmlAttribute(name = "enable-persistence")
     protected Boolean enablePersistence;
+    @XmlAttribute(name = "state")
+    protected String state;

Review comment:
       `state` is too ambiguous. You should call it something more specific like `gatewaySenderState` or `asyncEventQueueState`.

##########
File path: geode-core/src/main/java/org/apache/geode/cache/wan/GatewaySenderState.java
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.cache.wan;
+
+/*
+ * GatewaySenderState doesn't reflect current state, but the desired state of GatewaySender
+ * at the startup of the member. This state is only persisted within cluster configuration when
+ * start, stop, pause or resume gateway-sender command is successfully executed. When member starts
+ * again, it will read the state from cluster configuration and bring gateway-sender
+ * that desired state.
+ */
+public enum GatewaySenderState {

Review comment:
       GatewaySenderState is ok. Is it only used to specify the state of the sender at startup? Another approach might be something like GatewaySenderStartupAction with values like START, PAUSE, NONE.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
##########
@@ -185,6 +188,16 @@
 
   protected volatile ConcurrentLinkedQueue<EntryEventImpl> tmpDroppedEvents =
       new ConcurrentLinkedQueue<>();
+
+  /**
+   * Contains all events that were stored in temporary dropped events queue when
+   * hook to collect tmp dropped events is enabled
+   */
+  private volatile ConcurrentLinkedQueue<EntryEventImpl> hookForTmpDroppedEvents;
+
+  @MutableForTesting
+  public static boolean ENABLE_HOOK_TMP_DROPPED_EVENTS = false;

Review comment:
       A cleaner approach is probably to use a SystemProperty and set a `private final boolean` field based on the value of the system property. We're trying to remove all statics with these annotations.
   
   Tests can then set the system property in the Server VM before starting anything up in that VM (Server, Cache, GatewaySender). If you're using some form of `ClusterStartupRule`, you could always set the system property in any VM within an `@BeforeClass` method.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/parallel/ConcurrentParallelGatewaySenderEventProcessor.java
##########
@@ -71,6 +71,11 @@
 
   public ConcurrentParallelGatewaySenderEventProcessor(AbstractGatewaySender sender,
       ThreadsMonitoring tMonitoring, boolean cleanQueues) {
+    this(sender, tMonitoring, cleanQueues, false);
+  }
+
+  public ConcurrentParallelGatewaySenderEventProcessor(AbstractGatewaySender sender,
+      ThreadsMonitoring tMonitoring, boolean cleanQueues, boolean shouldOnlyRecoverQueues) {

Review comment:
       I wouldn't use "should" in the name of the parameter. Use something more imperative like `recoverQueuesOnly`.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/AbstractGatewaySender.java
##########
@@ -1180,17 +1203,28 @@ private void recordDroppedEvent(EntryEventImpl event) {
       eventProcessor.registerEventDroppedInPrimaryQueue(event);
     } else {
       tmpDroppedEvents.add(event);
+      if (ENABLE_HOOK_TMP_DROPPED_EVENTS) {
+        if (hookForTmpDroppedEvents == null) {
+          hookForTmpDroppedEvents = new ConcurrentLinkedQueue<>();
+        }
+        hookForTmpDroppedEvents.add(event);
+      }
       if (logger.isDebugEnabled()) {
         logger.debug("added to tmpDroppedEvents event: {}", event);
       }
     }
   }
 
   @VisibleForTesting
-  int getTmpDroppedEventSize() {
+  public int getTmpDroppedEventSize() {

Review comment:
       I think you should rename this method to `getTempDroppedEventSize` and change the fields from `tmp` to `temp` as well.
   
   If you can arrange for the tests to be in the same package, it would be better for these `@VisibleForTesting` methods to be package-private or protected instead of public. This is mainly because `AbstractGatewaySender` is the impl of the class that we expose to Users. This keeps it looking cleaner to some Users who may use Reflection, a debugger, IDE auto-completion, etc.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/PartitionedRegion.java
##########
@@ -1192,10 +1193,14 @@ public void postCreateRegion() {
            * get the ParallelGatewaySender to create the colocated partitioned region for this
            * region.
            */
+          AbstractGatewaySender senderImpl = (AbstractGatewaySender) sender;

Review comment:
       [comment] I know this was just moved, but I wanted to remind you to use interfaces wherever possible. `AbstractGatewaySender` can be replaced by `InternalGatewaySender`.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/InternalGatewaySender.java
##########
@@ -46,7 +46,7 @@
 
   void destroy(boolean initiator);
 
-  void setStartEventProcessorInPausedState();
+  void setStartEventProcessorInPausedState(boolean isPaused);

Review comment:
       The parameter makes this method `void setStartEventProcessorInPausedState(boolean isPaused)` pretty weird. The method name says its setting it "InPausedState" but then the parameters changes that intention.
   
   I think the method should be renamed to `void setStartEventProcessor(boolean isPaused)`.

##########
File path: geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderAttributes.java
##########
@@ -41,6 +42,8 @@
 
   private int batchTimeInterval = GatewaySender.DEFAULT_BATCH_TIME_INTERVAL;
 
+  public GatewaySenderState state;
+

Review comment:
       This should `private` with a setter method to set it.




-- 
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