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/19 02:29:13 UTC

[GitHub] [geode] DonalEvans commented on a change in pull request #7278: GEODE-9835: Add SSCAN to Redis supported commands

DonalEvans commented on a change in pull request #7278:
URL: https://github.com/apache/geode/pull/7278#discussion_r787241591



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableObjectOpenCustomHashSetWithCursor.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.redis.internal.data.collections;
+
+
+import static it.unimi.dsi.fastutil.HashCommon.mix;
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+
+import java.util.Collection;
+
+import it.unimi.dsi.fastutil.objects.ObjectOpenCustomHashSet;
+
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.internal.size.Sizeable;
+
+public abstract class SizeableObjectOpenCustomHashSetWithCursor<K>
+    extends ObjectOpenCustomHashSet<K>
+    implements Sizeable {
+  private static final long serialVersionUID = 9174920505089089517L;
+  private static final int OPEN_HASH_SET_OVERHEAD =
+      memoryOverhead(SizeableObjectOpenCustomHashSetWithCursor.class);
+
+  private int memberOverhead;
+
+  public SizeableObjectOpenCustomHashSetWithCursor(int expected, Strategy<? super K> strategy) {
+    super(expected, strategy);
+  }
+
+  public SizeableObjectOpenCustomHashSetWithCursor(Strategy<? super K> strategy) {
+    super(strategy);
+  }
+
+  public SizeableObjectOpenCustomHashSetWithCursor(Collection<? extends K> c,
+      Strategy<? super K> strategy) {
+    super(c, strategy);
+  }
+
+  @Override
+  public boolean add(K k) {
+    boolean added = super.add(k);
+    if (added) {
+      memberOverhead += sizeElement(k);
+    }
+    return added;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public boolean remove(Object k) {
+    boolean removed = super.remove(k);
+    if (removed) {
+      memberOverhead -= sizeElement((K) k);
+    }
+    return removed;
+  }
+
+  @Override
+  public int getSizeInBytes() {
+    // The object referenced by the "strategy" field is not sized
+    // since it is usually a singleton instance.
+    return OPEN_HASH_SET_OVERHEAD + memoryOverhead(key) + memberOverhead;
+  }
+
+  /**
+   * Scan entries and pass them to the given consumer function, starting at the passed in
+   * cursor. This method will scan until at least count entries are returned, or the entire
+   * map has been scanned. Once the returned cursor is 0, the entire map is scanned.

Review comment:
       Comments in this class referring to maps should be changed to sets and references to keys or values should be changed to elements, as we're scanning a Set, not a Map. Also, rather than using `K` as a generic parameter, `E` should be used.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/server/AbstractHitsMissesIntegrationTest.java
##########
@@ -554,23 +559,12 @@ public void testSrandmember() {
     runCommandAndAssertHitsAndMisses(SET_KEY, k -> jedis.srandmember(k));
   }
 
-  @Test
-  public void testSscan() {
-    runCommandAndAssertHitsAndMisses(SET_KEY, k -> jedis.sscan(k, "0"));
-  }
-
   @Test
   public void testSinterstore() {
     runMultiKeyCommandAndAssertNoStatUpdates(SET_KEY,
         (k1, k2) -> jedis.sinterstore(HASHTAG + "dest", k1, k2));
   }
 
-  @Test
-  public void testSunionstore() {
-    runMultiKeyCommandAndAssertNoStatUpdates(SET_KEY,
-        (k1, k2) -> jedis.sunionstore(HASHTAG + "dest", k1, k2));
-  }
-

Review comment:
       This test should not be being removed in this PR.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSet.java
##########
@@ -202,42 +201,29 @@ static int setOpStoreResult(RegionProvider regionProvider, RedisKey destinationK
     return diff.size();
   }
 
-  public Pair<BigInteger, List<Object>> sscan(GlobPattern matchPattern, int count,
-      BigInteger cursor) {
-    List<Object> returnList = new ArrayList<>();
-    int size = members.size();
-    BigInteger beforeCursor = new BigInteger("0");
-    int numElements = 0;
-    int i = -1;
-    for (byte[] value : members) {
-      i++;
-      if (beforeCursor.compareTo(cursor) < 0) {
-        beforeCursor = beforeCursor.add(new BigInteger("1"));
-        continue;
-      }
+  public Pair<Integer, List<byte[]>> sscan(GlobPattern matchPattern, int count,
+      int cursor) {
 
-      if (matchPattern != null) {
-        if (matchPattern.matches(value)) {
-          returnList.add(value);
-          numElements++;
-        }
-      } else {
-        returnList.add(value);
-        numElements++;
-      }
+    // No need to allocate more space than it's possible to use given the size of the hash. We need
+    // to add 1 to hlen() to ensure that if count > hash.size(), we return a cursor of 0

Review comment:
       This comment needs updating, since we're not operating on a hash set here.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -14,33 +14,53 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertAtLeastNArgs;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_CURSOR;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_SYNTAX;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
+import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.Jedis;
 import redis.clients.jedis.JedisCluster;
 import redis.clients.jedis.Protocol;
 import redis.clients.jedis.ScanParams;
 import redis.clients.jedis.ScanResult;
 
+import org.apache.geode.redis.ConcurrentLoopingThreads;
 import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.redis.internal.commands.executor.cluster.CRC16;
+import org.apache.geode.redis.internal.services.RegionProvider;
 import org.apache.geode.test.awaitility.GeodeAwaitility;
 
 public abstract class AbstractSScanIntegrationTest implements RedisIntegrationTest {
   protected JedisCluster jedis;
   private static final int REDIS_CLIENT_TIMEOUT =
       Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());

Review comment:
       Rather than defining this constant, the `RedisClusterStartupRule.REDIS_CLIENT_TIMEOUT` constant should be used in the `setUp()` method. Also, instead of hard-coding `"localhost"` as the bind address there, the `RedisClusterStartupRule.BIND_ADDRESS` constant should be used.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableObjectOpenCustomHashSetWithCursor.java
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.redis.internal.data.collections;
+
+
+import static it.unimi.dsi.fastutil.HashCommon.mix;
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+
+import java.util.Collection;
+
+import it.unimi.dsi.fastutil.objects.ObjectOpenCustomHashSet;
+
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.internal.size.Sizeable;
+
+public abstract class SizeableObjectOpenCustomHashSetWithCursor<K>
+    extends ObjectOpenCustomHashSet<K>
+    implements Sizeable {
+  private static final long serialVersionUID = 9174920505089089517L;
+  private static final int OPEN_HASH_SET_OVERHEAD =
+      memoryOverhead(SizeableObjectOpenCustomHashSetWithCursor.class);
+
+  private int memberOverhead;
+
+  public SizeableObjectOpenCustomHashSetWithCursor(int expected, Strategy<? super K> strategy) {
+    super(expected, strategy);
+  }
+
+  public SizeableObjectOpenCustomHashSetWithCursor(Strategy<? super K> strategy) {
+    super(strategy);
+  }
+
+  public SizeableObjectOpenCustomHashSetWithCursor(Collection<? extends K> c,
+      Strategy<? super K> strategy) {
+    super(c, strategy);
+  }
+
+  @Override
+  public boolean add(K k) {
+    boolean added = super.add(k);
+    if (added) {
+      memberOverhead += sizeElement(k);
+    }
+    return added;
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public boolean remove(Object k) {
+    boolean removed = super.remove(k);
+    if (removed) {
+      memberOverhead -= sizeElement((K) k);
+    }
+    return removed;
+  }
+
+  @Override
+  public int getSizeInBytes() {
+    // The object referenced by the "strategy" field is not sized
+    // since it is usually a singleton instance.
+    return OPEN_HASH_SET_OVERHEAD + memoryOverhead(key) + memberOverhead;
+  }
+
+  /**
+   * Scan entries and pass them to the given consumer function, starting at the passed in
+   * cursor. This method will scan until at least count entries are returned, or the entire
+   * map has been scanned. Once the returned cursor is 0, the entire map is scanned.
+   *
+   * This method may emit more than *count* number of elements if there are hash collisions.
+   *
+   * @param cursor The cursor to start from. Should be 0 for the initial scan. Subsequent calls
+   *        should use the cursor returned by the previous scan call.
+   * @param count The number of elements to scan
+   * @param consumer A function to pass the scanned keys and values to
+   * @param privateData Some data to pass to the function, for example a map to collect values in.
+   *        This
+   *        allows the function to be stateless.
+   * @param <D> The type of the data passed to the function/
+   * @return The next cursor to scan from, or 0 if the scan has touched all elements.
+   */
+  public <D> int scan(int cursor, int count,
+      SizeableObjectOpenCustomHashSetWithCursor.EntryConsumer<K, D> consumer, D privateData) {
+    // Implementation notes
+    //
+    // This stateless scan cursor algorithm is based on the dictScan cursor
+    // implementation from dict.c in redis. Please see the comments in that class for the full
+    // details. That iteration algorithm was designed by Pieter Noordhuis.
+    //
+    // There is one wrinkle due to the fact that we are using a different type of hashtable here.
+    // The parent class, Object2ObjectOpenHashMap, uses an open addressing with a linear

Review comment:
       This comment also needs updating, as the parent class of this is not a `Object2ObjectOpenHashMap`.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -53,97 +73,143 @@ public void tearDown() {
     jedis.close();
   }
 
+  @Test
+  public void givenLessThanTwoArguments_returnsWrongNumberOfArgumentsError() {
+    assertAtLeastNArgs(jedis, Protocol.Command.SSCAN, 2);
+  }
+
   @Test
   public void givenNoKeyArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenNoCursorArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN, "key"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenArgumentsAreNotOddAndKeyExists_returnsSyntaxError() {

Review comment:
       This test name could be more accurate as "givenIncorrectOptionalArgumentAndKeyExists_returnsSyntaxError" since the number of arguments isn't actually the relevant factor here, just that there is a syntax error of some kind.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -53,97 +73,143 @@ public void tearDown() {
     jedis.close();
   }
 
+  @Test
+  public void givenLessThanTwoArguments_returnsWrongNumberOfArgumentsError() {
+    assertAtLeastNArgs(jedis, Protocol.Command.SSCAN, 2);
+  }
+
   @Test
   public void givenNoKeyArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenNoCursorArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN, "key"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }

Review comment:
       These two tests are redundant with the "givenLessThanTwoArguments_returnsWrongNumberOfArgumentsError()` test, so they can be removed.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -53,97 +73,143 @@ public void tearDown() {
     jedis.close();
   }
 
+  @Test
+  public void givenLessThanTwoArguments_returnsWrongNumberOfArgumentsError() {
+    assertAtLeastNArgs(jedis, Protocol.Command.SSCAN, 2);
+  }
+
   @Test
   public void givenNoKeyArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenNoCursorArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN, "key"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenArgumentsAreNotOddAndKeyExists_returnsSyntaxError() {
-    jedis.sadd("a", "1");
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "a*"))
+    jedis.sadd(KEY, FIELD_ONE);
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "a*"))
         .hasMessageContaining(ERROR_SYNTAX);
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenArgumentsAreNotOddAndKeyDoesNotExist_returnsEmptyArray() {
     List<Object> result =
-        (List<Object>) jedis.sendCommand("key!", Protocol.Command.SSCAN, "key!", "0", "a*");
+        (List<Object>) jedis.sendCommand("key!", Protocol.Command.SSCAN, "key!", ZERO_CURSOR, "a*");
 
-    assertThat((byte[]) result.get(0)).isEqualTo("0".getBytes());
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
     assertThat((List<Object>) result.get(1)).isEmpty();
   }
 
