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 2022/01/03 13:54:19 UTC

[GitHub] [geode] Kris-10-0 opened a new pull request #7227: feature/GEODE-9832: SMOVE Command Support

Kris-10-0 opened a new pull request #7227:
URL: https://github.com/apache/geode/pull/7227


   <!-- Thank you for submitting a contribution to Apache Geode. -->
   
   <!-- In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken: 
   -->
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced in the commit message?
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `develop`)?
   
   - [x] Is your initial contribution a single, squashed commit?
   
   - [x] Does `gradlew build` run cleanly?
   
   - [x] Have you written or updated unit tests to verify your changes?
   
   - [NA] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   
   <!-- Note:
   Please ensure that once the PR is submitted, check Concourse for build issues and
   submit an update to your PR as soon as possible. If you need help, please send an
   email to dev@geode.apache.org.
   -->
   


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



[GitHub] [geode] DonalEvans commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r779211279



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SMoveExecutor.java
##########
@@ -14,47 +14,41 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
-import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
-import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisSet.smove;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.geode.cache.Region;
 import org.apache.geode.redis.internal.commands.Command;
 import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
 import org.apache.geode.redis.internal.commands.executor.RedisResponse;
-import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisDataMovedException;
 import org.apache.geode.redis.internal.data.RedisKey;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
 
 public class SMoveExecutor implements CommandExecutor {
 
   @Override
   public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<RedisKey, RedisData> region = context.getRegion();
-    RedisKey source = command.getKey();
-    RedisKey destination = new RedisKey(commandElems.get(2));
-    byte[] member = commandElems.get(3);
-
-    // TODO: this command should lock both source and destination before changing them
+    List<RedisKey> commandKeys = command.getProcessedCommandKeys();
+    List<RedisKey> setKeys = commandKeys.subList(1, 3);
 
-    String destinationType = context.dataLockedExecute(destination, false, RedisData::type);
-    if (!destinationType.equals(REDIS_SET.toString()) && !destinationType.equals("none")) {
-      return RedisResponse.wrongType(ERROR_WRONG_TYPE);
+    byte[] member = commandElems.get(3);
+    RegionProvider regionProvider = context.getRegionProvider();
+    try {
+      for (RedisKey k : setKeys) {
+        regionProvider.ensureKeyIsLocal(k);

Review comment:
       Sounds good




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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r779181496



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SMoveExecutor.java
##########
@@ -14,47 +14,41 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
-import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
-import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisSet.smove;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.geode.cache.Region;
 import org.apache.geode.redis.internal.commands.Command;
 import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
 import org.apache.geode.redis.internal.commands.executor.RedisResponse;
-import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisDataMovedException;
 import org.apache.geode.redis.internal.data.RedisKey;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
 
 public class SMoveExecutor implements CommandExecutor {
 
   @Override
   public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<RedisKey, RedisData> region = context.getRegion();
-    RedisKey source = command.getKey();
-    RedisKey destination = new RedisKey(commandElems.get(2));
-    byte[] member = commandElems.get(3);
-
-    // TODO: this command should lock both source and destination before changing them
+    List<RedisKey> commandKeys = command.getProcessedCommandKeys();
+    List<RedisKey> setKeys = commandKeys.subList(1, 3);
 
-    String destinationType = context.dataLockedExecute(destination, false, RedisData::type);
-    if (!destinationType.equals(REDIS_SET.toString()) && !destinationType.equals("none")) {
-      return RedisResponse.wrongType(ERROR_WRONG_TYPE);
+    byte[] member = commandElems.get(3);
+    RegionProvider regionProvider = context.getRegionProvider();
+    try {
+      for (RedisKey k : setKeys) {
+        regionProvider.ensureKeyIsLocal(k);

Review comment:
       Let's leave it like this. I've opened a Jira to standardize this a bit - we can push this check down to where we've gathered all the keys to lock before executing the actual command (in `RegionProvider`). https://issues.apache.org/jira/browse/GEODE-9922




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



[GitHub] [geode] DonalEvans commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r779218431



##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSMoveIntegrationTest.java
##########
@@ -163,24 +172,53 @@ public void ensureSetConsistency_whenRunningConcurrently() {
         i -> jedis.srem(sourceKey, movedMember),
         i -> moved.set(jedis.smove(sourceKey, destKey, movedMember)))
             .runWithAction(() -> {
-              // Check sdiffstore return size of diff
-              assertThat(moved).satisfiesAnyOf(
-                  smoveResult -> assertThat(smoveResult.get()).isEqualTo(0),
-                  smoveResult -> assertThat(smoveResult.get()).isEqualTo(1));
-              // Checks if values were moved or not from source key
-              assertThat(sourceKey).satisfiesAnyOf(
-                  source -> assertThat(jedis.smembers(source))
-                      .containsExactlyInAnyOrder(sourceMembers),
-                  source -> assertThat(jedis.smembers(source))
-                      .containsExactlyInAnyOrder(sourceMemberRemoved));
-              // Checks if values were moved or not to destination key
-              assertThat(destKey).satisfiesAnyOf(
-                  dest -> assertThat(jedis.smembers(dest))
-                      .containsExactlyInAnyOrder(destMembers),
-                  dest -> assertThat(jedis.smembers(dest))
-                      .containsExactlyInAnyOrder(destMemberAdded));
+              if (moved.get() == 1) {
+                assertThat(jedis.smembers(sourceKey))
+                    .containsExactlyInAnyOrder(sourceMemberRemoved);
+                assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(destMemberAdded);
+              } else {
+                assertThat(jedis.smembers(sourceKey))
+                    .containsExactlyInAnyOrder(sourceMemberRemoved);
+                assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(destMembers);
+              }
+              jedis.sadd(sourceKey, movedMember);
+              jedis.srem(destKey, movedMember);
+            });
+  }
+
+  @Test
+  public void ensureSetConsistency_whenRunningConcurrently_withSMovesFromSameSourceAndDifferentDestination() {
+    String[] sourceMemberRemoved = ArrayUtils.remove(sourceMembers, 0);
+    String[] destMemberAdded = ArrayUtils.add(destMembers, movedMember);
+    String[] nonExisistentMemberAdded = {movedMember};
+
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.sadd(destKey, destMembers);
+
+    final AtomicLong movedToNonExistent = new AtomicLong(0);
+    final AtomicLong movedToDest = new AtomicLong(0);
+    new ConcurrentLoopingThreads(1000,
+        i -> movedToNonExistent.set(jedis.smove(sourceKey, nonExistentSetKey, movedMember)),
+        i -> movedToDest.set(jedis.smove(sourceKey, destKey, movedMember)))
+            .runWithAction(() -> {
+              assertThat(jedis.smembers(sourceKey)).containsExactlyInAnyOrder(sourceMemberRemoved);
+
+              if (movedToNonExistent.get() == 1) {
+                assertThat(jedis.smembers(nonExistentSetKey))
+                    .containsExactlyInAnyOrder(nonExisistentMemberAdded);
+              } else {
+                assertThat(jedis.exists(nonExistentSetKey)).isFalse();
+              }
+
+              if (movedToDest.get() == 1) {
+                assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(destMemberAdded);
+              } else {
+                assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(destMembers);
+              }

Review comment:
       It would be good to add an assertion here that only one of the moves returned 1, since it shouldn't be possible for both moves to succeed.




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



[GitHub] [geode] DonalEvans commented on pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on pull request #7227:
URL: https://github.com/apache/geode/pull/7227#issuecomment-1006175157


   It looks like there's a failing tcl test due to incorrect behaviour when the destination key exists but is not a Set and the member being moved is not present in the source set. The following test passes when run against native Redis but fails when run against geode-for-redis:
   ```
     @Test
     public void smove_withNonSetDestinationKey_returnsWrongtypeError() {
       jedis.sadd(sourceKey, sourceMembers);
       jedis.set(destKey, "aString");
   
       assertThatThrownBy(() -> jedis.smove(sourceKey, destKey, "nonPresentMember"))
           .hasMessageContaining("WRONGTYPE");
     }
   ```
   
   The solution to this is to move this line:
   ```
   RedisSet destination = regionProvider.getTypedRedisData(REDIS_SET, destKey, false);
   ```
   to be immediately below this one:
   ```
   RedisSet source = regionProvider.getTypedRedisData(REDIS_SET, sourceKey, false);
   ``` 
   in `RedisSet.smove()`


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



[GitHub] [geode] Kris-10-0 commented on pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
Kris-10-0 commented on pull request #7227:
URL: https://github.com/apache/geode/pull/7227#issuecomment-1006195062


   Forced pushed to rebase


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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r778118879



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SMoveExecutor.java
##########
@@ -14,47 +14,41 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
-import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
-import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisSet.smove;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.geode.cache.Region;
 import org.apache.geode.redis.internal.commands.Command;
 import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
 import org.apache.geode.redis.internal.commands.executor.RedisResponse;
-import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisDataMovedException;
 import org.apache.geode.redis.internal.data.RedisKey;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
 
 public class SMoveExecutor implements CommandExecutor {
 
   @Override
   public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<RedisKey, RedisData> region = context.getRegion();
-    RedisKey source = command.getKey();
-    RedisKey destination = new RedisKey(commandElems.get(2));
-    byte[] member = commandElems.get(3);
-
-    // TODO: this command should lock both source and destination before changing them
+    List<RedisKey> commandKeys = command.getProcessedCommandKeys();
+    List<RedisKey> setKeys = commandKeys.subList(1, 3);
 
-    String destinationType = context.dataLockedExecute(destination, false, RedisData::type);
-    if (!destinationType.equals(REDIS_SET.toString()) && !destinationType.equals("none")) {
-      return RedisResponse.wrongType(ERROR_WRONG_TYPE);
+    byte[] member = commandElems.get(3);
+    RegionProvider regionProvider = context.getRegionProvider();
+    try {
+      for (RedisKey k : setKeys) {
+        regionProvider.ensureKeyIsLocal(k);
+      }
+    } catch (RedisDataMovedException ex) {
+      return RedisResponse.error(ex.getMessage());

Review comment:
       I was curious about this and it looks like native Redis returns a `CROSSSLOT` error. So this should be:
   ```
   return RedisResponse.crossSlot(ERROR_WRONG_SLOT);
   ```




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



[GitHub] [geode] Kris-10-0 commented on pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
Kris-10-0 commented on pull request #7227:
URL: https://github.com/apache/geode/pull/7227#issuecomment-1007758972


   Forced pushed to rebase


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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r777708262



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSet.java
##########
@@ -69,6 +69,30 @@ public RedisSet(int expectedSize) {
    */
   public RedisSet() {}
 
+  public static int smove(RegionProvider regionProvider, List<RedisKey> keys, byte[] member) {
+    Region<RedisKey, RedisData> region = regionProvider.getDataRegion();
+    RedisKey sourceKey = keys.get(0);
+    RedisKey destKey = keys.get(1);
+
+    RedisSet source = regionProvider.getTypedRedisData(REDIS_SET, sourceKey, false);
+    if (source.scard() == 0 || !source.members.contains(member)) {
+      return 0;
+    }
+
+    RedisSet destination = regionProvider.getTypedRedisData(REDIS_SET, keys.get(1), false);

Review comment:
       should this use "destKey" instead of "keys.get(1)"?




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



[GitHub] [geode] Kris-10-0 closed pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
Kris-10-0 closed pull request #7227:
URL: https://github.com/apache/geode/pull/7227


   


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



[GitHub] [geode] Kris-10-0 closed pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
Kris-10-0 closed pull request #7227:
URL: https://github.com/apache/geode/pull/7227


   


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



[GitHub] [geode] DonalEvans commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r778377500



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SMoveExecutor.java
##########
@@ -14,47 +14,41 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
-import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
-import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisSet.smove;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.geode.cache.Region;
 import org.apache.geode.redis.internal.commands.Command;
 import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
 import org.apache.geode.redis.internal.commands.executor.RedisResponse;
-import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisDataMovedException;
 import org.apache.geode.redis.internal.data.RedisKey;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
 
 public class SMoveExecutor implements CommandExecutor {
 
   @Override
   public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<RedisKey, RedisData> region = context.getRegion();
-    RedisKey source = command.getKey();
-    RedisKey destination = new RedisKey(commandElems.get(2));
-    byte[] member = commandElems.get(3);
-
-    // TODO: this command should lock both source and destination before changing them
+    List<RedisKey> commandKeys = command.getProcessedCommandKeys();
+    List<RedisKey> setKeys = commandKeys.subList(1, 3);
 
-    String destinationType = context.dataLockedExecute(destination, false, RedisData::type);
-    if (!destinationType.equals(REDIS_SET.toString()) && !destinationType.equals("none")) {
-      return RedisResponse.wrongType(ERROR_WRONG_TYPE);
+    byte[] member = commandElems.get(3);
+    RegionProvider regionProvider = context.getRegionProvider();
+    try {
+      for (RedisKey k : setKeys) {
+        regionProvider.ensureKeyIsLocal(k);

Review comment:
       It's not enough to just check that both keys are local. Rather, we need to check that both keys map to the same slot, and that the bucket associated with that slot is local.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SMoveExecutor.java
##########
@@ -14,47 +14,41 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
-import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
-import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisSet.smove;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.geode.cache.Region;
 import org.apache.geode.redis.internal.commands.Command;
 import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
 import org.apache.geode.redis.internal.commands.executor.RedisResponse;
-import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisDataMovedException;
 import org.apache.geode.redis.internal.data.RedisKey;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
 
 public class SMoveExecutor implements CommandExecutor {
 
   @Override
   public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<RedisKey, RedisData> region = context.getRegion();
-    RedisKey source = command.getKey();
-    RedisKey destination = new RedisKey(commandElems.get(2));
-    byte[] member = commandElems.get(3);
-
-    // TODO: this command should lock both source and destination before changing them
+    List<RedisKey> commandKeys = command.getProcessedCommandKeys();
+    List<RedisKey> setKeys = commandKeys.subList(1, 3);

Review comment:
       Rather than creating a `RedisKey` for every element in `command`, then only using a subset of them, it would be better to explicitly get the source key and destination key byte arrays and create a `RedisKey` for each, similar to how the rename command is implemented in `AbstractRenameExecutor`. This results in slightly more verbose code, as you can't iterate the list of keys and would have to pass each individually into the `smove()` method, but since we know there are only two, this isn't a huge problem.
   
   Doing it this way would also mean that you don't have to create a second ArrayList for use by the `lockedExecute()` method, since the `smove()` method would no longer be working on the same list as the one passed to `lockedExecute()` and so modification of that list is no longer an issue, so this would result in slightly smaller memory overhead.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSet.java
##########
@@ -69,6 +69,30 @@ public RedisSet(int expectedSize) {
    */
   public RedisSet() {}
 
+  public static int smove(RegionProvider regionProvider, List<RedisKey> keys, byte[] member) {
+    Region<RedisKey, RedisData> region = regionProvider.getDataRegion();
+    RedisKey sourceKey = keys.get(0);
+    RedisKey destKey = keys.get(1);
+
+    RedisSet source = regionProvider.getTypedRedisData(REDIS_SET, sourceKey, false);
+    if (source.scard() == 0 || !source.members.contains(member)) {

Review comment:
       This would be more consistent as:
   ```
   if (source.members.isEmpty() || !source.members.contains(member)) {
   ```
   rather than mixing use of internal and public access to the `members` collection.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSMoveIntegrationTest.java
##########
@@ -34,16 +33,19 @@
 import org.apache.geode.redis.ConcurrentLoopingThreads;
 import org.apache.geode.redis.RedisIntegrationTest;
 import org.apache.geode.redis.internal.RedisConstants;
-import org.apache.geode.test.awaitility.GeodeAwaitility;
 
 public abstract class AbstractSMoveIntegrationTest implements RedisIntegrationTest {

Review comment:
       It would be good to also have a test showing the behaviour when the source and destination key do not map to the same slot.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSet.java
##########
@@ -69,6 +69,30 @@ public RedisSet(int expectedSize) {
    */
   public RedisSet() {}
 
+  public static int smove(RegionProvider regionProvider, List<RedisKey> keys, byte[] member) {
+    Region<RedisKey, RedisData> region = regionProvider.getDataRegion();
+    RedisKey sourceKey = keys.get(0);
+    RedisKey destKey = keys.get(1);
+
+    RedisSet source = regionProvider.getTypedRedisData(REDIS_SET, sourceKey, false);
+    if (source.scard() == 0 || !source.members.contains(member)) {
+      return 0;
+    }
+
+    RedisSet destination = regionProvider.getTypedRedisData(REDIS_SET, keys.get(1), false);
+
+    List<byte[]> movedMember = new ArrayList<>();
+    movedMember.add(member);
+    source.srem(movedMember, region, sourceKey);
+    if (destination.scard() == 0) {
+      region.put(destKey, new RedisSet(movedMember));
+    } else {
+      destination.sadd(movedMember, region, destKey);
+    }

Review comment:
       Because of the implementation of `sadd()` in `NullRedisSet`, this if/else is not necessary. Just calling `destination.sadd(movedMember, region, destKey);` will result in the correct behaviour in both cases.




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



[GitHub] [geode] DonalEvans commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r778984894



##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSMoveIntegrationTest.java
##########
@@ -34,16 +33,19 @@
 import org.apache.geode.redis.ConcurrentLoopingThreads;
 import org.apache.geode.redis.RedisIntegrationTest;
 import org.apache.geode.redis.internal.RedisConstants;
-import org.apache.geode.test.awaitility.GeodeAwaitility;
 
 public abstract class AbstractSMoveIntegrationTest implements RedisIntegrationTest {

Review comment:
       No, different slots. Currently all the tests are using keys which have the same tag, {user1}, which means that they'll all map to the same slot (that's the purpose of using tags). It would be good to have a test where one key is `{tag1}key` and the other key is `{tag2}key` so that the slots they map to are different and we can see the behaviour for that case is correct.




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



[GitHub] [geode] DonalEvans merged pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
DonalEvans merged pull request #7227:
URL: https://github.com/apache/geode/pull/7227


   


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



[GitHub] [geode] Kris-10-0 commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
Kris-10-0 commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r778486234



##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSMoveIntegrationTest.java
##########
@@ -53,118 +55,132 @@ public void tearDown() {
   }
 
   @Test
-  public void errors_GivenWrongNumberOfArguments() {
+  public void smove_givenWrongNumberOfArguments_returnsError() {
     assertExactNumberOfArgs(jedis, Protocol.Command.SMOVE, 3);
   }
 
   @Test
-  public void testSmove_returnsWrongType_whenWrongSourceIsUsed() {
-    jedis.set("{user1}a-string", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-string", "{user1}some-set", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
-
-    jedis.hset("{user1}a-hash", "field", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-hash", "{user1}some-set", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
+  public void smove_withWrongTypeSource_returnsWrongTypeError() {
+    jedis.set(sourceKey, "value");
+    jedis.sadd(destKey, destMembers);
+
+    assertThatThrownBy(() -> jedis.smove(sourceKey, destKey, movedMember))
+        .hasMessageContaining(RedisConstants.ERROR_WRONG_TYPE);
   }
 
   @Test
-  public void testSmove_returnsWrongType_whenWrongDestinationIsUsed() {
-    jedis.sadd("{user1}a-set", "foobaz");
+  public void smove_withWrongTypeDest_returnsWrongTypeError() {
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.set(destKey, "value");
 
-    jedis.set("{user1}a-string", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-set", "{user1}a-string", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
+    assertThatThrownBy(() -> jedis.smove(sourceKey, destKey, movedMember))
+        .hasMessageContaining(RedisConstants.ERROR_WRONG_TYPE);
+  }
 
-    jedis.hset("{user1}a-hash", "field", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-set", "{user1}a-hash", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
+  @Test
+  public void smove_withWrongTypeSourceAndDest_returnsWrongTypeError() {
+    jedis.set(sourceKey, "sourceMember");
+    jedis.set(destKey, "destMember");
+
+    assertThatThrownBy(() -> jedis.smove(sourceKey, destKey, movedMember))
+        .hasMessageContaining(RedisConstants.ERROR_WRONG_TYPE);
   }
 
   @Test
-  public void testSMove() {
-    String source = "{user1}source";
-    String dest = "{user1}dest";
-    String test = "{user1}test";
-    int elements = 10;
-    String[] strings = generateStrings(elements, "value-");
-    jedis.sadd(source, strings);
-
-    long i = 1;
-    for (String entry : strings) {
-      long results = jedis.smove(source, dest, entry);
-      assertThat(results).isEqualTo(1);
-      assertThat(jedis.sismember(dest, entry)).isTrue();
-
-      results = jedis.scard(source);
-      assertThat(results).isEqualTo(strings.length - i);
-      assertThat(jedis.scard(dest)).isEqualTo(i);
-      i++;
-    }
-
-    assertThat(jedis.smove(test, dest, "unknown-value")).isEqualTo(0);
+  public void smove_withNonExistentSource_returnsZero_sourceKeyDoesNotExist() {
+    jedis.sadd(destKey, destMembers);
+
+    assertThat(jedis.smove(nonExistentSetKey, destKey, movedMember))
+        .isEqualTo(0);
+    assertThat(jedis.exists(nonExistentSetKey)).isFalse();
   }
 
   @Test
-  public void testSMoveNegativeCases() {
-    String source = "{user1}source";
-    String dest = "{user1}dest";
-    jedis.sadd(source, "sourceField");
-    jedis.sadd(dest, "destField");
-    String nonexistentField = "nonexistentField";
-
-    assertThat(jedis.smove(source, dest, nonexistentField)).isEqualTo(0);
-    assertThat(jedis.sismember(dest, nonexistentField)).isFalse();
-    assertThat(jedis.smove(source, "{user1}nonexistentDest", nonexistentField)).isEqualTo(0);
-    assertThat(jedis.smove("{user1}nonExistentSource", dest, nonexistentField)).isEqualTo(0);
+  public void smove_withNonExistentMemberInSource_returnsZero_memberNotAddedToDest() {
+    String nonExistentMember = "foo";
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.sadd(destKey, destMembers);
+
+    assertThat(jedis.smove(nonExistentSetKey, destKey, nonExistentMember))
+        .isEqualTo(0);
+    assertThat(jedis.sismember(destKey, nonExistentMember)).isFalse();
+  }
+
+  @Test
+  public void smove_withExistentSourceAndNonExistentDest_returnsOne_memberMovedFromSourceToCreatedDest() {
+    jedis.sadd(sourceKey, sourceMembers);
+
+    String[] sourceResult = ArrayUtils.remove(sourceMembers, 0);
+    String[] destResult = new String[] {movedMember};
+
+    assertThat(jedis.smove(sourceKey, destKey, movedMember))
+        .isEqualTo(1);
+
+    assertThat(jedis.smembers(sourceKey)).containsExactlyInAnyOrder(sourceResult);
+    assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(destResult);
   }
 
   @Test
-  public void testConcurrentSMove() {
-    String source = "{user1}source";
-    String dest = "{user1}dest";
-    int elements = 10000;
-    String[] strings = generateStrings(elements, "value-");
-    jedis.sadd(source, strings);
-
-    AtomicLong counter = new AtomicLong(0);
-    new ConcurrentLoopingThreads(elements,
-        (i) -> counter.getAndAdd(jedis.smove(source, dest, strings[i])),
-        (i) -> counter.getAndAdd(jedis.smove(source, dest, strings[i]))).run();
-
-    assertThat(counter.get()).isEqualTo(new Long(strings.length));
-    assertThat(jedis.smembers(dest)).containsExactlyInAnyOrder(strings);
-    assertThat(jedis.scard(source)).isEqualTo(0L);
+  public void smove_withExistentSourceAndDest_returnsOne_memberMovedFromSourceToDest() {
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.sadd(destKey, destMembers);
+
+    String[] sourceResult = ArrayUtils.remove(sourceMembers, 0);
+    String[] destResult = ArrayUtils.add(destMembers, movedMember);
+
+    assertThat(jedis.smove(sourceKey, destKey, movedMember))
+        .isEqualTo(1);
+
+    assertThat(jedis.smembers(sourceKey)).containsExactlyInAnyOrder(sourceResult);
+    assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(destResult);
   }
 
   @Test
-  public void testConcurrentSMove_withDifferentDestination() {
-    String source = "{user1}source";
-    String dest1 = "{user1}dest1";
-    String dest2 = "{user1}dest2";
-    int elements = 10000;
-    String[] strings = generateStrings(elements, "value-");
-    jedis.sadd(source, strings);
-
-    AtomicLong counter = new AtomicLong(0);
-    new ConcurrentLoopingThreads(elements,
-        (i) -> counter.getAndAdd(jedis.smove(source, dest1, strings[i])),
-        (i) -> counter.getAndAdd(jedis.smove(source, dest2, strings[i]))).run();
-
-    List<String> result = new ArrayList<>();
-    result.addAll(jedis.smembers(dest1));
-    result.addAll(jedis.smembers(dest2));
-
-    assertThat(counter.get()).isEqualTo(new Long(strings.length));
-    assertThat(result).containsExactlyInAnyOrder(strings);
-    assertThat(jedis.scard(source)).isEqualTo(0L);
+  public void smove_withExistentSourceAndDest_withMemberInDest_returnsOne_memberRemovedFromSource() {
+    jedis.sadd(sourceKey, sourceMembers);
+    String[] newDestMembers = ArrayUtils.add(destMembers, movedMember);
+    jedis.sadd(destKey, newDestMembers);
+
+    String[] sourceResult = ArrayUtils.remove(sourceMembers, 0);
+
+    assertThat(jedis.smove(sourceKey, destKey, movedMember))
+        .isEqualTo(1);
+
+    assertThat(jedis.smembers(sourceKey)).containsExactlyInAnyOrder(sourceResult);
+    assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(newDestMembers);
   }
 
-  private String[] generateStrings(int elements, String prefix) {
-    Set<String> strings = new HashSet<>();
-    for (int i = 0; i < elements; i++) {
-      strings.add(prefix + i);
-    }
-    return strings.toArray(new String[strings.size()]);
+  @Test
+  public void ensureSetConsistency_whenRunningConcurrently() {
+    String[] sourceMemberRemoved = ArrayUtils.remove(sourceMembers, 0);
+    String[] destMemberAdded = ArrayUtils.add(destMembers, movedMember);
+
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.sadd(destKey, destMembers);
+
+    final AtomicLong moved = new AtomicLong(0);
+    new ConcurrentLoopingThreads(1000,
+        i -> jedis.srem(sourceKey, movedMember),
+        i -> moved.set(jedis.smove(sourceKey, destKey, movedMember)))
+            .runWithAction(() -> {
+              // Check sdiffstore return size of diff
+              assertThat(moved).satisfiesAnyOf(
+                  smoveResult -> assertThat(smoveResult.get()).isEqualTo(0),
+                  smoveResult -> assertThat(smoveResult.get()).isEqualTo(1));
+              // Checks if values were moved or not from source key
+              assertThat(sourceKey).satisfiesAnyOf(
+                  source -> assertThat(jedis.smembers(source))
+                      .containsExactlyInAnyOrder(sourceMembers),
+                  source -> assertThat(jedis.smembers(source))
+                      .containsExactlyInAnyOrder(sourceMemberRemoved));
+              // Checks if values were moved or not to destination key
+              assertThat(destKey).satisfiesAnyOf(
+                  dest -> assertThat(jedis.smembers(dest))
+                      .containsExactlyInAnyOrder(destMembers),
+                  dest -> assertThat(jedis.smembers(dest))
+                      .containsExactlyInAnyOrder(destMemberAdded));
+              jedis.sadd(sourceKey, movedMember);
+              jedis.srem(destKey, movedMember);
+            });

Review comment:
       Added at line 190. in ensureSetConsistency_whenRunningConcurrently_withSMovesFromSameSourceAndDifferentDestination()




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



[GitHub] [geode] Kris-10-0 commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
Kris-10-0 commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r778487244



##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSMoveIntegrationTest.java
##########
@@ -34,16 +33,19 @@
 import org.apache.geode.redis.ConcurrentLoopingThreads;
 import org.apache.geode.redis.RedisIntegrationTest;
 import org.apache.geode.redis.internal.RedisConstants;
-import org.apache.geode.test.awaitility.GeodeAwaitility;
 
 public abstract class AbstractSMoveIntegrationTest implements RedisIntegrationTest {

Review comment:
       Is this supposed to be when the source and destination key do map to the same slot? (Use the same key for destination and source?)




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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r777633900



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SMoveExecutor.java
##########
@@ -14,47 +14,41 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
-import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
-import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisSet.smove;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.geode.cache.Region;
 import org.apache.geode.redis.internal.commands.Command;
 import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
 import org.apache.geode.redis.internal.commands.executor.RedisResponse;
-import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisDataMovedException;
 import org.apache.geode.redis.internal.data.RedisKey;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
 
 public class SMoveExecutor implements CommandExecutor {
 
   @Override
   public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<RedisKey, RedisData> region = context.getRegion();
-    RedisKey source = command.getKey();
-    RedisKey destination = new RedisKey(commandElems.get(2));
-    byte[] member = commandElems.get(3);
-
-    // TODO: this command should lock both source and destination before changing them
+    List<RedisKey> commandKeys = command.getProcessedCommandKeys();
+    List<RedisKey> setKeys = commandKeys.subList(1, 3);
 
-    String destinationType = context.dataLockedExecute(destination, false, RedisData::type);
-    if (!destinationType.equals(REDIS_SET.toString()) && !destinationType.equals("none")) {
-      return RedisResponse.wrongType(ERROR_WRONG_TYPE);
+    byte[] member = commandElems.get(3);
+    RegionProvider regionProvider = context.getRegionProvider();
+    try {
+      for (RedisKey k : setKeys) {
+        regionProvider.ensureKeyIsLocal(k);
+      }
+    } catch (RedisDataMovedException ex) {

Review comment:
       Do you know that this is necessary? Is there a test that covers this scenario for native Redis (I couldn't see one).

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSMoveIntegrationTest.java
##########
@@ -53,118 +55,132 @@ public void tearDown() {
   }
 
   @Test
-  public void errors_GivenWrongNumberOfArguments() {
+  public void smove_givenWrongNumberOfArguments_returnsError() {
     assertExactNumberOfArgs(jedis, Protocol.Command.SMOVE, 3);
   }
 
   @Test
-  public void testSmove_returnsWrongType_whenWrongSourceIsUsed() {
-    jedis.set("{user1}a-string", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-string", "{user1}some-set", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
-
-    jedis.hset("{user1}a-hash", "field", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-hash", "{user1}some-set", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
+  public void smove_withWrongTypeSource_returnsWrongTypeError() {
+    jedis.set(sourceKey, "value");
+    jedis.sadd(destKey, destMembers);
+
+    assertThatThrownBy(() -> jedis.smove(sourceKey, destKey, movedMember))
+        .hasMessageContaining(RedisConstants.ERROR_WRONG_TYPE);
   }
 
   @Test
-  public void testSmove_returnsWrongType_whenWrongDestinationIsUsed() {
-    jedis.sadd("{user1}a-set", "foobaz");
+  public void smove_withWrongTypeDest_returnsWrongTypeError() {
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.set(destKey, "value");
 
-    jedis.set("{user1}a-string", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-set", "{user1}a-string", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
+    assertThatThrownBy(() -> jedis.smove(sourceKey, destKey, movedMember))
+        .hasMessageContaining(RedisConstants.ERROR_WRONG_TYPE);
+  }
 
-    jedis.hset("{user1}a-hash", "field", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-set", "{user1}a-hash", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
+  @Test
+  public void smove_withWrongTypeSourceAndDest_returnsWrongTypeError() {
+    jedis.set(sourceKey, "sourceMember");
+    jedis.set(destKey, "destMember");
+
+    assertThatThrownBy(() -> jedis.smove(sourceKey, destKey, movedMember))
+        .hasMessageContaining(RedisConstants.ERROR_WRONG_TYPE);
   }
 
   @Test
-  public void testSMove() {
-    String source = "{user1}source";
-    String dest = "{user1}dest";
-    String test = "{user1}test";
-    int elements = 10;
-    String[] strings = generateStrings(elements, "value-");
-    jedis.sadd(source, strings);
-
-    long i = 1;
-    for (String entry : strings) {
-      long results = jedis.smove(source, dest, entry);
-      assertThat(results).isEqualTo(1);
-      assertThat(jedis.sismember(dest, entry)).isTrue();
-
-      results = jedis.scard(source);
-      assertThat(results).isEqualTo(strings.length - i);
-      assertThat(jedis.scard(dest)).isEqualTo(i);
-      i++;
-    }
-
-    assertThat(jedis.smove(test, dest, "unknown-value")).isEqualTo(0);
+  public void smove_withNonExistentSource_returnsZero_sourceKeyDoesNotExist() {
+    jedis.sadd(destKey, destMembers);
+
+    assertThat(jedis.smove(nonExistentSetKey, destKey, movedMember))
+        .isEqualTo(0);
+    assertThat(jedis.exists(nonExistentSetKey)).isFalse();
   }
 
   @Test
-  public void testSMoveNegativeCases() {
-    String source = "{user1}source";
-    String dest = "{user1}dest";
-    jedis.sadd(source, "sourceField");
-    jedis.sadd(dest, "destField");
-    String nonexistentField = "nonexistentField";
-
-    assertThat(jedis.smove(source, dest, nonexistentField)).isEqualTo(0);
-    assertThat(jedis.sismember(dest, nonexistentField)).isFalse();
-    assertThat(jedis.smove(source, "{user1}nonexistentDest", nonexistentField)).isEqualTo(0);
-    assertThat(jedis.smove("{user1}nonExistentSource", dest, nonexistentField)).isEqualTo(0);
+  public void smove_withNonExistentMemberInSource_returnsZero_memberNotAddedToDest() {
+    String nonExistentMember = "foo";
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.sadd(destKey, destMembers);
+
+    assertThat(jedis.smove(nonExistentSetKey, destKey, nonExistentMember))
+        .isEqualTo(0);
+    assertThat(jedis.sismember(destKey, nonExistentMember)).isFalse();
+  }
+
+  @Test
+  public void smove_withExistentSourceAndNonExistentDest_returnsOne_memberMovedFromSourceToCreatedDest() {
+    jedis.sadd(sourceKey, sourceMembers);
+
+    String[] sourceResult = ArrayUtils.remove(sourceMembers, 0);
+    String[] destResult = new String[] {movedMember};
+
+    assertThat(jedis.smove(sourceKey, destKey, movedMember))
+        .isEqualTo(1);
+
+    assertThat(jedis.smembers(sourceKey)).containsExactlyInAnyOrder(sourceResult);
+    assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(destResult);
   }
 
   @Test
-  public void testConcurrentSMove() {
-    String source = "{user1}source";
-    String dest = "{user1}dest";
-    int elements = 10000;
-    String[] strings = generateStrings(elements, "value-");
-    jedis.sadd(source, strings);
-
-    AtomicLong counter = new AtomicLong(0);
-    new ConcurrentLoopingThreads(elements,
-        (i) -> counter.getAndAdd(jedis.smove(source, dest, strings[i])),
-        (i) -> counter.getAndAdd(jedis.smove(source, dest, strings[i]))).run();
-
-    assertThat(counter.get()).isEqualTo(new Long(strings.length));
-    assertThat(jedis.smembers(dest)).containsExactlyInAnyOrder(strings);
-    assertThat(jedis.scard(source)).isEqualTo(0L);
+  public void smove_withExistentSourceAndDest_returnsOne_memberMovedFromSourceToDest() {
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.sadd(destKey, destMembers);
+
+    String[] sourceResult = ArrayUtils.remove(sourceMembers, 0);
+    String[] destResult = ArrayUtils.add(destMembers, movedMember);
+
+    assertThat(jedis.smove(sourceKey, destKey, movedMember))
+        .isEqualTo(1);
+
+    assertThat(jedis.smembers(sourceKey)).containsExactlyInAnyOrder(sourceResult);
+    assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(destResult);
   }
 
   @Test
-  public void testConcurrentSMove_withDifferentDestination() {
-    String source = "{user1}source";
-    String dest1 = "{user1}dest1";
-    String dest2 = "{user1}dest2";
-    int elements = 10000;
-    String[] strings = generateStrings(elements, "value-");
-    jedis.sadd(source, strings);
-
-    AtomicLong counter = new AtomicLong(0);
-    new ConcurrentLoopingThreads(elements,
-        (i) -> counter.getAndAdd(jedis.smove(source, dest1, strings[i])),
-        (i) -> counter.getAndAdd(jedis.smove(source, dest2, strings[i]))).run();
-
-    List<String> result = new ArrayList<>();
-    result.addAll(jedis.smembers(dest1));
-    result.addAll(jedis.smembers(dest2));
-
-    assertThat(counter.get()).isEqualTo(new Long(strings.length));
-    assertThat(result).containsExactlyInAnyOrder(strings);
-    assertThat(jedis.scard(source)).isEqualTo(0L);
+  public void smove_withExistentSourceAndDest_withMemberInDest_returnsOne_memberRemovedFromSource() {
+    jedis.sadd(sourceKey, sourceMembers);
+    String[] newDestMembers = ArrayUtils.add(destMembers, movedMember);
+    jedis.sadd(destKey, newDestMembers);
+
+    String[] sourceResult = ArrayUtils.remove(sourceMembers, 0);
+
+    assertThat(jedis.smove(sourceKey, destKey, movedMember))
+        .isEqualTo(1);
+
+    assertThat(jedis.smembers(sourceKey)).containsExactlyInAnyOrder(sourceResult);
+    assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(newDestMembers);
   }
 
-  private String[] generateStrings(int elements, String prefix) {
-    Set<String> strings = new HashSet<>();
-    for (int i = 0; i < elements; i++) {
-      strings.add(prefix + i);
-    }
-    return strings.toArray(new String[strings.size()]);
+  @Test
+  public void ensureSetConsistency_whenRunningConcurrently() {
+    String[] sourceMemberRemoved = ArrayUtils.remove(sourceMembers, 0);
+    String[] destMemberAdded = ArrayUtils.add(destMembers, movedMember);
+
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.sadd(destKey, destMembers);
+
+    final AtomicLong moved = new AtomicLong(0);
+    new ConcurrentLoopingThreads(1000,
+        i -> jedis.srem(sourceKey, movedMember),
+        i -> moved.set(jedis.smove(sourceKey, destKey, movedMember)))
+            .runWithAction(() -> {
+              // Check sdiffstore return size of diff
+              assertThat(moved).satisfiesAnyOf(
+                  smoveResult -> assertThat(smoveResult.get()).isEqualTo(0),
+                  smoveResult -> assertThat(smoveResult.get()).isEqualTo(1));
+              // Checks if values were moved or not from source key
+              assertThat(sourceKey).satisfiesAnyOf(
+                  source -> assertThat(jedis.smembers(source))
+                      .containsExactlyInAnyOrder(sourceMembers),
+                  source -> assertThat(jedis.smembers(source))
+                      .containsExactlyInAnyOrder(sourceMemberRemoved));
+              // Checks if values were moved or not to destination key
+              assertThat(destKey).satisfiesAnyOf(
+                  dest -> assertThat(jedis.smembers(dest))
+                      .containsExactlyInAnyOrder(destMembers),
+                  dest -> assertThat(jedis.smembers(dest))
+                      .containsExactlyInAnyOrder(destMemberAdded));

Review comment:
       I think this can be simplified and made a bit more explicit. If moved == 1 then the member must have moved and we should check for that. Similarly, if moved == 0, we can check that the member was just deleted and not moved.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSMoveIntegrationTest.java
##########
@@ -53,118 +55,132 @@ public void tearDown() {
   }
 
   @Test
-  public void errors_GivenWrongNumberOfArguments() {
+  public void smove_givenWrongNumberOfArguments_returnsError() {
     assertExactNumberOfArgs(jedis, Protocol.Command.SMOVE, 3);
   }
 
   @Test
-  public void testSmove_returnsWrongType_whenWrongSourceIsUsed() {
-    jedis.set("{user1}a-string", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-string", "{user1}some-set", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
-
-    jedis.hset("{user1}a-hash", "field", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-hash", "{user1}some-set", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
+  public void smove_withWrongTypeSource_returnsWrongTypeError() {
+    jedis.set(sourceKey, "value");
+    jedis.sadd(destKey, destMembers);
+
+    assertThatThrownBy(() -> jedis.smove(sourceKey, destKey, movedMember))
+        .hasMessageContaining(RedisConstants.ERROR_WRONG_TYPE);
   }
 
   @Test
-  public void testSmove_returnsWrongType_whenWrongDestinationIsUsed() {
-    jedis.sadd("{user1}a-set", "foobaz");
+  public void smove_withWrongTypeDest_returnsWrongTypeError() {
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.set(destKey, "value");
 
-    jedis.set("{user1}a-string", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-set", "{user1}a-string", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
+    assertThatThrownBy(() -> jedis.smove(sourceKey, destKey, movedMember))
+        .hasMessageContaining(RedisConstants.ERROR_WRONG_TYPE);
+  }
 
-    jedis.hset("{user1}a-hash", "field", "value");
-    assertThatThrownBy(() -> jedis.smove("{user1}a-set", "{user1}a-hash", "foo"))
-        .hasMessage("WRONGTYPE " + RedisConstants.ERROR_WRONG_TYPE);
+  @Test
+  public void smove_withWrongTypeSourceAndDest_returnsWrongTypeError() {
+    jedis.set(sourceKey, "sourceMember");
+    jedis.set(destKey, "destMember");
+
+    assertThatThrownBy(() -> jedis.smove(sourceKey, destKey, movedMember))
+        .hasMessageContaining(RedisConstants.ERROR_WRONG_TYPE);
   }
 
   @Test
-  public void testSMove() {
-    String source = "{user1}source";
-    String dest = "{user1}dest";
-    String test = "{user1}test";
-    int elements = 10;
-    String[] strings = generateStrings(elements, "value-");
-    jedis.sadd(source, strings);
-
-    long i = 1;
-    for (String entry : strings) {
-      long results = jedis.smove(source, dest, entry);
-      assertThat(results).isEqualTo(1);
-      assertThat(jedis.sismember(dest, entry)).isTrue();
-
-      results = jedis.scard(source);
-      assertThat(results).isEqualTo(strings.length - i);
-      assertThat(jedis.scard(dest)).isEqualTo(i);
-      i++;
-    }
-
-    assertThat(jedis.smove(test, dest, "unknown-value")).isEqualTo(0);
+  public void smove_withNonExistentSource_returnsZero_sourceKeyDoesNotExist() {
+    jedis.sadd(destKey, destMembers);
+
+    assertThat(jedis.smove(nonExistentSetKey, destKey, movedMember))
+        .isEqualTo(0);
+    assertThat(jedis.exists(nonExistentSetKey)).isFalse();
   }
 
   @Test
-  public void testSMoveNegativeCases() {
-    String source = "{user1}source";
-    String dest = "{user1}dest";
-    jedis.sadd(source, "sourceField");
-    jedis.sadd(dest, "destField");
-    String nonexistentField = "nonexistentField";
-
-    assertThat(jedis.smove(source, dest, nonexistentField)).isEqualTo(0);
-    assertThat(jedis.sismember(dest, nonexistentField)).isFalse();
-    assertThat(jedis.smove(source, "{user1}nonexistentDest", nonexistentField)).isEqualTo(0);
-    assertThat(jedis.smove("{user1}nonExistentSource", dest, nonexistentField)).isEqualTo(0);
+  public void smove_withNonExistentMemberInSource_returnsZero_memberNotAddedToDest() {
+    String nonExistentMember = "foo";
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.sadd(destKey, destMembers);
+
+    assertThat(jedis.smove(nonExistentSetKey, destKey, nonExistentMember))
+        .isEqualTo(0);
+    assertThat(jedis.sismember(destKey, nonExistentMember)).isFalse();
+  }
+
+  @Test
+  public void smove_withExistentSourceAndNonExistentDest_returnsOne_memberMovedFromSourceToCreatedDest() {
+    jedis.sadd(sourceKey, sourceMembers);
+
+    String[] sourceResult = ArrayUtils.remove(sourceMembers, 0);
+    String[] destResult = new String[] {movedMember};
+
+    assertThat(jedis.smove(sourceKey, destKey, movedMember))
+        .isEqualTo(1);
+
+    assertThat(jedis.smembers(sourceKey)).containsExactlyInAnyOrder(sourceResult);
+    assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(destResult);
   }
 
   @Test
-  public void testConcurrentSMove() {
-    String source = "{user1}source";
-    String dest = "{user1}dest";
-    int elements = 10000;
-    String[] strings = generateStrings(elements, "value-");
-    jedis.sadd(source, strings);
-
-    AtomicLong counter = new AtomicLong(0);
-    new ConcurrentLoopingThreads(elements,
-        (i) -> counter.getAndAdd(jedis.smove(source, dest, strings[i])),
-        (i) -> counter.getAndAdd(jedis.smove(source, dest, strings[i]))).run();
-
-    assertThat(counter.get()).isEqualTo(new Long(strings.length));
-    assertThat(jedis.smembers(dest)).containsExactlyInAnyOrder(strings);
-    assertThat(jedis.scard(source)).isEqualTo(0L);
+  public void smove_withExistentSourceAndDest_returnsOne_memberMovedFromSourceToDest() {
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.sadd(destKey, destMembers);
+
+    String[] sourceResult = ArrayUtils.remove(sourceMembers, 0);
+    String[] destResult = ArrayUtils.add(destMembers, movedMember);
+
+    assertThat(jedis.smove(sourceKey, destKey, movedMember))
+        .isEqualTo(1);
+
+    assertThat(jedis.smembers(sourceKey)).containsExactlyInAnyOrder(sourceResult);
+    assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(destResult);
   }
 
   @Test
-  public void testConcurrentSMove_withDifferentDestination() {
-    String source = "{user1}source";
-    String dest1 = "{user1}dest1";
-    String dest2 = "{user1}dest2";
-    int elements = 10000;
-    String[] strings = generateStrings(elements, "value-");
-    jedis.sadd(source, strings);
-
-    AtomicLong counter = new AtomicLong(0);
-    new ConcurrentLoopingThreads(elements,
-        (i) -> counter.getAndAdd(jedis.smove(source, dest1, strings[i])),
-        (i) -> counter.getAndAdd(jedis.smove(source, dest2, strings[i]))).run();
-
-    List<String> result = new ArrayList<>();
-    result.addAll(jedis.smembers(dest1));
-    result.addAll(jedis.smembers(dest2));
-
-    assertThat(counter.get()).isEqualTo(new Long(strings.length));
-    assertThat(result).containsExactlyInAnyOrder(strings);
-    assertThat(jedis.scard(source)).isEqualTo(0L);
+  public void smove_withExistentSourceAndDest_withMemberInDest_returnsOne_memberRemovedFromSource() {
+    jedis.sadd(sourceKey, sourceMembers);
+    String[] newDestMembers = ArrayUtils.add(destMembers, movedMember);
+    jedis.sadd(destKey, newDestMembers);
+
+    String[] sourceResult = ArrayUtils.remove(sourceMembers, 0);
+
+    assertThat(jedis.smove(sourceKey, destKey, movedMember))
+        .isEqualTo(1);
+
+    assertThat(jedis.smembers(sourceKey)).containsExactlyInAnyOrder(sourceResult);
+    assertThat(jedis.smembers(destKey)).containsExactlyInAnyOrder(newDestMembers);
   }
 
-  private String[] generateStrings(int elements, String prefix) {
-    Set<String> strings = new HashSet<>();
-    for (int i = 0; i < elements; i++) {
-      strings.add(prefix + i);
-    }
-    return strings.toArray(new String[strings.size()]);
+  @Test
+  public void ensureSetConsistency_whenRunningConcurrently() {
+    String[] sourceMemberRemoved = ArrayUtils.remove(sourceMembers, 0);
+    String[] destMemberAdded = ArrayUtils.add(destMembers, movedMember);
+
+    jedis.sadd(sourceKey, sourceMembers);
+    jedis.sadd(destKey, destMembers);
+
+    final AtomicLong moved = new AtomicLong(0);
+    new ConcurrentLoopingThreads(1000,
+        i -> jedis.srem(sourceKey, movedMember),
+        i -> moved.set(jedis.smove(sourceKey, destKey, movedMember)))
+            .runWithAction(() -> {
+              // Check sdiffstore return size of diff
+              assertThat(moved).satisfiesAnyOf(
+                  smoveResult -> assertThat(smoveResult.get()).isEqualTo(0),
+                  smoveResult -> assertThat(smoveResult.get()).isEqualTo(1));
+              // Checks if values were moved or not from source key
+              assertThat(sourceKey).satisfiesAnyOf(
+                  source -> assertThat(jedis.smembers(source))
+                      .containsExactlyInAnyOrder(sourceMembers),
+                  source -> assertThat(jedis.smembers(source))
+                      .containsExactlyInAnyOrder(sourceMemberRemoved));
+              // Checks if values were moved or not to destination key
+              assertThat(destKey).satisfiesAnyOf(
+                  dest -> assertThat(jedis.smembers(dest))
+                      .containsExactlyInAnyOrder(destMembers),
+                  dest -> assertThat(jedis.smembers(dest))
+                      .containsExactlyInAnyOrder(destMemberAdded));
+              jedis.sadd(sourceKey, movedMember);
+              jedis.srem(destKey, movedMember);
+            });

Review comment:
       In addition, it would be good to have a concurrent test that does an `smove` in each thread using the same source but different destination keys.




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



[GitHub] [geode] DonalEvans commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r778443201



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SMoveExecutor.java
##########
@@ -14,47 +14,41 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
-import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
-import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisSet.smove;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.geode.cache.Region;
 import org.apache.geode.redis.internal.commands.Command;
 import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
 import org.apache.geode.redis.internal.commands.executor.RedisResponse;
-import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisDataMovedException;
 import org.apache.geode.redis.internal.data.RedisKey;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
 
 public class SMoveExecutor implements CommandExecutor {
 
   @Override
   public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<RedisKey, RedisData> region = context.getRegion();
-    RedisKey source = command.getKey();
-    RedisKey destination = new RedisKey(commandElems.get(2));
-    byte[] member = commandElems.get(3);
-
-    // TODO: this command should lock both source and destination before changing them
+    List<RedisKey> commandKeys = command.getProcessedCommandKeys();
+    List<RedisKey> setKeys = commandKeys.subList(1, 3);
 
-    String destinationType = context.dataLockedExecute(destination, false, RedisData::type);
-    if (!destinationType.equals(REDIS_SET.toString()) && !destinationType.equals("none")) {
-      return RedisResponse.wrongType(ERROR_WRONG_TYPE);
+    byte[] member = commandElems.get(3);
+    RegionProvider regionProvider = context.getRegionProvider();
+    try {
+      for (RedisKey k : setKeys) {
+        regionProvider.ensureKeyIsLocal(k);

Review comment:
       The `AbstractRenameExecutor` and `ZStoreExecutor` both explicitly check that the slots of the two keys are equal, so we should probably be consistent about that here too. Are there places where we have multi-key commands that only check the bucketId?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SMoveExecutor.java
##########
@@ -14,47 +14,41 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
-import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
-import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisSet.smove;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.geode.cache.Region;
 import org.apache.geode.redis.internal.commands.Command;
 import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
 import org.apache.geode.redis.internal.commands.executor.RedisResponse;
-import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisDataMovedException;
 import org.apache.geode.redis.internal.data.RedisKey;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
 
 public class SMoveExecutor implements CommandExecutor {
 
   @Override
   public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<RedisKey, RedisData> region = context.getRegion();
-    RedisKey source = command.getKey();
-    RedisKey destination = new RedisKey(commandElems.get(2));
-    byte[] member = commandElems.get(3);
-
-    // TODO: this command should lock both source and destination before changing them
+    List<RedisKey> commandKeys = command.getProcessedCommandKeys();
+    List<RedisKey> setKeys = commandKeys.subList(1, 3);
 
-    String destinationType = context.dataLockedExecute(destination, false, RedisData::type);
-    if (!destinationType.equals(REDIS_SET.toString()) && !destinationType.equals("none")) {
-      return RedisResponse.wrongType(ERROR_WRONG_TYPE);
+    byte[] member = commandElems.get(3);
+    RegionProvider regionProvider = context.getRegionProvider();
+    try {
+      for (RedisKey k : setKeys) {
+        regionProvider.ensureKeyIsLocal(k);

Review comment:
       The `AbstractRenameExecutor` and `ZStoreExecutor` both explicitly check that the slots of the two (or more) keys are equal, so we should probably be consistent about that here too. Are there places where we have multi-key commands that only check the bucketId?




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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #7227:
URL: https://github.com/apache/geode/pull/7227#discussion_r778399410



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SMoveExecutor.java
##########
@@ -14,47 +14,41 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
-import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
-import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisSet.smove;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.geode.cache.Region;
 import org.apache.geode.redis.internal.commands.Command;
 import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
 import org.apache.geode.redis.internal.commands.executor.RedisResponse;
-import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisDataMovedException;
 import org.apache.geode.redis.internal.data.RedisKey;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
 
 public class SMoveExecutor implements CommandExecutor {
 
   @Override
   public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<RedisKey, RedisData> region = context.getRegion();
-    RedisKey source = command.getKey();
-    RedisKey destination = new RedisKey(commandElems.get(2));
-    byte[] member = commandElems.get(3);
-
-    // TODO: this command should lock both source and destination before changing them
+    List<RedisKey> commandKeys = command.getProcessedCommandKeys();
+    List<RedisKey> setKeys = commandKeys.subList(1, 3);
 
-    String destinationType = context.dataLockedExecute(destination, false, RedisData::type);
-    if (!destinationType.equals(REDIS_SET.toString()) && !destinationType.equals("none")) {
-      return RedisResponse.wrongType(ERROR_WRONG_TYPE);
+    byte[] member = commandElems.get(3);
+    RegionProvider regionProvider = context.getRegionProvider();
+    try {
+      for (RedisKey k : setKeys) {
+        regionProvider.ensureKeyIsLocal(k);

Review comment:
       Up till now we've only been concerned that the keys are in the same bucket. (I think we had a discussion about this a while back). Perhaps we should discuss again but it should be sufficient to just compare the `bucketId`s of each key.




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



[GitHub] [geode] Kris-10-0 commented on pull request #7227: feature/GEODE-9832: SMOVE Command Support

Posted by GitBox <gi...@apache.org>.
Kris-10-0 commented on pull request #7227:
URL: https://github.com/apache/geode/pull/7227#issuecomment-1007758972


   Forced pushed to rebase


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