boglesby commented on a change in pull request #6441:
URL: https://github.com/apache/geode/pull/6441#discussion_r645213578



##########
File path: 
geode-core/src/main/java/org/apache/geode/internal/cache/LocalRegion.java
##########
@@ -5111,7 +5111,7 @@ public void txApplyPutPart2(RegionEntry regionEntry, 
Object key, long lastModifi
 
   public boolean basicBridgeCreate(final Object key, final byte[] value, 
boolean isObject,
       Object callbackArg, final ClientProxyMembershipID client, boolean 
fromClient,
-      EntryEventImpl clientEvent, boolean throwEntryExists)
+      EntryEventImpl clientEvent, boolean throwEntryExists, boolean 
generateCallbacks)
       throws TimeoutException, EntryExistsException, CacheWriterException {

Review comment:
       LocalRegion.basicBridgeCreate is only ever called with 
generateCallbacks=true
   
   The GatewayReceiverCommand create case uses true
   
   The GatewayReceiverCommand update case conditionally 
LocalRegion.basicBridgePut with false.
   
   I see GatewaySenderEventImpl.initializeAction only sets UPDATE_ACTION or 
UPDATE_ACTION_NO_GENERATE_CALLBACKS:
   ```
   } else if (operation == 
EnumListenerEvent.AFTER_UPDATE_WITH_GENERATE_CALLBACKS) {
     if (event.isGenerateCallbacks()) {
       this.action = UPDATE_ACTION;
     } else {
       this.action = UPDATE_ACTION_NO_GENERATE_CALLBACKS;
     }
   }
   ```
   That causes only basicBridgePut to be invoked in the GatewayReceiverCommand. 
Is that intentional? If so, it doesn't look like you need the new parameter on 
basicBridgeCreate.
   

##########
File path: 
geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/ReplicateRegionFunction.java
##########
@@ -0,0 +1,465 @@
+/*
+ * 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.management.internal.cli.functions;
+
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.FutureTask;
+import java.util.stream.Collectors;
+
+import org.apache.logging.log4j.Logger;
+
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Declarable;
+import org.apache.geode.cache.EntryDestroyedException;
+import org.apache.geode.cache.Operation;
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.client.AllConnectionsInUseException;
+import org.apache.geode.cache.client.NoAvailableServersException;
+import org.apache.geode.cache.client.ServerConnectivityException;
+import org.apache.geode.cache.client.internal.Connection;
+import org.apache.geode.cache.client.internal.PoolImpl;
+import 
org.apache.geode.cache.client.internal.pooling.ConnectionDestroyedException;
+import org.apache.geode.cache.client.internal.pooling.PooledConnection;
+import org.apache.geode.cache.execute.FunctionContext;
+import org.apache.geode.cache.wan.GatewayQueueEvent;
+import org.apache.geode.cache.wan.GatewaySender;
+import org.apache.geode.internal.cache.BucketRegion;
+import org.apache.geode.internal.cache.DefaultEntryEventFactory;
+import org.apache.geode.internal.cache.EntryEventImpl;
+import org.apache.geode.internal.cache.EntrySnapshot;
+import org.apache.geode.internal.cache.EnumListenerEvent;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.internal.cache.InternalRegion;
+import org.apache.geode.internal.cache.NonTXEntry;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.wan.AbstractGatewaySender;
+import org.apache.geode.internal.cache.wan.BatchException70;
+import org.apache.geode.internal.cache.wan.GatewaySenderEventDispatcher;
+import org.apache.geode.internal.cache.wan.GatewaySenderEventImpl;
+import org.apache.geode.internal.cache.wan.InternalGatewaySender;
+import org.apache.geode.logging.internal.executors.LoggingExecutors;
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.management.cli.CliFunction;
+import org.apache.geode.management.internal.functions.CliFunctionResult;
+import org.apache.geode.management.internal.i18n.CliStrings;
+
+/**
+ * Class for WAN replicating the contents of a region
+ * It must be executed in all members of the Geode cluster that host the region
+ * to be replicated. (called with onServers() or withMembers() passing the list
+ * of all members hosting the region).
+ * It also offers the possibility to cancel an ongoing execution of this 
function.
+ * The replication itself is executed in a new thread with a known name
+ * (parameterized with the regionName and senderId) in order to allow
+ * to cancel ongoing invocations by interrupting that thread.
+ *
+ * It accepts the following arguments in an array of objects
+ * 0: regionName (String)
+ * 1: senderId (String)
+ * 2: isCancel (Boolean): If true, it indicates that an ongoing execution of 
this
+ * function for the given region and senderId must be stopped. Otherwise,
+ * it indicates that the region must be replicated.
+ * 3: maxRate (Long) maximum replication rate in entries per second. In the 
case of
+ * parallel gateway senders, the maxRate is per server hosting the region.
+ * 4: batchSize (Integer): the size of the batches. Region entries are 
replicated in batches of the
+ * passed size. After each batch is sent, the function checks if the command
+ * must be canceled and also sleeps for some time if necessary to adjust the
+ * replication rate to the one passed as argument.
+ */
+public class ReplicateRegionFunction extends CliFunction<Object[]> implements 
Declarable {
+  private static final Logger logger = LogService.getLogger();
+  private static final long serialVersionUID = 1L;
+
+  public static final String ID = ReplicateRegionFunction.class.getName();
+
+  private static final int MAX_BATCH_SEND_RETRIES = 1;
+
+  private Clock clock = Clock.systemDefaultZone();
+  private ThreadSleeper threadSleeper = new ThreadSleeper();
+
+  static class ThreadSleeper implements Serializable {
+    void millis(long millis) throws InterruptedException {
+      Thread.sleep(millis);
+    }
+  }
+
+  @VisibleForTesting
+  public void setClock(Clock clock) {
+    this.clock = clock;
+  }
+
+  @VisibleForTesting
+  public void setThreadSleeper(ThreadSleeper ts) {
+    this.threadSleeper = ts;
+  }
+
+  @Override
+  public String getId() {
+    return ID;
+  }
+
+  @Override
+  public boolean hasResult() {
+    return true;
+  }
+
+  @Override
+  public boolean isHA() {
+    return false;
+  }
+
+  @Override
+  public CliFunctionResult executeFunction(FunctionContext<Object[]> context) {
+    final Object[] args = context.getArguments();
+    if (args.length < 5) {
+      throw new IllegalStateException(
+          "Arguments length does not match required length.");
+    }
+    final String regionName = (String) args[0];
+    final String senderId = (String) args[1];
+    final boolean isCancel = (Boolean) args[2];
+    long maxRate = (Long) args[3];
+    int batchSize = (Integer) args[4];
+
+    final InternalCache cache = (InternalCache) context.getCache();
+
+    if (isCancel) {
+      return cancelReplicateRegion(context, regionName, senderId);
+    }
+
+    final Region region = cache.getRegion(regionName);
+    if (region == null) {
+      return new CliFunctionResult(context.getMemberName(), 
CliFunctionResult.StatusState.ERROR,
+          
CliStrings.format(CliStrings.REPLICATE_REGION__MSG__REGION__NOT__FOUND, 
regionName));
+    }
+
+    GatewaySender sender = cache.getGatewaySender(senderId);
+    if (sender == null) {
+      return new CliFunctionResult(context.getMemberName(), 
CliFunctionResult.StatusState.ERROR,
+          
CliStrings.format(CliStrings.REPLICATE_REGION__MSG__SENDER__NOT__FOUND, 
senderId));
+    }
+
+    if (sender.isParallel() && !(region instanceof PartitionedRegion)) {
+      return new CliFunctionResult(context.getMemberName(), 
CliFunctionResult.StatusState.ERROR,
+          
CliStrings.REPLICATE_REGION__MSG__CANNOT__REPLICATE__NON__PARTITIONED__REGION__WITH__PARALLEL__SENDER);
+    }
+
+    if (!sender.isRunning()) {
+      return new CliFunctionResult(context.getMemberName(), 
CliFunctionResult.StatusState.ERROR,
+          
CliStrings.format(CliStrings.REPLICATE_REGION__MSG__SENDER__NOT__RUNNING, 
senderId));
+    }
+
+    if (!sender.isParallel() && !((InternalGatewaySender) sender).isPrimary()) 
{
+      return new CliFunctionResult(context.getMemberName(), 
CliFunctionResult.StatusState.OK,
+          
CliStrings.format(CliStrings.REPLICATE_REGION__MSG__SENDER__SERIAL__AND__NOT__PRIMARY,
+              senderId));
+    }
+
+    try {
+      return executeReplicateFunctionInNewThread(context, region, regionName, 
sender, maxRate,
+          batchSize);
+    } catch (InterruptedException e) {
+      return new CliFunctionResult(context.getMemberName(), 
CliFunctionResult.StatusState.ERROR,
+          CliStrings.REPLICATE_REGION__MSG__EXECUTION__CANCELED);
+    } catch (ExecutionException e) {
+      Writer buffer = new StringWriter();
+      PrintWriter pw = new PrintWriter(buffer);
+      e.printStackTrace(pw);
+      logger.error("Exception when running replicate command: {}", 
buffer.toString());
+      return new CliFunctionResult(context.getMemberName(), 
CliFunctionResult.StatusState.ERROR,
+          
CliStrings.format(CliStrings.REPLICATE_REGION__MSG__EXECUTION__FAILED, 
e.getMessage()));
+    }
+  }
+
+  private CliFunctionResult 
executeReplicateFunctionInNewThread(FunctionContext<Object[]> context,
+      Region region, String regionName, GatewaySender sender, long maxRate, 
int batchSize)
+      throws InterruptedException, ExecutionException {
+    Callable<CliFunctionResult> callable =
+        new ReplicateRegionCallable(context, region, sender, maxRate, 
batchSize);
+    FutureTask<CliFunctionResult> futureTask = new FutureTask<>(callable);
+    ExecutorService executor = LoggingExecutors
+        
.newSingleThreadExecutor(getReplicateRegionFunctionThreadName(regionName,
+            sender.getId()), true);
+    executor.execute(futureTask);

Review comment:
       Should ReplicateRegionFunction create an ExecutorService when it is 
instantiated rather than creating one for every 
executeReplicateFunctionInNewThread call?

##########
File path: 
geode-core/src/main/java/org/apache/geode/internal/cache/wan/GatewaySenderEventImpl.java
##########
@@ -1094,6 +1096,13 @@ protected void initializeAction(EnumListenerEvent 
operation) {
       // Initialize number of parts
       // Since there is no value, there is one less part
       this.numberOfParts = (this.callbackArgument == null) ? 7 : 8;
+    } else if (operation == 
EnumListenerEvent.AFTER_UPDATE_WITH_GENERATE_CALLBACKS) {
+      if (event.isGenerateCallbacks()) {
+        this.action = UPDATE_ACTION;
+      } else {
+        this.action = UPDATE_ACTION_NO_GENERATE_CALLBACKS;
+      }
+      this.numberOfParts = (this.callbackArgument == null) ? 8 : 9;
     }

Review comment:
       How does event.isGenerateCallbacks() get to be true? I only see it be 
false in my quick test.

##########
File path: 
geode-core/src/main/java/org/apache/geode/internal/cache/tier/sockets/command/GatewayReceiverCommand.java
##########
@@ -350,6 +352,7 @@ public void cmdExecute(final Message clientMessage, final 
ServerConnection serve
               break;
 
             case 1: // Update
+            case GatewaySenderEventImpl.UPDATE_ACTION_NO_GENERATE_CALLBACKS:
               try {

Review comment:
       What happens if the other site is an older version? We need to verify 
there aren't any versioning issues with the changes to GatewaySenderEventImpl 
and GatewayReceiverCommand.
   




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

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


Reply via email to