+  @Test
+  public void givenMatchArgumentWithoutPatternOnExistingKey_returnsSyntaxError() {
+    jedis.sadd(KEY, FIELD_ONE);
+
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "MATCH"))
+            .hasMessageContaining(ERROR_SYNTAX);
+  }
+
+  @Test
+  public void givenMatchArgumentWithoutPatternOnNonExistentKey_returnsEmptyArray() {
+    List<Object> result =
+        uncheckedCast(jedis.sendCommand("nonexistentKey", Protocol.Command.SSCAN, "nonexistentKey",
+            ZERO_CURSOR, "MATCH"));
+
+    assertThat((List<?>) result.get(1)).isEmpty();
+  }
+
+  @Test
+  public void givenCountArgumentWithoutNumberOnExistingKey_returnsSyntaxError() {
+    jedis.sadd(KEY, FIELD_ONE);
+
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "COUNT"))
+            .hasMessageContaining(ERROR_SYNTAX);
+  }
+
+  @Test
+  public void givenCountArgumentWithoutNumberOnNonExistentKey_returnsEmptyArray() {
+    List<Object> result =
+        uncheckedCast(
+            jedis.sendCommand("nonexistentKey", Protocol.Command.SSCAN, "nonexistentKey",
+                ZERO_CURSOR, "COUNT"));
+
+    assertThat((List<?>) result.get(1)).isEmpty();
+  }
+
   @Test
   public void givenMatchOrCountKeywordNotSpecified_returnsSyntaxError() {
-    jedis.sadd("a", "1");
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "a*", "1"))
-        .hasMessageContaining(ERROR_SYNTAX);
+    jedis.sadd(KEY, FIELD_ONE);
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "a*", FIELD_ONE))
+            .hasMessageContaining(ERROR_SYNTAX);
   }
 
   @Test
   public void givenCount_whenCountParameterIsNotAnInteger_returnsNotIntegerError() {
-    jedis.sadd("a", "1");
+    jedis.sadd(KEY, FIELD_ONE);
     assertThatThrownBy(
-        () -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "COUNT", "MATCH"))
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "COUNT", "MATCH"))
             .hasMessageContaining(ERROR_NOT_INTEGER);
   }
 
   @Test
   public void givenMultipleCounts_whenAnyCountParameterIsNotAnInteger_returnsNotIntegerError() {
-    jedis.sadd("a", "1");
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "COUNT", "2",
-        "COUNT", "sjlfs", "COUNT", "1"))
-            .hasMessageContaining(ERROR_NOT_INTEGER);
+    jedis.sadd(KEY, FIELD_ONE);
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "COUNT", FIELD_TWO,
+            "COUNT", "sjlfs", "COUNT", FIELD_ONE))

Review comment:
       `FIELD_ONE` and `FIELD_TWO` here should be replaced with "1" and "2" (or any other integer value) since they represent count values, not fields.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -53,97 +73,143 @@ public void tearDown() {
     jedis.close();
   }
 
+  @Test
+  public void givenLessThanTwoArguments_returnsWrongNumberOfArgumentsError() {
+    assertAtLeastNArgs(jedis, Protocol.Command.SSCAN, 2);
+  }
+
   @Test
   public void givenNoKeyArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenNoCursorArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN, "key"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenArgumentsAreNotOddAndKeyExists_returnsSyntaxError() {
-    jedis.sadd("a", "1");
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "a*"))
+    jedis.sadd(KEY, FIELD_ONE);
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "a*"))
         .hasMessageContaining(ERROR_SYNTAX);
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenArgumentsAreNotOddAndKeyDoesNotExist_returnsEmptyArray() {
     List<Object> result =
-        (List<Object>) jedis.sendCommand("key!", Protocol.Command.SSCAN, "key!", "0", "a*");
+        (List<Object>) jedis.sendCommand("key!", Protocol.Command.SSCAN, "key!", ZERO_CURSOR, "a*");
 
-    assertThat((byte[]) result.get(0)).isEqualTo("0".getBytes());
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
     assertThat((List<Object>) result.get(1)).isEmpty();
   }
 
+  @Test
+  public void givenMatchArgumentWithoutPatternOnExistingKey_returnsSyntaxError() {
+    jedis.sadd(KEY, FIELD_ONE);
+
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "MATCH"))
+            .hasMessageContaining(ERROR_SYNTAX);
+  }
+
+  @Test
+  public void givenMatchArgumentWithoutPatternOnNonExistentKey_returnsEmptyArray() {
+    List<Object> result =
+        uncheckedCast(jedis.sendCommand("nonexistentKey", Protocol.Command.SSCAN, "nonexistentKey",
+            ZERO_CURSOR, "MATCH"));
+
+    assertThat((List<?>) result.get(1)).isEmpty();
+  }
+
+  @Test
+  public void givenCountArgumentWithoutNumberOnExistingKey_returnsSyntaxError() {
+    jedis.sadd(KEY, FIELD_ONE);
+
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "COUNT"))
+            .hasMessageContaining(ERROR_SYNTAX);
+  }
+
+  @Test
+  public void givenCountArgumentWithoutNumberOnNonExistentKey_returnsEmptyArray() {
+    List<Object> result =
+        uncheckedCast(
+            jedis.sendCommand("nonexistentKey", Protocol.Command.SSCAN, "nonexistentKey",
+                ZERO_CURSOR, "COUNT"));
+
+    assertThat((List<?>) result.get(1)).isEmpty();
+  }
+
   @Test
   public void givenMatchOrCountKeywordNotSpecified_returnsSyntaxError() {
-    jedis.sadd("a", "1");
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "a*", "1"))
-        .hasMessageContaining(ERROR_SYNTAX);
+    jedis.sadd(KEY, FIELD_ONE);
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "a*", FIELD_ONE))
+            .hasMessageContaining(ERROR_SYNTAX);
   }
 
   @Test
   public void givenCount_whenCountParameterIsNotAnInteger_returnsNotIntegerError() {
-    jedis.sadd("a", "1");
+    jedis.sadd(KEY, FIELD_ONE);
     assertThatThrownBy(
-        () -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "COUNT", "MATCH"))
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "COUNT", "MATCH"))
             .hasMessageContaining(ERROR_NOT_INTEGER);
   }
 
   @Test
   public void givenMultipleCounts_whenAnyCountParameterIsNotAnInteger_returnsNotIntegerError() {
-    jedis.sadd("a", "1");
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "COUNT", "2",
-        "COUNT", "sjlfs", "COUNT", "1"))
-            .hasMessageContaining(ERROR_NOT_INTEGER);
+    jedis.sadd(KEY, FIELD_ONE);
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "COUNT", FIELD_TWO,
+            "COUNT", "sjlfs", "COUNT", FIELD_ONE))
+                .hasMessageContaining(ERROR_NOT_INTEGER);
   }
 
   @Test
   public void givenMultipleCounts_whenAnyCountParameterIsLessThanOne_returnsSyntaxError() {
-    jedis.sadd("a", "1");
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "COUNT", "2",
-        "COUNT", "0", "COUNT", "1"))
-            .hasMessageContaining(ERROR_SYNTAX);
+    jedis.sadd(KEY, FIELD_ONE);
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "COUNT", FIELD_TWO,
+            "COUNT", "0", "COUNT", FIELD_ONE))

