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/07/01 21:44:10 UTC

[GitHub] [geode] boglesby commented on a change in pull request #6601: GEODE-9369: Command to copy region entries from a WAN site to another

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



##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/functions/WanCopyRegionFunction.java
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.Serializable;
+import java.time.Clock;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+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.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.internal.serialization.KnownVersion;
+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 copying via WAN the contents of a region
+ * It must be executed in all members of the Geode cluster that host the region
+ * to be copied. (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 copying 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 copied.
+ * 3: maxRate (Long) maximum copy 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 copied 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
+ * copy rate to the one passed as argument.
+ */
+public class WanCopyRegionFunction extends CliFunction<Object[]> implements Declarable {
+  private static final Logger logger = LogService.getLogger();
+  private static final long serialVersionUID = 1L;
+
+  public static final String ID = WanCopyRegionFunction.class.getName();
+
+  private static final int MAX_BATCH_SEND_RETRIES = 1;
+
+  private static final int THREAD_POOL_SIZE = 10;
+
+  private final Clock clock;
+  private final ThreadSleeper threadSleeper;
+
+  private static final ExecutorService executor = LoggingExecutors
+      .newFixedThreadPool(THREAD_POOL_SIZE, "wanCopyRegionFunctionThread_", true);
+
+  /**
+   * Contains the ongoing executions of this function
+   */
+  private static final Map<String, Future> executions = new ConcurrentHashMap<>();
+
+  private volatile int batchId = 0;
+
+  public WanCopyRegionFunction() {
+    this(Clock.systemDefaultZone(), new ThreadSleeper());
+  }
+
+  @VisibleForTesting
+  WanCopyRegionFunction(Clock clock, ThreadSleeper threadSleeper) {
+    this.clock = clock;
+    this.threadSleeper = threadSleeper;
+  }
+
+  @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];
+
+    if (regionName.endsWith("*") && senderId.equals("*") && isCancel) {
+      return cancelAllWanCopyRegion(context);
+    }
+
+    if (isCancel) {
+      return cancelWanCopyRegion(context, regionName, senderId);
+    }
+
+    final InternalCache cache = (InternalCache) context.getCache();
+
+    final Region region = cache.getRegion(regionName);
+    if (region == null) {
+      return new CliFunctionResult(context.getMemberName(), CliFunctionResult.StatusState.ERROR,
+          CliStrings.format(CliStrings.WAN_COPY_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.WAN_COPY_REGION__MSG__SENDER__NOT__FOUND, senderId));
+    }
+
+    if (!region.getAttributes().getGatewaySenderIds().contains(sender.getId())) {
+      return new CliFunctionResult(context.getMemberName(), CliFunctionResult.StatusState.ERROR,
+          CliStrings.format(CliStrings.WAN_COPY_REGION__MSG__REGION__NOT__USING_SENDER, regionName,
+              senderId));
+    }
+
+    if (!sender.isRunning()) {
+      return new CliFunctionResult(context.getMemberName(), CliFunctionResult.StatusState.ERROR,
+          CliStrings.format(CliStrings.WAN_COPY_REGION__MSG__SENDER__NOT__RUNNING, senderId));
+    }
+
+    if (!sender.isParallel() && !((InternalGatewaySender) sender).isPrimary()) {
+      return new CliFunctionResult(context.getMemberName(), CliFunctionResult.StatusState.OK,
+          CliStrings.format(CliStrings.WAN_COPY_REGION__MSG__SENDER__SERIAL__AND__NOT__PRIMARY,
+              senderId));
+    }
+
+    try {
+      return executeWanCopyRegionFunctionInNewThread(context, region, regionName, sender, maxRate,
+          batchSize);
+    } catch (InterruptedException | CancellationException e) {
+      return new CliFunctionResult(context.getMemberName(), CliFunctionResult.StatusState.ERROR,
+          CliStrings.WAN_COPY_REGION__MSG__CANCELED__BEFORE__HAVING__COPIED);
+    } catch (ExecutionException e) {
+      return new CliFunctionResult(context.getMemberName(), CliFunctionResult.StatusState.ERROR,
+          CliStrings.format(CliStrings.WAN_COPY_REGION__MSG__EXECUTION__FAILED, e.getMessage()));
+    }
+  }
+
+  private CliFunctionResult executeWanCopyRegionFunctionInNewThread(
+      FunctionContext<Object[]> context,
+      Region region, String regionName, GatewaySender sender, long maxRate, int batchSize)
+      throws InterruptedException, ExecutionException, CancellationException {
+    String executionName = getExecutionName(regionName, sender.getId());
+    Callable<CliFunctionResult> callable =
+        new wanCopyRegionCallable(context, region, sender, maxRate, batchSize);
+    FutureTask<CliFunctionResult> future = new FutureTask<>(callable);
+
+    if (executions.putIfAbsent(executionName, future) != null) {
+      return new CliFunctionResult(context.getMemberName(), CliFunctionResult.StatusState.ERROR,
+          CliStrings.format(CliStrings.WAN_COPY_REGION__MSG__ALREADY__RUNNING__COMMAND,
+              regionName, sender.getId()));
+    }
+
+    try {
+      executor.execute(future);
+      return future.get();
+    } finally {
+      executions.remove(executionName);
+    }
+  }
+
+  class wanCopyRegionCallable implements Callable<CliFunctionResult> {

Review comment:
       This class name should start with a capital 'W' and maybe be static.

##########
File path: geode-gfsh/src/main/java/org/apache/geode/management/internal/cli/commands/WanCopyRegionCommand.java
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.commands;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.springframework.shell.core.annotation.CliCommand;
+import org.springframework.shell.core.annotation.CliOption;
+
+import org.apache.geode.cache.execute.FunctionInvocationTargetException;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.management.cli.CliMetaData;
+import org.apache.geode.management.cli.ConverterHint;
+import org.apache.geode.management.cli.GfshCommand;
+import org.apache.geode.management.internal.cli.functions.WanCopyRegionFunction;
+import org.apache.geode.management.internal.cli.result.model.ResultModel;
+import org.apache.geode.management.internal.functions.CliFunctionResult;
+import org.apache.geode.management.internal.i18n.CliStrings;
+import org.apache.geode.security.ResourcePermission.Operation;
+import org.apache.geode.security.ResourcePermission.Resource;
+
+public class WanCopyRegionCommand extends GfshCommand {
+  private final WanCopyRegionFunction wanCopyRegionFunction = new WanCopyRegionFunction();
+
+  @CliCommand(value = CliStrings.WAN_COPY_REGION, help = CliStrings.WAN_COPY_REGION__HELP)
+  @CliMetaData(relatedTopic = {CliStrings.TOPIC_GEODE_DATA, CliStrings.TOPIC_GEODE_REGION})
+  public ResultModel wanCopyRegion(
+      @CliOption(key = CliStrings.WAN_COPY_REGION__REGION, mandatory = true,
+          optionContext = ConverterHint.REGION_PATH,
+          help = CliStrings.WAN_COPY_REGION__REGION__HELP) String regionName,
+      @CliOption(key = CliStrings.WAN_COPY_REGION__SENDERID,
+          optionContext = ConverterHint.GATEWAY_SENDER_ID,
+          help = CliStrings.WAN_COPY_REGION__SENDERID__HELP) String senderId,

Review comment:
       This is along the same lines as Jinmei's comment below.
   
   If I execute the command with no sender id (which is currently not mandatory), I see:
   ```
   Executing - wan-copy region --region=/data_pr_parallel
   
   Member | Status | Message
   ------ | ------ | ---------------------
   ln-1   | ERROR  | Sender null not found
   ```
   I think the sender id should be mandatory:
   ```
   @CliOption(key = CliStrings.WAN_COPY_REGION__SENDERID, mandatory = true,
       optionContext = ConverterHint.GATEWAY_SENDER_ID,
       help = CliStrings.WAN_COPY_REGION__SENDERID__HELP) String senderId,
   ```
   That provides a better message:
   ```
   Executing - wan-copy region --region=/data_pr_parallel
   
   Jul 01, 2021 1:42:01 PM org.springframework.shell.core.SimpleParser printHintMessage
   WARNING: You should specify option (--sender-id, --max-rate, --batch-size, --cancel) for this command
     --sender-id=  is required
   Use "help wan-copy region" (without the quotes) for detailed usage information.
   ```
   




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