Review comment:
       `FIELD_ONE` and `FIELD_TWO` here should be replaced with "1" and "2" (or any other integer value) since they represent count values, not fields.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -156,152 +222,172 @@ public void givenNonexistentKey_andCursorIsNotInteger_returnsInvalidCursorError(
 
   @Test
   public void givenExistentSetKey_andCursorIsNotAnInteger_returnsInvalidCursorError() {
-    jedis.set("a", "b");
+    jedis.set(KEY, "b");
 
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "sjfls"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, "sjfls"))
         .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNonexistentKey_returnsEmptyArray() {
-    ScanResult<String> result = jedis.sscan("nonexistent", "0");
+    ScanResult<String> result = jedis.sscan("nonexistent", ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
     assertThat(result.getResult()).isEmpty();
   }
 
+  @Test
+  public void givenNegativeCursor_returnsEntriesUsingAbsoluteValueOfCursor() {
+    List<String> set = initializeThreeMemberSet();
+
+    String cursor = "-100";
+    ScanResult<String> result;
+    List<String> allEntries = new ArrayList<>();
+
+    do {
+      result = jedis.sscan(KEY, cursor);
+      allEntries.addAll(result.getResult());
+      cursor = result.getCursor();
+    } while (!result.isCompleteIteration());
+
+    assertThat(allEntries).hasSize(3);
+    assertThat(allEntries).containsExactlyInAnyOrderElementsOf(set);
+  }
+
   @Test
   public void givenSetWithOneMember_returnsMember() {
-    jedis.sadd("a", "1");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactly("1");
+    assertThat(result.getResult()).containsExactly(FIELD_ONE);
   }
 
   @Test
   public void givenSetWithMultipleMembers_returnsAllMembers() {

Review comment:
       This test would be better named "givenSetWithThreeMembers_returnsAllMembers" as we certainly don't expect that this behaviour holds true for an arbitrary set size. The assertion should also be changed to `containsOnly()` to allow duplicate entries.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -156,152 +222,172 @@ public void givenNonexistentKey_andCursorIsNotInteger_returnsInvalidCursorError(
 
   @Test
   public void givenExistentSetKey_andCursorIsNotAnInteger_returnsInvalidCursorError() {
-    jedis.set("a", "b");
+    jedis.set(KEY, "b");
 
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "sjfls"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, "sjfls"))
         .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNonexistentKey_returnsEmptyArray() {
-    ScanResult<String> result = jedis.sscan("nonexistent", "0");
+    ScanResult<String> result = jedis.sscan("nonexistent", ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
     assertThat(result.getResult()).isEmpty();
   }
 
+  @Test
+  public void givenNegativeCursor_returnsEntriesUsingAbsoluteValueOfCursor() {
+    List<String> set = initializeThreeMemberSet();
+
+    String cursor = "-100";
+    ScanResult<String> result;
+    List<String> allEntries = new ArrayList<>();
+
+    do {
+      result = jedis.sscan(KEY, cursor);
+      allEntries.addAll(result.getResult());
+      cursor = result.getCursor();
+    } while (!result.isCompleteIteration());
+
+    assertThat(allEntries).hasSize(3);
+    assertThat(allEntries).containsExactlyInAnyOrderElementsOf(set);
+  }
+
   @Test
   public void givenSetWithOneMember_returnsMember() {
-    jedis.sadd("a", "1");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactly("1");
+    assertThat(result.getResult()).containsExactly(FIELD_ONE);
   }
 
   @Test
   public void givenSetWithMultipleMembers_returnsAllMembers() {
-    jedis.sadd("a", "1", "2", "3");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactlyInAnyOrder("1", "2", "3");
+    assertThat(result.getResult()).containsExactlyInAnyOrder(FIELD_ONE, FIELD_TWO, FIELD_THREE);
   }
 
   @Test
   public void givenCount_returnsAllMembersWithoutDuplicates() {

Review comment:
       This test is flawed, as for small set sizes, native Redis will return all elements in one SSCAN regardless of the value of COUNT, and duplicate elements are allowed. A significant improvement would be to populate a set with 1000 elements, then call SSCAN once and assert that the number of elements returned is greater than or equal to the appropriate COUNT value and that they are a subset of the total set contents.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -156,152 +222,172 @@ public void givenNonexistentKey_andCursorIsNotInteger_returnsInvalidCursorError(
 
   @Test
   public void givenExistentSetKey_andCursorIsNotAnInteger_returnsInvalidCursorError() {
-    jedis.set("a", "b");
+    jedis.set(KEY, "b");
 
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "sjfls"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, "sjfls"))
         .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNonexistentKey_returnsEmptyArray() {
-    ScanResult<String> result = jedis.sscan("nonexistent", "0");
+    ScanResult<String> result = jedis.sscan("nonexistent", ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
     assertThat(result.getResult()).isEmpty();
   }
 
+  @Test
+  public void givenNegativeCursor_returnsEntriesUsingAbsoluteValueOfCursor() {
+    List<String> set = initializeThreeMemberSet();
+
+    String cursor = "-100";
+    ScanResult<String> result;
+    List<String> allEntries = new ArrayList<>();
+
+    do {
+      result = jedis.sscan(KEY, cursor);
+      allEntries.addAll(result.getResult());
+      cursor = result.getCursor();
+    } while (!result.isCompleteIteration());
+
+    assertThat(allEntries).hasSize(3);
+    assertThat(allEntries).containsExactlyInAnyOrderElementsOf(set);
+  }
+
   @Test
   public void givenSetWithOneMember_returnsMember() {
-    jedis.sadd("a", "1");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactly("1");
+    assertThat(result.getResult()).containsExactly(FIELD_ONE);
   }
 
   @Test
   public void givenSetWithMultipleMembers_returnsAllMembers() {
-    jedis.sadd("a", "1", "2", "3");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactlyInAnyOrder("1", "2", "3");
+    assertThat(result.getResult()).containsExactlyInAnyOrder(FIELD_ONE, FIELD_TWO, FIELD_THREE);
   }
 
   @Test
   public void givenCount_returnsAllMembersWithoutDuplicates() {
-    jedis.sadd("a", "1", "2", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     ScanParams scanParams = new ScanParams();
     scanParams.count(1);
-    String cursor = "0";
+    String cursor = ZERO_CURSOR;
     ScanResult<byte[]> result;
     List<byte[]> allMembersFromScan = new ArrayList<>();
 
     do {
-      result = jedis.sscan("a".getBytes(), cursor.getBytes(), scanParams);
+      result = jedis.sscan(KEY.getBytes(), cursor.getBytes(), scanParams);
       allMembersFromScan.addAll(result.getResult());
       cursor = result.getCursor();
     } while (!result.isCompleteIteration());
 
-    assertThat(allMembersFromScan).containsExactlyInAnyOrder("1".getBytes(),
-        "2".getBytes(),
-        "3".getBytes());
+    assertThat(allMembersFromScan).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenMultipleCounts_returnsAllEntriesWithoutDuplicates() {

Review comment:
       This test is flawed, as for small set sizes, native Redis will return all elements in one SSCAN regardless of the value of COUNT, and duplicate elements are allowed. A significant improvement would be to populate a set with 1000 elements, then call SSCAN once and assert that the number of elements returned is greater than or equal to the appropriate COUNT value and that they are a subset of the total set contents.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -156,152 +222,172 @@ public void givenNonexistentKey_andCursorIsNotInteger_returnsInvalidCursorError(
 
   @Test
   public void givenExistentSetKey_andCursorIsNotAnInteger_returnsInvalidCursorError() {
-    jedis.set("a", "b");
+    jedis.set(KEY, "b");
 
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "sjfls"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, "sjfls"))
         .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNonexistentKey_returnsEmptyArray() {
-    ScanResult<String> result = jedis.sscan("nonexistent", "0");
+    ScanResult<String> result = jedis.sscan("nonexistent", ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
     assertThat(result.getResult()).isEmpty();
   }
 
+  @Test
+  public void givenNegativeCursor_returnsEntriesUsingAbsoluteValueOfCursor() {
+    List<String> set = initializeThreeMemberSet();
+
+    String cursor = "-100";
+    ScanResult<String> result;
+    List<String> allEntries = new ArrayList<>();
+
+    do {
+      result = jedis.sscan(KEY, cursor);
+      allEntries.addAll(result.getResult());
+      cursor = result.getCursor();
+    } while (!result.isCompleteIteration());
+
+    assertThat(allEntries).hasSize(3);
+    assertThat(allEntries).containsExactlyInAnyOrderElementsOf(set);
+  }
+
   @Test
   public void givenSetWithOneMember_returnsMember() {
-    jedis.sadd("a", "1");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactly("1");
+    assertThat(result.getResult()).containsExactly(FIELD_ONE);
   }
 
   @Test
   public void givenSetWithMultipleMembers_returnsAllMembers() {
-    jedis.sadd("a", "1", "2", "3");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactlyInAnyOrder("1", "2", "3");
+    assertThat(result.getResult()).containsExactlyInAnyOrder(FIELD_ONE, FIELD_TWO, FIELD_THREE);
   }
 
   @Test
   public void givenCount_returnsAllMembersWithoutDuplicates() {
-    jedis.sadd("a", "1", "2", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     ScanParams scanParams = new ScanParams();
     scanParams.count(1);
-    String cursor = "0";
+    String cursor = ZERO_CURSOR;
     ScanResult<byte[]> result;
     List<byte[]> allMembersFromScan = new ArrayList<>();
 
     do {
-      result = jedis.sscan("a".getBytes(), cursor.getBytes(), scanParams);
+      result = jedis.sscan(KEY.getBytes(), cursor.getBytes(), scanParams);
       allMembersFromScan.addAll(result.getResult());
       cursor = result.getCursor();
     } while (!result.isCompleteIteration());
 
-    assertThat(allMembersFromScan).containsExactlyInAnyOrder("1".getBytes(),
-        "2".getBytes(),
-        "3".getBytes());
+    assertThat(allMembersFromScan).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenMultipleCounts_returnsAllEntriesWithoutDuplicates() {
-    jedis.sadd("a", "1", "12", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     List<Object> result;
 
     List<byte[]> allEntries = new ArrayList<>();
-    String cursor = "0";
+    String cursor = ZERO_CURSOR;
 
     do {
       result =
-          (List<Object>) jedis.sendCommand("a", Protocol.Command.SSCAN, "a", cursor, "COUNT", "2",
-              "COUNT", "1");
+          (List<Object>) jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, cursor, "COUNT",
+              FIELD_TWO,
+              "COUNT", FIELD_ONE);
       allEntries.addAll((List<byte[]>) result.get(1));
       cursor = new String((byte[]) result.get(0));
-    } while (!Arrays.equals((byte[]) result.get(0), "0".getBytes()));
+    } while (!Arrays.equals((byte[]) result.get(0), ZERO_CURSOR.getBytes()));
 
-    assertThat((byte[]) result.get(0)).isEqualTo("0".getBytes());
-    assertThat(allEntries).containsExactlyInAnyOrder("1".getBytes(),
-        "12".getBytes(),
-        "3".getBytes());
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
+    assertThat(allEntries).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
   }
 
   @Test
   public void givenMatch_returnsAllMatchingMembersWithoutDuplicates() {
-    jedis.sadd("a", "1", "12", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     ScanParams scanParams = new ScanParams();
     scanParams.match("1*");
 
     ScanResult<byte[]> result =
-        jedis.sscan("a".getBytes(), "0".getBytes(), scanParams);
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactlyInAnyOrder("1".getBytes(),
-        "12".getBytes());
+    assertThat(result.getResult()).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes());
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenMultipleMatches_returnsMembersMatchingLastMatchParameter() {
-    jedis.sadd("a", "1", "12", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
-    List<Object> result = (List<Object>) jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0",
-        "MATCH", "3*", "MATCH", "1*");
+    List<Object> result =
+        (List<Object>) jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR,
+            "MATCH", "3*", "MATCH", "1*");
 
-    assertThat((byte[]) result.get(0)).isEqualTo("0".getBytes());
-    assertThat((List<byte[]>) result.get(1)).containsExactlyInAnyOrder("1".getBytes(),
-        "12".getBytes());
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
+    assertThat((List<byte[]>) result.get(1)).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes());
   }
 
   @Test
   public void givenMatchAndCount_returnsAllMembersWithoutDuplicates() {

Review comment:
       This test would be better named "givenSetWithThreeMembersAndMatchAndCount_returnsAllMatchingMembers" with the assertions changed to `containsOnly()` to allow duplicate entries.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -323,26 +409,217 @@ public void givenNegativeCursor_returnsMembersUsingAbsoluteValueOfCursor() {
 
   @Test
   public void givenCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, UNSIGNED_LONG_CAPACITY.add(BigInteger.ONE).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNegativeCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "-18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, SIGNED_LONG_CAPACITY.add(BigInteger.valueOf(-1)).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenInvalidRegexSyntax_returnsEmptyArray() {
-    jedis.sadd("a", "1");
+    jedis.sadd(KEY, FIELD_ONE);
     ScanParams scanParams = new ScanParams();
     scanParams.count(1);
     scanParams.match("\\p");
 
     ScanResult<byte[]> result =
-        jedis.sscan("a".getBytes(), "0".getBytes(), scanParams);
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
 
     assertThat(result.getResult()).isEmpty();
   }
+
+  @Test
+  public void should_notReturnValue_givenValueWasRemovedBeforeSSCANISCalled() {
+    List<String> set = initializeThreeMemberSet();
+
+    jedis.srem(KEY, FIELD_THREE);
+    set.remove(FIELD_THREE);
+
+    GeodeAwaitility.await().untilAsserted(
+        () -> assertThat(jedis.sismember(KEY, FIELD_THREE)).isFalse());
+
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
+
+    assertThat(result.getResult()).containsExactlyInAnyOrderElementsOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenNonzeroCursorOnFirstCall() {
+    List<String> set = initializeThreeMemberSet();
+
+    ScanResult<String> result = jedis.sscan(KEY, "5");
+
+    assertThat(result.getResult()).isSubsetOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenCountEqualToIntegerMaxValue() {
+    List<byte[]> set = initializeThreeMemberByteSet();
+
+    ScanParams scanParams = new ScanParams().count(Integer.MAX_VALUE);
+
+    ScanResult<byte[]> result =
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
+    assertThat(result.getResult())
+        .containsExactlyInAnyOrderElementsOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenCountGreaterThanIntegerMaxValue() {
+    initializeThreeMemberByteSet();
+
+    String greaterThanInt = String.valueOf(Integer.MAX_VALUE);

Review comment:
       This value is equal to `Integer.MAX_VALUE` rather than being greater than it. This should be:
   ```
   String greaterThanInt = String.valueOf(2L * Integer.MAX_VALUE);
   ```

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -323,26 +409,217 @@ public void givenNegativeCursor_returnsMembersUsingAbsoluteValueOfCursor() {
 
   @Test
   public void givenCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, UNSIGNED_LONG_CAPACITY.add(BigInteger.ONE).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNegativeCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "-18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, SIGNED_LONG_CAPACITY.add(BigInteger.valueOf(-1)).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenInvalidRegexSyntax_returnsEmptyArray() {
-    jedis.sadd("a", "1");
+    jedis.sadd(KEY, FIELD_ONE);
     ScanParams scanParams = new ScanParams();
     scanParams.count(1);
     scanParams.match("\\p");
 
     ScanResult<byte[]> result =
-        jedis.sscan("a".getBytes(), "0".getBytes(), scanParams);
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
 
     assertThat(result.getResult()).isEmpty();
   }
+
+  @Test
+  public void should_notReturnValue_givenValueWasRemovedBeforeSSCANISCalled() {
+    List<String> set = initializeThreeMemberSet();
+
+    jedis.srem(KEY, FIELD_THREE);
+    set.remove(FIELD_THREE);
+
+    GeodeAwaitility.await().untilAsserted(
+        () -> assertThat(jedis.sismember(KEY, FIELD_THREE)).isFalse());
+
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
+
+    assertThat(result.getResult()).containsExactlyInAnyOrderElementsOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenNonzeroCursorOnFirstCall() {
+    List<String> set = initializeThreeMemberSet();
+
+    ScanResult<String> result = jedis.sscan(KEY, "5");
+
+    assertThat(result.getResult()).isSubsetOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenCountEqualToIntegerMaxValue() {
+    List<byte[]> set = initializeThreeMemberByteSet();
+
+    ScanParams scanParams = new ScanParams().count(Integer.MAX_VALUE);
+
+    ScanResult<byte[]> result =
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
+    assertThat(result.getResult())
+        .containsExactlyInAnyOrderElementsOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenCountGreaterThanIntegerMaxValue() {
+    initializeThreeMemberByteSet();
+
+    String greaterThanInt = String.valueOf(Integer.MAX_VALUE);
+    List<Object> result =
+        uncheckedCast(jedis.sendCommand(KEY.getBytes(), Protocol.Command.SSCAN,
+            KEY.getBytes(), ZERO_CURSOR.getBytes(),
+            "COUNT".getBytes(), greaterThanInt.getBytes()));
+
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
+
+    List<byte[]> fields = uncheckedCast(result.get(1));
+    assertThat(fields).containsExactlyInAnyOrder(
+        FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
+  }
+
+  /**** Concurrency ***/
+
+  @Test
+  public void should_notLoseFields_givenConcurrentThreadsDoingSScansAndChangingValues() {
+    final List<String> initialMemberData = makeSet();
+    jedis.sadd(KEY, initialMemberData.toArray(new String[initialMemberData.size()]));
+    final int iterationCount = 500;
+
+    Jedis jedis1 = jedis.getConnectionFromSlot(SLOT_FOR_KEY);
+    Jedis jedis2 = jedis.getConnectionFromSlot(SLOT_FOR_KEY);
+
+    new ConcurrentLoopingThreads(iterationCount,
+        (i) -> multipleSScanAndAssertOnSizeOfResultSet(jedis1, initialMemberData),
+        (i) -> multipleSScanAndAssertOnSizeOfResultSet(jedis2, initialMemberData),
+        (i) -> {
+          int fieldSuffix = i % SIZE_OF_SET;
+          jedis.sadd(KEY, BASE_FIELD + fieldSuffix);
+        }).run();
+
+    jedis1.close();
+    jedis2.close();
+  }
+
+  @Test
+  public void should_notLoseKeysForConsistentlyPresentFields_givenConcurrentThreadsAddingAndRemovingFields() {

Review comment:
       This test's name and variables should be updated to reflect that fact that it's testing a set and not a hash. Sets don't have keys, fields or values, they only have members. The name could be something like "should_returnAllConsistentlyPresentMembers_givenConcurrentThreadsAddingAndRemovingMembers"

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -323,26 +409,217 @@ public void givenNegativeCursor_returnsMembersUsingAbsoluteValueOfCursor() {
 
   @Test
   public void givenCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, UNSIGNED_LONG_CAPACITY.add(BigInteger.ONE).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNegativeCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {

Review comment:
       It would be good to add tests that show the behaviour when the cursor value is outside the range `Long.MIN_VALUE > x > Long.MAX_VALUE`, and tests for the behaviour when the cursor in just inside that range.
   
   The behaviour of geode-for-redis differs from native Redis here, as they accept cursor values up to `UNSIGNED_LONG_CAPACITY` but we only accept ones up to `Long.MAX_VALUE`, so in order to test this, the test cases for failing with values outside the range should be put in `SScanIntegrationTest` (not the Abstract parent class). Also, looking at that class, the test `givenDifferentCursorThanSpecifiedByPreviousSscan_returnsAllMembers` is wrong and should be removed.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -323,26 +409,217 @@ public void givenNegativeCursor_returnsMembersUsingAbsoluteValueOfCursor() {
 
   @Test
   public void givenCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, UNSIGNED_LONG_CAPACITY.add(BigInteger.ONE).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNegativeCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "-18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, SIGNED_LONG_CAPACITY.add(BigInteger.valueOf(-1)).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenInvalidRegexSyntax_returnsEmptyArray() {
-    jedis.sadd("a", "1");
+    jedis.sadd(KEY, FIELD_ONE);
     ScanParams scanParams = new ScanParams();
     scanParams.count(1);
     scanParams.match("\\p");
 
     ScanResult<byte[]> result =
-        jedis.sscan("a".getBytes(), "0".getBytes(), scanParams);
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
 
     assertThat(result.getResult()).isEmpty();
   }
+
+  @Test
+  public void should_notReturnValue_givenValueWasRemovedBeforeSSCANISCalled() {
+    List<String> set = initializeThreeMemberSet();
+
+    jedis.srem(KEY, FIELD_THREE);
+    set.remove(FIELD_THREE);
+
+    GeodeAwaitility.await().untilAsserted(
+        () -> assertThat(jedis.sismember(KEY, FIELD_THREE)).isFalse());
+
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
+
+    assertThat(result.getResult()).containsExactlyInAnyOrderElementsOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenNonzeroCursorOnFirstCall() {
+    List<String> set = initializeThreeMemberSet();
+
+    ScanResult<String> result = jedis.sscan(KEY, "5");
+
+    assertThat(result.getResult()).isSubsetOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenCountEqualToIntegerMaxValue() {
+    List<byte[]> set = initializeThreeMemberByteSet();
+
+    ScanParams scanParams = new ScanParams().count(Integer.MAX_VALUE);
+
+    ScanResult<byte[]> result =
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
+    assertThat(result.getResult())
+        .containsExactlyInAnyOrderElementsOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenCountGreaterThanIntegerMaxValue() {
+    initializeThreeMemberByteSet();
+
+    String greaterThanInt = String.valueOf(Integer.MAX_VALUE);
+    List<Object> result =
+        uncheckedCast(jedis.sendCommand(KEY.getBytes(), Protocol.Command.SSCAN,
+            KEY.getBytes(), ZERO_CURSOR.getBytes(),
+            "COUNT".getBytes(), greaterThanInt.getBytes()));
+
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
+
+    List<byte[]> fields = uncheckedCast(result.get(1));
+    assertThat(fields).containsExactlyInAnyOrder(
+        FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
+  }
+
+  /**** Concurrency ***/
+
+  @Test
+  public void should_notLoseFields_givenConcurrentThreadsDoingSScansAndChangingValues() {

Review comment:
       This test is not really doing anything, since the calls to `sadd` in the `ConcurrentLoopingThreads` are just adding already existing members. I suspect that the test this is copied from was for a Redis Hash, which has fields and values, and the values were being modified while asserting that this didn't result in the fields not showing up in an HSCAN, but this test case doesn't make sense for a Redis Set, so this should just be removed.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -323,26 +409,217 @@ public void givenNegativeCursor_returnsMembersUsingAbsoluteValueOfCursor() {
 
   @Test
   public void givenCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, UNSIGNED_LONG_CAPACITY.add(BigInteger.ONE).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNegativeCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "-18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, SIGNED_LONG_CAPACITY.add(BigInteger.valueOf(-1)).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenInvalidRegexSyntax_returnsEmptyArray() {

Review comment:
       This test is flawed, as there is no such thing as an invalid syntax for glob-style patterns. This test is just testing that given a non-matching pattern, an empty array is returned, so the name should reflect that. In fact, if the test is modified to the below, then it fails, as the element `p` matches and is returned:
   ```
     @Test
     public void givenInvalidRegexSyntax_returnsEmptyArray() {
       jedis.sadd(KEY, "\\p", "p");
       ScanParams scanParams = new ScanParams();
       scanParams.count(10);
       scanParams.match("\\p");
   
       ScanResult<byte[]> result =
           jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
   
       assertThat(result.getResult()).isEmpty();
     }
   ```

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -14,33 +14,53 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertAtLeastNArgs;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_CURSOR;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_SYNTAX;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
+import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.Jedis;
 import redis.clients.jedis.JedisCluster;
 import redis.clients.jedis.Protocol;
 import redis.clients.jedis.ScanParams;
 import redis.clients.jedis.ScanResult;
 
+import org.apache.geode.redis.ConcurrentLoopingThreads;
 import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.redis.internal.commands.executor.cluster.CRC16;
+import org.apache.geode.redis.internal.services.RegionProvider;
 import org.apache.geode.test.awaitility.GeodeAwaitility;
 
 public abstract class AbstractSScanIntegrationTest implements RedisIntegrationTest {
   protected JedisCluster jedis;
   private static final int REDIS_CLIENT_TIMEOUT =
       Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  public static final String KEY = "key";
+  public static final int SLOT_FOR_KEY = CRC16.calculate(KEY) % RegionProvider.REDIS_SLOTS;

Review comment:
       This could be simplified slightly to `KeyHashUtil.slotForKey(KEY.getBytes());`. If you wanted to, you could even add a method to `KeyHashUtil` that takes a String argument and then converts to to a byte array using `Coder.stringToBytes()` then calls the `byte[]` argument version, so we don't have to call `getBytes()` here.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -53,97 +73,143 @@ public void tearDown() {
     jedis.close();
   }
 
+  @Test
+  public void givenLessThanTwoArguments_returnsWrongNumberOfArgumentsError() {
+    assertAtLeastNArgs(jedis, Protocol.Command.SSCAN, 2);
+  }
+
   @Test
   public void givenNoKeyArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenNoCursorArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN, "key"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenArgumentsAreNotOddAndKeyExists_returnsSyntaxError() {
-    jedis.sadd("a", "1");
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "a*"))
+    jedis.sadd(KEY, FIELD_ONE);
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "a*"))
         .hasMessageContaining(ERROR_SYNTAX);
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenArgumentsAreNotOddAndKeyDoesNotExist_returnsEmptyArray() {
     List<Object> result =
-        (List<Object>) jedis.sendCommand("key!", Protocol.Command.SSCAN, "key!", "0", "a*");
+        (List<Object>) jedis.sendCommand("key!", Protocol.Command.SSCAN, "key!", ZERO_CURSOR, "a*");
 
-    assertThat((byte[]) result.get(0)).isEqualTo("0".getBytes());
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
     assertThat((List<Object>) result.get(1)).isEmpty();
   }
 
+  @Test
+  public void givenMatchArgumentWithoutPatternOnExistingKey_returnsSyntaxError() {
+    jedis.sadd(KEY, FIELD_ONE);
+
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "MATCH"))
+            .hasMessageContaining(ERROR_SYNTAX);
+  }
+
+  @Test
+  public void givenMatchArgumentWithoutPatternOnNonExistentKey_returnsEmptyArray() {
+    List<Object> result =
+        uncheckedCast(jedis.sendCommand("nonexistentKey", Protocol.Command.SSCAN, "nonexistentKey",
+            ZERO_CURSOR, "MATCH"));
+
+    assertThat((List<?>) result.get(1)).isEmpty();
+  }
+
+  @Test
+  public void givenCountArgumentWithoutNumberOnExistingKey_returnsSyntaxError() {
+    jedis.sadd(KEY, FIELD_ONE);
+
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "COUNT"))
+            .hasMessageContaining(ERROR_SYNTAX);
+  }
+
+  @Test
+  public void givenCountArgumentWithoutNumberOnNonExistentKey_returnsEmptyArray() {
+    List<Object> result =
+        uncheckedCast(
+            jedis.sendCommand("nonexistentKey", Protocol.Command.SSCAN, "nonexistentKey",
+                ZERO_CURSOR, "COUNT"));
+
+    assertThat((List<?>) result.get(1)).isEmpty();
+  }
+
   @Test
   public void givenMatchOrCountKeywordNotSpecified_returnsSyntaxError() {
-    jedis.sadd("a", "1");
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "a*", "1"))
-        .hasMessageContaining(ERROR_SYNTAX);
+    jedis.sadd(KEY, FIELD_ONE);
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "a*", FIELD_ONE))

Review comment:
       Rather than using `FIELD_ONE` here, it's okay to use "1" (or any other integer really) as this argument is intended to be the `COUNT` value and using `FIELD_ONE` could cause confusion.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -156,152 +222,172 @@ public void givenNonexistentKey_andCursorIsNotInteger_returnsInvalidCursorError(
 
   @Test
   public void givenExistentSetKey_andCursorIsNotAnInteger_returnsInvalidCursorError() {
-    jedis.set("a", "b");
+    jedis.set(KEY, "b");
 
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "sjfls"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, "sjfls"))
         .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNonexistentKey_returnsEmptyArray() {
-    ScanResult<String> result = jedis.sscan("nonexistent", "0");
+    ScanResult<String> result = jedis.sscan("nonexistent", ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
     assertThat(result.getResult()).isEmpty();
   }
 
+  @Test
+  public void givenNegativeCursor_returnsEntriesUsingAbsoluteValueOfCursor() {

Review comment:
       This test (and any other similar ones) is incorrect, as this is not the behaviour seen with native Redis. For example, if a set is loaded with 1000 entries using the redis cli, the following results are seen:
   ```
   127.0.0.1:6379> sscan key 10
   1) "586"
   2)  1) "928"
       2) "884"
       3) "28"
       4) "834"
       5) "411"
       6) "397"
       7) "346"
       8) "902"
       9) "218"
      10) "586"
      11) "440"
   127.0.0.1:6379> sscan key -10
   1) "398"
   2)  1) "688"
       2) "41"
       3) "9"
       4) "204"
       5) "674"
       6) "639"
       7) "550"
       8) "979"
       9) "2"
      10) "677"
      11) "643"
   ```
   showing that a cursor value of -10 is not equivalent to a cursor value of 10. Moreover, the only valid values for CURSOR in a SCAN command are 0 or the previously returned value, so behaviour with a negative cursor value is technically undefined (see [Redis documentation for SCAN](https://redis.io/commands/scan#calling-scan-with-a-corrupted-cursor)). This test would be better as just "negativeCursor_doesNotError" as that's all we guarantee.  

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -53,97 +73,143 @@ public void tearDown() {
     jedis.close();
   }
 
+  @Test
+  public void givenLessThanTwoArguments_returnsWrongNumberOfArgumentsError() {
+    assertAtLeastNArgs(jedis, Protocol.Command.SSCAN, 2);
+  }
+
   @Test
   public void givenNoKeyArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenNoCursorArgument_returnsWrongNumberOfArgumentsError() {
-    assertThatThrownBy(() -> jedis.sendCommand("key", Protocol.Command.SSCAN, "key"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY))
         .hasMessageContaining("ERR wrong number of arguments for 'sscan' command");
   }
 
   @Test
   public void givenArgumentsAreNotOddAndKeyExists_returnsSyntaxError() {
-    jedis.sadd("a", "1");
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0", "a*"))
+    jedis.sadd(KEY, FIELD_ONE);
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR, "a*"))
         .hasMessageContaining(ERROR_SYNTAX);
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenArgumentsAreNotOddAndKeyDoesNotExist_returnsEmptyArray() {

Review comment:
       This test name could be more accurate as "givenIncorrectOptionalArgumentAndKeyDoesNotExist_returnsEmptyArray" since the number of arguments isn't actually the relevant factor here, just that there is a syntax error of some kind.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/RedisCommandType.java
##########
@@ -252,6 +252,8 @@
       new Parameter().exact(2).flags(READONLY, SORT_FOR_SCRIPT)),
   SMOVE(new SMoveExecutor(), SUPPORTED, new Parameter().exact(4).lastKey(2).flags(WRITE, FAST)),
   SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(WRITE, FAST)),
+  SSCAN(new SScanExecutor(), UNSUPPORTED, new Parameter().min(3).flags(READONLY, RANDOM),

Review comment:
       This command should be using the `SUPPORTED` value of RedisCommandSupportLevel, not `UNSUPPORTED`. Also, I think that the `firstKey(0)` flag should not be present and can be removed.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -156,152 +222,172 @@ public void givenNonexistentKey_andCursorIsNotInteger_returnsInvalidCursorError(
 
   @Test
   public void givenExistentSetKey_andCursorIsNotAnInteger_returnsInvalidCursorError() {
-    jedis.set("a", "b");
+    jedis.set(KEY, "b");
 
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "sjfls"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, "sjfls"))
         .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNonexistentKey_returnsEmptyArray() {
-    ScanResult<String> result = jedis.sscan("nonexistent", "0");
+    ScanResult<String> result = jedis.sscan("nonexistent", ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
     assertThat(result.getResult()).isEmpty();
   }
 
+  @Test
+  public void givenNegativeCursor_returnsEntriesUsingAbsoluteValueOfCursor() {
+    List<String> set = initializeThreeMemberSet();
+
+    String cursor = "-100";
+    ScanResult<String> result;
+    List<String> allEntries = new ArrayList<>();
+
+    do {
+      result = jedis.sscan(KEY, cursor);
+      allEntries.addAll(result.getResult());
+      cursor = result.getCursor();
+    } while (!result.isCompleteIteration());
+
+    assertThat(allEntries).hasSize(3);
+    assertThat(allEntries).containsExactlyInAnyOrderElementsOf(set);
+  }
+
   @Test
   public void givenSetWithOneMember_returnsMember() {
-    jedis.sadd("a", "1");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactly("1");
+    assertThat(result.getResult()).containsExactly(FIELD_ONE);
   }
 
   @Test
   public void givenSetWithMultipleMembers_returnsAllMembers() {
-    jedis.sadd("a", "1", "2", "3");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactlyInAnyOrder("1", "2", "3");
+    assertThat(result.getResult()).containsExactlyInAnyOrder(FIELD_ONE, FIELD_TWO, FIELD_THREE);
   }
 
   @Test
   public void givenCount_returnsAllMembersWithoutDuplicates() {
-    jedis.sadd("a", "1", "2", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     ScanParams scanParams = new ScanParams();
     scanParams.count(1);
-    String cursor = "0";
+    String cursor = ZERO_CURSOR;
     ScanResult<byte[]> result;
     List<byte[]> allMembersFromScan = new ArrayList<>();
 
     do {
-      result = jedis.sscan("a".getBytes(), cursor.getBytes(), scanParams);
+      result = jedis.sscan(KEY.getBytes(), cursor.getBytes(), scanParams);
       allMembersFromScan.addAll(result.getResult());
       cursor = result.getCursor();
     } while (!result.isCompleteIteration());
 
-    assertThat(allMembersFromScan).containsExactlyInAnyOrder("1".getBytes(),
-        "2".getBytes(),
-        "3".getBytes());
+    assertThat(allMembersFromScan).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenMultipleCounts_returnsAllEntriesWithoutDuplicates() {
-    jedis.sadd("a", "1", "12", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     List<Object> result;
 
     List<byte[]> allEntries = new ArrayList<>();
-    String cursor = "0";
+    String cursor = ZERO_CURSOR;
 
     do {
       result =
-          (List<Object>) jedis.sendCommand("a", Protocol.Command.SSCAN, "a", cursor, "COUNT", "2",
-              "COUNT", "1");
+          (List<Object>) jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, cursor, "COUNT",
+              FIELD_TWO,
+              "COUNT", FIELD_ONE);
       allEntries.addAll((List<byte[]>) result.get(1));
       cursor = new String((byte[]) result.get(0));
-    } while (!Arrays.equals((byte[]) result.get(0), "0".getBytes()));
+    } while (!Arrays.equals((byte[]) result.get(0), ZERO_CURSOR.getBytes()));
 
-    assertThat((byte[]) result.get(0)).isEqualTo("0".getBytes());
-    assertThat(allEntries).containsExactlyInAnyOrder("1".getBytes(),
-        "12".getBytes(),
-        "3".getBytes());
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
+    assertThat(allEntries).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
   }
 
   @Test
   public void givenMatch_returnsAllMatchingMembersWithoutDuplicates() {

Review comment:
       This test is also flawed for the same reasons as earlier tests. A better name would be "givenSetWithThreeEntriesAndMatch_returnsOnlyMatchingElements" with the assertions changed to `containsOnly()` to allow duplicate entries.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -14,33 +14,53 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertAtLeastNArgs;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_CURSOR;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_SYNTAX;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
+import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.Jedis;
 import redis.clients.jedis.JedisCluster;
 import redis.clients.jedis.Protocol;
 import redis.clients.jedis.ScanParams;
 import redis.clients.jedis.ScanResult;
 
+import org.apache.geode.redis.ConcurrentLoopingThreads;
 import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.redis.internal.commands.executor.cluster.CRC16;
+import org.apache.geode.redis.internal.services.RegionProvider;
 import org.apache.geode.test.awaitility.GeodeAwaitility;
 
 public abstract class AbstractSScanIntegrationTest implements RedisIntegrationTest {
   protected JedisCluster jedis;
   private static final int REDIS_CLIENT_TIMEOUT =
       Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  public static final String KEY = "key";
+  public static final int SLOT_FOR_KEY = CRC16.calculate(KEY) % RegionProvider.REDIS_SLOTS;
+  public static final String ZERO_CURSOR = "0";
+  public static final BigInteger UNSIGNED_LONG_CAPACITY = new BigInteger("18446744073709551615");
+  public static final BigInteger SIGNED_LONG_CAPACITY = new BigInteger("-18446744073709551615");

Review comment:
       This would be better named "NEGATIVE_LONG_CAPACITY" since it's not accurate to say that the capacity of a signed long is -18446744073709551615.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -156,152 +222,172 @@ public void givenNonexistentKey_andCursorIsNotInteger_returnsInvalidCursorError(
 
   @Test
   public void givenExistentSetKey_andCursorIsNotAnInteger_returnsInvalidCursorError() {
-    jedis.set("a", "b");
+    jedis.set(KEY, "b");
 
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "sjfls"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, "sjfls"))
         .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNonexistentKey_returnsEmptyArray() {
-    ScanResult<String> result = jedis.sscan("nonexistent", "0");
+    ScanResult<String> result = jedis.sscan("nonexistent", ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
     assertThat(result.getResult()).isEmpty();
   }
 
+  @Test
+  public void givenNegativeCursor_returnsEntriesUsingAbsoluteValueOfCursor() {
+    List<String> set = initializeThreeMemberSet();
+
+    String cursor = "-100";
+    ScanResult<String> result;
+    List<String> allEntries = new ArrayList<>();
+
+    do {
+      result = jedis.sscan(KEY, cursor);
+      allEntries.addAll(result.getResult());
+      cursor = result.getCursor();
+    } while (!result.isCompleteIteration());
+
+    assertThat(allEntries).hasSize(3);
+    assertThat(allEntries).containsExactlyInAnyOrderElementsOf(set);
+  }
+
   @Test
   public void givenSetWithOneMember_returnsMember() {
-    jedis.sadd("a", "1");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactly("1");
+    assertThat(result.getResult()).containsExactly(FIELD_ONE);
   }
 
   @Test
   public void givenSetWithMultipleMembers_returnsAllMembers() {
-    jedis.sadd("a", "1", "2", "3");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactlyInAnyOrder("1", "2", "3");
+    assertThat(result.getResult()).containsExactlyInAnyOrder(FIELD_ONE, FIELD_TWO, FIELD_THREE);
   }
 
   @Test
   public void givenCount_returnsAllMembersWithoutDuplicates() {
-    jedis.sadd("a", "1", "2", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     ScanParams scanParams = new ScanParams();
     scanParams.count(1);
-    String cursor = "0";
+    String cursor = ZERO_CURSOR;
     ScanResult<byte[]> result;
     List<byte[]> allMembersFromScan = new ArrayList<>();
 
     do {
-      result = jedis.sscan("a".getBytes(), cursor.getBytes(), scanParams);
+      result = jedis.sscan(KEY.getBytes(), cursor.getBytes(), scanParams);
       allMembersFromScan.addAll(result.getResult());
       cursor = result.getCursor();
     } while (!result.isCompleteIteration());
 
-    assertThat(allMembersFromScan).containsExactlyInAnyOrder("1".getBytes(),
-        "2".getBytes(),
-        "3".getBytes());
+    assertThat(allMembersFromScan).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenMultipleCounts_returnsAllEntriesWithoutDuplicates() {
-    jedis.sadd("a", "1", "12", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     List<Object> result;
 
     List<byte[]> allEntries = new ArrayList<>();
-    String cursor = "0";
+    String cursor = ZERO_CURSOR;
 
     do {
       result =
-          (List<Object>) jedis.sendCommand("a", Protocol.Command.SSCAN, "a", cursor, "COUNT", "2",
-              "COUNT", "1");
+          (List<Object>) jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, cursor, "COUNT",
+              FIELD_TWO,
+              "COUNT", FIELD_ONE);
       allEntries.addAll((List<byte[]>) result.get(1));
       cursor = new String((byte[]) result.get(0));
-    } while (!Arrays.equals((byte[]) result.get(0), "0".getBytes()));
+    } while (!Arrays.equals((byte[]) result.get(0), ZERO_CURSOR.getBytes()));
 
-    assertThat((byte[]) result.get(0)).isEqualTo("0".getBytes());
-    assertThat(allEntries).containsExactlyInAnyOrder("1".getBytes(),
-        "12".getBytes(),
-        "3".getBytes());
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
+    assertThat(allEntries).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
   }
 
   @Test
   public void givenMatch_returnsAllMatchingMembersWithoutDuplicates() {
-    jedis.sadd("a", "1", "12", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     ScanParams scanParams = new ScanParams();
     scanParams.match("1*");
 
     ScanResult<byte[]> result =
-        jedis.sscan("a".getBytes(), "0".getBytes(), scanParams);
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactlyInAnyOrder("1".getBytes(),
-        "12".getBytes());
+    assertThat(result.getResult()).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes());
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenMultipleMatches_returnsMembersMatchingLastMatchParameter() {

Review comment:
       This test is also flawed for the same reasons as earlier tests. A better name would be "givenSetWithThreeEntriesAndMultipleMatchArguments_returnsOnlyElementsMatchingLastMatchArgument" with the assertions changed to `containsOnly()` to allow duplicate entries.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -156,152 +222,172 @@ public void givenNonexistentKey_andCursorIsNotInteger_returnsInvalidCursorError(
 
   @Test
   public void givenExistentSetKey_andCursorIsNotAnInteger_returnsInvalidCursorError() {
-    jedis.set("a", "b");
+    jedis.set(KEY, "b");
 
-    assertThatThrownBy(() -> jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "sjfls"))
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, "sjfls"))
         .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNonexistentKey_returnsEmptyArray() {
-    ScanResult<String> result = jedis.sscan("nonexistent", "0");
+    ScanResult<String> result = jedis.sscan("nonexistent", ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
     assertThat(result.getResult()).isEmpty();
   }
 
+  @Test
+  public void givenNegativeCursor_returnsEntriesUsingAbsoluteValueOfCursor() {
+    List<String> set = initializeThreeMemberSet();
+
+    String cursor = "-100";
+    ScanResult<String> result;
+    List<String> allEntries = new ArrayList<>();
+
+    do {
+      result = jedis.sscan(KEY, cursor);
+      allEntries.addAll(result.getResult());
+      cursor = result.getCursor();
+    } while (!result.isCompleteIteration());
+
+    assertThat(allEntries).hasSize(3);
+    assertThat(allEntries).containsExactlyInAnyOrderElementsOf(set);
+  }
+
   @Test
   public void givenSetWithOneMember_returnsMember() {
-    jedis.sadd("a", "1");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactly("1");
+    assertThat(result.getResult()).containsExactly(FIELD_ONE);
   }
 
   @Test
   public void givenSetWithMultipleMembers_returnsAllMembers() {
-    jedis.sadd("a", "1", "2", "3");
-    ScanResult<String> result = jedis.sscan("a", "0");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactlyInAnyOrder("1", "2", "3");
+    assertThat(result.getResult()).containsExactlyInAnyOrder(FIELD_ONE, FIELD_TWO, FIELD_THREE);
   }
 
   @Test
   public void givenCount_returnsAllMembersWithoutDuplicates() {
-    jedis.sadd("a", "1", "2", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     ScanParams scanParams = new ScanParams();
     scanParams.count(1);
-    String cursor = "0";
+    String cursor = ZERO_CURSOR;
     ScanResult<byte[]> result;
     List<byte[]> allMembersFromScan = new ArrayList<>();
 
     do {
-      result = jedis.sscan("a".getBytes(), cursor.getBytes(), scanParams);
+      result = jedis.sscan(KEY.getBytes(), cursor.getBytes(), scanParams);
       allMembersFromScan.addAll(result.getResult());
       cursor = result.getCursor();
     } while (!result.isCompleteIteration());
 
-    assertThat(allMembersFromScan).containsExactlyInAnyOrder("1".getBytes(),
-        "2".getBytes(),
-        "3".getBytes());
+    assertThat(allMembersFromScan).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenMultipleCounts_returnsAllEntriesWithoutDuplicates() {
-    jedis.sadd("a", "1", "12", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     List<Object> result;
 
     List<byte[]> allEntries = new ArrayList<>();
-    String cursor = "0";
+    String cursor = ZERO_CURSOR;
 
     do {
       result =
-          (List<Object>) jedis.sendCommand("a", Protocol.Command.SSCAN, "a", cursor, "COUNT", "2",
-              "COUNT", "1");
+          (List<Object>) jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, cursor, "COUNT",
+              FIELD_TWO,
+              "COUNT", FIELD_ONE);
       allEntries.addAll((List<byte[]>) result.get(1));
       cursor = new String((byte[]) result.get(0));
-    } while (!Arrays.equals((byte[]) result.get(0), "0".getBytes()));
+    } while (!Arrays.equals((byte[]) result.get(0), ZERO_CURSOR.getBytes()));
 
-    assertThat((byte[]) result.get(0)).isEqualTo("0".getBytes());
-    assertThat(allEntries).containsExactlyInAnyOrder("1".getBytes(),
-        "12".getBytes(),
-        "3".getBytes());
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
+    assertThat(allEntries).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
   }
 
   @Test
   public void givenMatch_returnsAllMatchingMembersWithoutDuplicates() {
-    jedis.sadd("a", "1", "12", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     ScanParams scanParams = new ScanParams();
     scanParams.match("1*");
 
     ScanResult<byte[]> result =
-        jedis.sscan("a".getBytes(), "0".getBytes(), scanParams);
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
 
     assertThat(result.isCompleteIteration()).isTrue();
-    assertThat(result.getResult()).containsExactlyInAnyOrder("1".getBytes(),
-        "12".getBytes());
+    assertThat(result.getResult()).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes());
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenMultipleMatches_returnsMembersMatchingLastMatchParameter() {
-    jedis.sadd("a", "1", "12", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
-    List<Object> result = (List<Object>) jedis.sendCommand("a", Protocol.Command.SSCAN, "a", "0",
-        "MATCH", "3*", "MATCH", "1*");
+    List<Object> result =
+        (List<Object>) jedis.sendCommand(KEY, Protocol.Command.SSCAN, KEY, ZERO_CURSOR,
+            "MATCH", "3*", "MATCH", "1*");
 
-    assertThat((byte[]) result.get(0)).isEqualTo("0".getBytes());
-    assertThat((List<byte[]>) result.get(1)).containsExactlyInAnyOrder("1".getBytes(),
-        "12".getBytes());
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
+    assertThat((List<byte[]>) result.get(1)).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes());
   }
 
   @Test
   public void givenMatchAndCount_returnsAllMembersWithoutDuplicates() {
-    jedis.sadd("a", "1", "12", "3");
+    jedis.sadd(KEY, FIELD_ONE, FIELD_TWO, FIELD_THREE);
 
     ScanParams scanParams = new ScanParams();
     scanParams.count(1);
     scanParams.match("1*");
     ScanResult<byte[]> result;
     List<byte[]> allMembersFromScan = new ArrayList<>();
-    String cursor = "0";
+    String cursor = ZERO_CURSOR;
 
     do {
-      result = jedis.sscan("a".getBytes(), cursor.getBytes(), scanParams);
+      result = jedis.sscan(KEY.getBytes(), cursor.getBytes(), scanParams);
       allMembersFromScan.addAll(result.getResult());
       cursor = result.getCursor();
     } while (!result.isCompleteIteration());
 
-    assertThat(allMembersFromScan).containsExactlyInAnyOrder("1".getBytes(),
-        "12".getBytes());
+    assertThat(allMembersFromScan).containsExactlyInAnyOrder(FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes());
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void givenMultipleCountsAndMatches_returnsAllEntriesWithoutDuplicates() {

Review comment:
       This test would be better named "givenSetWithThreeMembersAndMultipleMatchAndCountArguments_returnsAllMatchingMembers" with the assertions changed to `containsOnly()` to allow duplicate entries.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -323,26 +409,217 @@ public void givenNegativeCursor_returnsMembersUsingAbsoluteValueOfCursor() {
 
   @Test
   public void givenCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, UNSIGNED_LONG_CAPACITY.add(BigInteger.ONE).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenNegativeCursorGreaterThanUnsignedLongCapacity_returnsCursorError() {
-    assertThatThrownBy(() -> jedis.sscan("a", "-18446744073709551616"))
-        .hasMessageContaining(ERROR_CURSOR);
+    assertThatThrownBy(
+        () -> jedis.sscan(KEY, SIGNED_LONG_CAPACITY.add(BigInteger.valueOf(-1)).toString()))
+            .hasMessageContaining(ERROR_CURSOR);
   }
 
   @Test
   public void givenInvalidRegexSyntax_returnsEmptyArray() {
-    jedis.sadd("a", "1");
+    jedis.sadd(KEY, FIELD_ONE);
     ScanParams scanParams = new ScanParams();
     scanParams.count(1);
     scanParams.match("\\p");
 
     ScanResult<byte[]> result =
-        jedis.sscan("a".getBytes(), "0".getBytes(), scanParams);
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
 
     assertThat(result.getResult()).isEmpty();
   }
+
+  @Test
+  public void should_notReturnValue_givenValueWasRemovedBeforeSSCANISCalled() {
+    List<String> set = initializeThreeMemberSet();
+
+    jedis.srem(KEY, FIELD_THREE);
+    set.remove(FIELD_THREE);
+
+    GeodeAwaitility.await().untilAsserted(
+        () -> assertThat(jedis.sismember(KEY, FIELD_THREE)).isFalse());
+
+    ScanResult<String> result = jedis.sscan(KEY, ZERO_CURSOR);
+
+    assertThat(result.getResult()).containsExactlyInAnyOrderElementsOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenNonzeroCursorOnFirstCall() {
+    List<String> set = initializeThreeMemberSet();
+
+    ScanResult<String> result = jedis.sscan(KEY, "5");
+
+    assertThat(result.getResult()).isSubsetOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenCountEqualToIntegerMaxValue() {
+    List<byte[]> set = initializeThreeMemberByteSet();
+
+    ScanParams scanParams = new ScanParams().count(Integer.MAX_VALUE);
+
+    ScanResult<byte[]> result =
+        jedis.sscan(KEY.getBytes(), ZERO_CURSOR.getBytes(), scanParams);
+    assertThat(result.getResult())
+        .containsExactlyInAnyOrderElementsOf(set);
+  }
+
+  @Test
+  public void should_notErrorGivenCountGreaterThanIntegerMaxValue() {
+    initializeThreeMemberByteSet();
+
+    String greaterThanInt = String.valueOf(Integer.MAX_VALUE);
+    List<Object> result =
+        uncheckedCast(jedis.sendCommand(KEY.getBytes(), Protocol.Command.SSCAN,
+            KEY.getBytes(), ZERO_CURSOR.getBytes(),
+            "COUNT".getBytes(), greaterThanInt.getBytes()));
+
+    assertThat((byte[]) result.get(0)).isEqualTo(ZERO_CURSOR.getBytes());
+
+    List<byte[]> fields = uncheckedCast(result.get(1));
+    assertThat(fields).containsExactlyInAnyOrder(
+        FIELD_ONE.getBytes(),
+        FIELD_TWO.getBytes(),
+        FIELD_THREE.getBytes());
+  }
+
+  /**** Concurrency ***/
+
+  @Test
+  public void should_notLoseFields_givenConcurrentThreadsDoingSScansAndChangingValues() {
+    final List<String> initialMemberData = makeSet();
+    jedis.sadd(KEY, initialMemberData.toArray(new String[initialMemberData.size()]));
+    final int iterationCount = 500;
+
+    Jedis jedis1 = jedis.getConnectionFromSlot(SLOT_FOR_KEY);
+    Jedis jedis2 = jedis.getConnectionFromSlot(SLOT_FOR_KEY);
+
+    new ConcurrentLoopingThreads(iterationCount,
+        (i) -> multipleSScanAndAssertOnSizeOfResultSet(jedis1, initialMemberData),
+        (i) -> multipleSScanAndAssertOnSizeOfResultSet(jedis2, initialMemberData),
+        (i) -> {
+          int fieldSuffix = i % SIZE_OF_SET;
+          jedis.sadd(KEY, BASE_FIELD + fieldSuffix);
+        }).run();
+
+    jedis1.close();
+    jedis2.close();
+  }
+
+  @Test
+  public void should_notLoseKeysForConsistentlyPresentFields_givenConcurrentThreadsAddingAndRemovingFields() {
+    final List<String> initialMemberData = makeSet();
+    jedis.sadd(KEY, initialMemberData.toArray(new String[initialMemberData.size()]));
+    final int iterationCount = 500;
+
+    Jedis jedis1 = jedis.getConnectionFromSlot(SLOT_FOR_KEY);
+    Jedis jedis2 = jedis.getConnectionFromSlot(SLOT_FOR_KEY);
+
+    new ConcurrentLoopingThreads(iterationCount,
+        (i) -> multipleSScanAndAssertOnContentOfResultSet(i, jedis1, initialMemberData),
+        (i) -> multipleSScanAndAssertOnContentOfResultSet(i, jedis2, initialMemberData),
+        (i) -> {
+          String field = "new_" + BASE_FIELD + i;
+          jedis.sadd(KEY, field);
+          jedis.srem(KEY, field);
+        }).run();
+
+    jedis1.close();
+    jedis2.close();
+  }
+
+  @Test
+  public void should_notAlterUnderlyingData_givenMultipleConcurrentSscans() {
+    final List<String> initialMemberData = makeSet();
+    jedis.sadd(KEY, initialMemberData.toArray(new String[initialMemberData.size()]));
+    final int iterationCount = 500;
+
+    Jedis jedis1 = jedis.getConnectionFromSlot(SLOT_FOR_KEY);
+    Jedis jedis2 = jedis.getConnectionFromSlot(SLOT_FOR_KEY);
+
+    new ConcurrentLoopingThreads(iterationCount,
+        (i) -> multipleSScanAndAssertOnContentOfResultSet(i, jedis1, initialMemberData),
+        (i) -> multipleSScanAndAssertOnContentOfResultSet(i, jedis2, initialMemberData))
+            .run();
+
+    initialMemberData
+        .forEach((field) -> assertThat(jedis.sismember(KEY, field)).isTrue());

Review comment:
       `field` should be `member`.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSScanIntegrationTest.java
##########
@@ -14,33 +14,53 @@
  */
 package org.apache.geode.redis.internal.commands.executor.set;
 
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertAtLeastNArgs;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_CURSOR;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_SYNTAX;
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatThrownBy;
 
+import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.stream.Collectors;
 
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.Jedis;
 import redis.clients.jedis.JedisCluster;
 import redis.clients.jedis.Protocol;
 import redis.clients.jedis.ScanParams;
 import redis.clients.jedis.ScanResult;
 
+import org.apache.geode.redis.ConcurrentLoopingThreads;
 import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.redis.internal.commands.executor.cluster.CRC16;
+import org.apache.geode.redis.internal.services.RegionProvider;
 import org.apache.geode.test.awaitility.GeodeAwaitility;
 
 public abstract class AbstractSScanIntegrationTest implements RedisIntegrationTest {
   protected JedisCluster jedis;
   private static final int REDIS_CLIENT_TIMEOUT =
       Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  public static final String KEY = "key";
+  public static final int SLOT_FOR_KEY = CRC16.calculate(KEY) % RegionProvider.REDIS_SLOTS;
+  public static final String ZERO_CURSOR = "0";
+  public static final BigInteger UNSIGNED_LONG_CAPACITY = new BigInteger("18446744073709551615");
+  public static final BigInteger SIGNED_LONG_CAPACITY = new BigInteger("-18446744073709551615");
+
+  public static final String FIELD_ONE = "1";
+  public static final String FIELD_TWO = "12";
+  public static final String FIELD_THREE = "3";
+
+  public static final String BASE_FIELD = "baseField_";
+  private final int SIZE_OF_SET = 100;

Review comment:
       To ensure that native Redis isn't using its more compact data structure (which behaves differently for SSCAN) in tests using this constant, this size should be increased to 1000.




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