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



##########
File path: 
geode-apis-compatible-with-redis/src/acceptanceTest/resources/0001-configure-redis-tests.patch
##########
@@ -1665,39 +1665,108 @@ index a8c817f6e..74f06fd4f 100644
 -            assert_error "*not*float*" {r zremrangebyscore fooz 1 str}
 -            assert_error "*not*float*" {r zremrangebyscore fooz 1 NaN}
 -        }
--
--        test "ZREMRANGEBYRANK basics" {
--            proc remrangebyrank {min max} {
--                create_zset zset {1 a 2 b 3 c 4 d 5 e}
--                assert_equal 1 [r exists zset]
--                r zremrangebyrank zset $min $max
--            }
--
--            # inner range
--            assert_equal 3 [remrangebyrank 1 3]
--            assert_equal {a e} [r zrange zset 0 -1]
--
--            # start underflow
--            assert_equal 1 [remrangebyrank -10 0]
--            assert_equal {b c d e} [r zrange zset 0 -1]
--
--            # start overflow
--            assert_equal 0 [remrangebyrank 10 -1]
--            assert_equal {a b c d e} [r zrange zset 0 -1]
--
--            # end underflow
--            assert_equal 0 [remrangebyrank 0 -10]
--            assert_equal {a b c d e} [r zrange zset 0 -1]
--
--            # end overflow
--            assert_equal 5 [remrangebyrank 0 10]
--            assert_equal {} [r zrange zset 0 -1]
--
--            # destroy when empty
--            assert_equal 5 [remrangebyrank 0 4]
--            assert_equal 0 [r exists zset]
--        }
--
++        #     assert_equal 9 [r zlexcount zset - +]

Review comment:
       I may just be reading this wrong (a diff of a diff is kind of confusing 
to look at) but are these tests commented out? They seem to be testing commands 
we now support, so could they be un-commented?

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRemRangeByRankExecutor.java
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.executor.sortedset;
+
+import java.util.List;
+
+import org.apache.geode.redis.internal.data.RedisKey;
+import org.apache.geode.redis.internal.executor.RedisResponse;
+
+public class ZRemRangeByRankExecutor
+    extends AbstractSortedSetRangeExecutor<SortedSetRankRangeOptions> {

Review comment:
       This class can extend `ZRangeExcecutor` instead, which removes the need 
to override the `isRev()` and `createRangeOptions()` methods. It might be good 
to rename the `ZRangeExcecutor` and `ZRevRangeExcecutor` classes to 
`ZRangeByRankExcecutor` and `ZRevRangeByRankExcecutor` for consistency with 
other class names if you do that though.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -355,6 +355,11 @@ long zrem(Region<RedisKey, RedisData> region, RedisKey 
key, List<byte[]> members
     return membersRemoved;
   }
 
+  long zremrangebyrank(Region<RedisKey, RedisData> region, RedisKey key,

Review comment:
       An overridden implementation of this should probably also be added to 
`NullRedisSortedSet` to allow faster return in the event that the key doesn't 
exist.

##########
File path: 
geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRemRangeByRankIntegrationTest.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.executor.sortedset;
+
+import static 
org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.REDIS_CLIENT_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.Protocol;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+
+@RunWith(JUnitParamsRunner.class)
+public abstract class AbstractZRemRangeByRankIntegrationTest implements 
RedisIntegrationTest {
+  public static final String KEY = "key";
+  public static final int SCORE = 1;
+  public static final String BASE_MEMBER_NAME = "v";

Review comment:
       If this value is changed to "member" then many of the tests don't need 
to define their own `member` variable and can instead use this constant.

##########
File path: 
geode-apis-compatible-with-redis/src/distributedTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRemRangeByRankDUnitTest.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.executor.sortedset;
+
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.REDIS_CLIENT_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.exceptions.JedisClusterMaxAttemptsException;
+
+import org.apache.geode.cache.Operation;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.PartitionedRegionHelper;
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.redis.internal.RegionProvider;
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisKey;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+import org.apache.geode.test.junit.rules.ExecutorServiceRule;
+
+public class ZRemRangeByRankDUnitTest {
+  @Rule
+  public ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  @Rule
+  public RedisClusterStartupRule clusterStartUp = new 
RedisClusterStartupRule(4);
+
+  private JedisCluster jedis;
+  private List<MemberVM> servers;
+  private static final String sortedSetKey = "key";
+  private final String baseName = "member1-";
+  private final int setSize = 1000;
+
+  @Before
+  public void setup() {
+    MemberVM locator = clusterStartUp.startLocatorVM(0);
+    int locatorPort = locator.getPort();
+    MemberVM server1 = clusterStartUp.startRedisVM(1, locatorPort);
+    MemberVM server2 = clusterStartUp.startRedisVM(2, locatorPort);
+    MemberVM server3 = clusterStartUp.startRedisVM(3, locatorPort);
+    servers = new ArrayList<>();
+    servers.add(server1);
+    servers.add(server2);
+    servers.add(server3);
+
+    int redisServerPort = clusterStartUp.getRedisPort(1);
+
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort), 
REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    jedis.close();
+  }
+
+  @Test
+  public void zRemRangeByRankCanRemoveMembersFromSortedSet() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    long removed = jedis.zremrangeByRank(sortedSetKey, 0, -1);
+    assertThat(removed).isEqualTo(setSize);
+
+    verifyDataDoesNotExist(memberScoreMap);
+    assertThat(jedis.exists(sortedSetKey)).isFalse();
+  }
+
+  @Test
+  public void zRemRangeByRankCanRemoveMembersConcurrentlyFromSortedSet() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    AtomicInteger totalRemoved = new AtomicInteger();
+    new ConcurrentLoopingThreads(setSize,
+        (i) -> doZRemRangeByRankOnMembers(i, totalRemoved),
+        (i) -> doZRemRangeByRankOnMembersInDifferentOrder(i, 
totalRemoved)).run();
+
+    assertThat(totalRemoved.get()).isEqualTo(setSize);
+    assertThat(jedis.exists(sortedSetKey)).isFalse();
+  }
+
+  private void doZRemRangeByRankOnMembers(int i, AtomicInteger total) {
+    long count = jedis.zremrangeByRank(sortedSetKey, i, i);
+    total.addAndGet((int) count);
+  }
+
+  private void doZRemRangeByRankOnMembersInDifferentOrder(int i, AtomicInteger 
total) {
+    long count = jedis.zremrangeByRank(sortedSetKey, setSize - i - 1, setSize 
- i - 1);
+    total.addAndGet((int) count);
+  }
+
+  @Test
+  public void 
zRemRangeByRankRemovesMembersFromSortedSetAfterPrimaryShutsDown() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    stopNodeWithPrimaryBucketOfTheKey(false);
+
+    doZRemRangeByRankWithRetries(memberScoreMap);
+
+    verifyDataDoesNotExist(memberScoreMap);
+    assertThat(jedis.exists(sortedSetKey)).isFalse();
+  }
+
+  private void doZRemRangeByRankWithRetries(Map<String, Double> map) {
+    int maxRetryAttempts = 10;
+    int retryAttempts = 0;
+    while (!zRemRangeByRankWithRetries(map, retryAttempts, maxRetryAttempts)) {
+      retryAttempts++;
+    }
+  }
+
+  private boolean zRemRangeByRankWithRetries(Map<String, Double> map, int 
retries, int maxRetries) {
+    long removed;
+    try {
+      removed = jedis.zremrangeByRank(sortedSetKey, 0, -1);
+    } catch (JedisClusterMaxAttemptsException e) {
+      if (retries < maxRetries) {
+        return false;
+      }
+      throw e;
+    }
+    assertThat(removed).isEqualTo(map.size());
+    return true;
+  }
+
+  private void doZRemRangeByRank(Map<String, Double> map) {
+    long removed = jedis.zremrangeByRank(sortedSetKey, 0, -1);
+    assertThat(removed).isEqualTo(map.size());
+  }
+
+  @Test
+  @Ignore("Fails due to GEODE-9310")
+  public void 
zRemRangeByRankCanRemoveMembersFromSortedSetDuringPrimaryIsCrashed()
+      throws Exception {
+    int mapSize = 300;
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(mapSize);
+
+    jedis.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    int number = 10;
+    String memberNotRemoved = baseName + number;
+    memberScoreMap.remove(memberNotRemoved);
+
+    Future<Void> future1 = executor.submit(() -> 
doZRemRangeByRank(memberScoreMap));
+    Future<Void> future2 = executor.submit(() -> 
stopNodeWithPrimaryBucketOfTheKey(true));

Review comment:
       The timing of these two calls is non-deterministic, so it's possible 
that on some runs, the range will be removed before the server crashes and on 
others, the server will crash before the remove command goes through. Something 
should be done to ensure the desired timing is hit every time the test is run.

##########
File path: 
geode-apis-compatible-with-redis/src/distributedTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRemRangeByRankDUnitTest.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.executor.sortedset;
+
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.REDIS_CLIENT_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.exceptions.JedisClusterMaxAttemptsException;
+
+import org.apache.geode.cache.Operation;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.PartitionedRegionHelper;
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.redis.internal.RegionProvider;
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisKey;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+import org.apache.geode.test.junit.rules.ExecutorServiceRule;
+
+public class ZRemRangeByRankDUnitTest {
+  @Rule
+  public ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  @Rule
+  public RedisClusterStartupRule clusterStartUp = new 
RedisClusterStartupRule(4);
+
+  private JedisCluster jedis;
+  private List<MemberVM> servers;
+  private static final String sortedSetKey = "key";
+  private final String baseName = "member1-";
+  private final int setSize = 1000;
+
+  @Before
+  public void setup() {
+    MemberVM locator = clusterStartUp.startLocatorVM(0);
+    int locatorPort = locator.getPort();
+    MemberVM server1 = clusterStartUp.startRedisVM(1, locatorPort);
+    MemberVM server2 = clusterStartUp.startRedisVM(2, locatorPort);
+    MemberVM server3 = clusterStartUp.startRedisVM(3, locatorPort);
+    servers = new ArrayList<>();
+    servers.add(server1);
+    servers.add(server2);
+    servers.add(server3);
+
+    int redisServerPort = clusterStartUp.getRedisPort(1);
+
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort), 
REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    jedis.close();
+  }
+
+  @Test
+  public void zRemRangeByRankCanRemoveMembersFromSortedSet() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    long removed = jedis.zremrangeByRank(sortedSetKey, 0, -1);
+    assertThat(removed).isEqualTo(setSize);
+
+    verifyDataDoesNotExist(memberScoreMap);
+    assertThat(jedis.exists(sortedSetKey)).isFalse();
+  }
+
+  @Test
+  public void zRemRangeByRankCanRemoveMembersConcurrentlyFromSortedSet() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    AtomicInteger totalRemoved = new AtomicInteger();
+    new ConcurrentLoopingThreads(setSize,
+        (i) -> doZRemRangeByRankOnMembers(i, totalRemoved),
+        (i) -> doZRemRangeByRankOnMembersInDifferentOrder(i, 
totalRemoved)).run();
+
+    assertThat(totalRemoved.get()).isEqualTo(setSize);
+    assertThat(jedis.exists(sortedSetKey)).isFalse();
+  }
+
+  private void doZRemRangeByRankOnMembers(int i, AtomicInteger total) {
+    long count = jedis.zremrangeByRank(sortedSetKey, i, i);
+    total.addAndGet((int) count);
+  }
+
+  private void doZRemRangeByRankOnMembersInDifferentOrder(int i, AtomicInteger 
total) {
+    long count = jedis.zremrangeByRank(sortedSetKey, setSize - i - 1, setSize 
- i - 1);
+    total.addAndGet((int) count);
+  }
+
+  @Test
+  public void 
zRemRangeByRankRemovesMembersFromSortedSetAfterPrimaryShutsDown() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    stopNodeWithPrimaryBucketOfTheKey(false);
+
+    doZRemRangeByRankWithRetries(memberScoreMap);
+
+    verifyDataDoesNotExist(memberScoreMap);
+    assertThat(jedis.exists(sortedSetKey)).isFalse();
+  }
+
+  private void doZRemRangeByRankWithRetries(Map<String, Double> map) {
+    int maxRetryAttempts = 10;
+    int retryAttempts = 0;
+    while (!zRemRangeByRankWithRetries(map, retryAttempts, maxRetryAttempts)) {
+      retryAttempts++;
+    }
+  }
+
+  private boolean zRemRangeByRankWithRetries(Map<String, Double> map, int 
retries, int maxRetries) {
+    long removed;
+    try {
+      removed = jedis.zremrangeByRank(sortedSetKey, 0, -1);
+    } catch (JedisClusterMaxAttemptsException e) {
+      if (retries < maxRetries) {
+        return false;
+      }
+      throw e;
+    }
+    assertThat(removed).isEqualTo(map.size());
+    return true;
+  }
+
+  private void doZRemRangeByRank(Map<String, Double> map) {
+    long removed = jedis.zremrangeByRank(sortedSetKey, 0, -1);
+    assertThat(removed).isEqualTo(map.size());
+  }
+
+  @Test
+  @Ignore("Fails due to GEODE-9310")

Review comment:
       GEODE-9310 is marked as resolved, so this probably shouldn't be ignored.

##########
File path: 
geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRemRangeByRankIntegrationTest.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.executor.sortedset;
+
+import static 
org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.REDIS_CLIENT_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.Protocol;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+
+@RunWith(JUnitParamsRunner.class)
+public abstract class AbstractZRemRangeByRankIntegrationTest implements 
RedisIntegrationTest {
+  public static final String KEY = "key";
+  public static final int SCORE = 1;
+  public static final String BASE_MEMBER_NAME = "v";
+
+  JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, getPort()), 
REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    flushAll();
+    jedis.close();
+  }
+
+  @Test
+  public void shouldError_givenWrongNumberOfArguments() {
+    assertExactNumberOfArgs(jedis, Protocol.Command.ZREMRANGEBYRANK, 3);
+  }
+
+  @Test
+  @Parameters({"a", "--", "++", "4="})
+  public void shouldError_givenInvalidMinOrMax(String invalidArgument) {
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.ZREMRANGEBYRANK, KEY, 
"1", invalidArgument))
+            .hasMessageContaining(ERROR_NOT_INTEGER);
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.ZREMRANGEBYRANK, KEY, 
invalidArgument, "5"))
+            .hasMessageContaining(ERROR_NOT_INTEGER);
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, 
Protocol.Command.ZREMRANGEBYRANK, KEY,
+        invalidArgument, invalidArgument))
+            .hasMessageContaining(ERROR_NOT_INTEGER);
+  }
+
+  @Test
+  public void shouldReturnZero_givenNonExistentKey() {
+    jedis.zadd(KEY, SCORE, "member1");
+    assertThat(jedis.zremrangeByRank("fakeKey", 0, 1)).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMinGreaterThanMax() {
+    jedis.zadd(KEY, SCORE, "member");
+
+    assertThat(jedis.zremrangeByRank(KEY, 1, 0)).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnMember_givenMemberRankInRange() {

Review comment:
       This test name should probably be "shouldReturnOne_...". Also, could 
either an additional test case be added for removing one member and confirming 
it's been removed (via ZSCORE) or could some assertions be added to this test 
to confirm that the members have actually been removed.

##########
File path: 
geode-apis-compatible-with-redis/src/distributedTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRemRangeByRankDUnitTest.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.executor.sortedset;
+
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.REDIS_CLIENT_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.exceptions.JedisClusterMaxAttemptsException;
+
+import org.apache.geode.cache.Operation;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.PartitionedRegionHelper;
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.redis.internal.RegionProvider;
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisKey;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+import org.apache.geode.test.junit.rules.ExecutorServiceRule;
+
+public class ZRemRangeByRankDUnitTest {
+  @Rule
+  public ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  @Rule
+  public RedisClusterStartupRule clusterStartUp = new 
RedisClusterStartupRule(4);
+
+  private JedisCluster jedis;
+  private List<MemberVM> servers;
+  private static final String sortedSetKey = "key";
+  private final String baseName = "member1-";
+  private final int setSize = 1000;
+
+  @Before
+  public void setup() {
+    MemberVM locator = clusterStartUp.startLocatorVM(0);
+    int locatorPort = locator.getPort();
+    MemberVM server1 = clusterStartUp.startRedisVM(1, locatorPort);
+    MemberVM server2 = clusterStartUp.startRedisVM(2, locatorPort);
+    MemberVM server3 = clusterStartUp.startRedisVM(3, locatorPort);
+    servers = new ArrayList<>();
+    servers.add(server1);
+    servers.add(server2);
+    servers.add(server3);
+
+    int redisServerPort = clusterStartUp.getRedisPort(1);
+
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort), 
REDIS_CLIENT_TIMEOUT);

Review comment:
       Could this test have a second client connected to a different server, to 
confirm that both clients agree on returned values (i.e. that everything is 
distributed correctly)? Or would that not add any real value to the test?

##########
File path: 
geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRemRangeByRankIntegrationTest.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.executor.sortedset;
+
+import static 
org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.REDIS_CLIENT_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.Protocol;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+
+@RunWith(JUnitParamsRunner.class)
+public abstract class AbstractZRemRangeByRankIntegrationTest implements 
RedisIntegrationTest {
+  public static final String KEY = "key";
+  public static final int SCORE = 1;
+  public static final String BASE_MEMBER_NAME = "v";
+
+  JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, getPort()), 
REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    flushAll();
+    jedis.close();
+  }
+
+  @Test
+  public void shouldError_givenWrongNumberOfArguments() {
+    assertExactNumberOfArgs(jedis, Protocol.Command.ZREMRANGEBYRANK, 3);
+  }
+
+  @Test
+  @Parameters({"a", "--", "++", "4="})
+  public void shouldError_givenInvalidMinOrMax(String invalidArgument) {
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.ZREMRANGEBYRANK, KEY, 
"1", invalidArgument))
+            .hasMessageContaining(ERROR_NOT_INTEGER);
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.ZREMRANGEBYRANK, KEY, 
invalidArgument, "5"))
+            .hasMessageContaining(ERROR_NOT_INTEGER);
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, 
Protocol.Command.ZREMRANGEBYRANK, KEY,
+        invalidArgument, invalidArgument))
+            .hasMessageContaining(ERROR_NOT_INTEGER);
+  }
+
+  @Test
+  public void shouldReturnZero_givenNonExistentKey() {
+    jedis.zadd(KEY, SCORE, "member1");
+    assertThat(jedis.zremrangeByRank("fakeKey", 0, 1)).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMinGreaterThanMax() {
+    jedis.zadd(KEY, SCORE, "member");
+
+    assertThat(jedis.zremrangeByRank(KEY, 1, 0)).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnMember_givenMemberRankInRange() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName + "0");
+    jedis.zadd(KEY, SCORE + 1, memberName + "1");
+    jedis.zadd(KEY, SCORE + 2, memberName + "2");
+
+    assertThat(jedis.zremrangeByRank(KEY, 2, 2)).isEqualTo(1);
+    assertThat(jedis.zremrangeByRank(KEY, 1, 1)).isEqualTo(1);
+    assertThat(jedis.zremrangeByRank(KEY, 0, 0)).isEqualTo(1);
+    assertThat(jedis.zcard(KEY)).isEqualTo(0);
+  }
+
+
+  @Test
+  public void shouldReturnZero_givenRangeExcludingMember() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    assertThat(jedis.zremrangeByRank(KEY, 1, 2)).isEqualTo(0);
+    assertThat(jedis.zcard(KEY)).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldRemoveMembers_givenMultipleMembersInRange() {
+    populateSortedSet();
+
+    assertThat(jedis.zcard(KEY)).isEqualTo(10);

Review comment:
       Could some assertions be added here to confirm that the expected members 
have been removed between each call to `zremrangeByRank()`?

##########
File path: 
geode-apis-compatible-with-redis/src/distributedTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRemRangeByRankDUnitTest.java
##########
@@ -0,0 +1,249 @@
+/*
+ * 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.executor.sortedset;
+
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.REDIS_CLIENT_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.exceptions.JedisClusterMaxAttemptsException;
+
+import org.apache.geode.cache.Operation;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.cache.PartitionedRegion;
+import org.apache.geode.internal.cache.PartitionedRegionHelper;
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.redis.internal.RegionProvider;
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisKey;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+import org.apache.geode.test.junit.rules.ExecutorServiceRule;
+
+public class ZRemRangeByRankDUnitTest {
+  @Rule
+  public ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  @Rule
+  public RedisClusterStartupRule clusterStartUp = new 
RedisClusterStartupRule(4);
+
+  private JedisCluster jedis;
+  private List<MemberVM> servers;
+  private static final String sortedSetKey = "key";
+  private final String baseName = "member1-";
+  private final int setSize = 1000;
+
+  @Before
+  public void setup() {
+    MemberVM locator = clusterStartUp.startLocatorVM(0);
+    int locatorPort = locator.getPort();
+    MemberVM server1 = clusterStartUp.startRedisVM(1, locatorPort);
+    MemberVM server2 = clusterStartUp.startRedisVM(2, locatorPort);
+    MemberVM server3 = clusterStartUp.startRedisVM(3, locatorPort);
+    servers = new ArrayList<>();
+    servers.add(server1);
+    servers.add(server2);
+    servers.add(server3);
+
+    int redisServerPort = clusterStartUp.getRedisPort(1);
+
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort), 
REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    jedis.close();
+  }
+
+  @Test
+  public void zRemRangeByRankCanRemoveMembersFromSortedSet() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    long removed = jedis.zremrangeByRank(sortedSetKey, 0, -1);
+    assertThat(removed).isEqualTo(setSize);
+
+    verifyDataDoesNotExist(memberScoreMap);
+    assertThat(jedis.exists(sortedSetKey)).isFalse();
+  }
+
+  @Test
+  public void zRemRangeByRankCanRemoveMembersConcurrentlyFromSortedSet() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    AtomicInteger totalRemoved = new AtomicInteger();
+    new ConcurrentLoopingThreads(setSize,
+        (i) -> doZRemRangeByRankOnMembers(i, totalRemoved),
+        (i) -> doZRemRangeByRankOnMembersInDifferentOrder(i, 
totalRemoved)).run();

Review comment:
       Would it be possible to have these removes remove a range rather than a 
single element, to better test the command? Also, while it might be tricky to 
implement, it would be great if the ranges could overlap somewhat, to really 
test that we don't get problems with trying to remove the same member twice or 
something.

##########
File path: 
geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRemRangeByRankIntegrationTest.java
##########
@@ -0,0 +1,190 @@
+/*
+ * 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.executor.sortedset;
+
+import static 
org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static 
org.apache.geode.test.dunit.rules.RedisClusterStartupRule.REDIS_CLIENT_TIMEOUT;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.Protocol;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+
+@RunWith(JUnitParamsRunner.class)
+public abstract class AbstractZRemRangeByRankIntegrationTest implements 
RedisIntegrationTest {
+  public static final String KEY = "key";
+  public static final int SCORE = 1;
+  public static final String BASE_MEMBER_NAME = "v";
+
+  JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, getPort()), 
REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    flushAll();
+    jedis.close();
+  }
+
+  @Test
+  public void shouldError_givenWrongNumberOfArguments() {
+    assertExactNumberOfArgs(jedis, Protocol.Command.ZREMRANGEBYRANK, 3);
+  }
+
+  @Test
+  @Parameters({"a", "--", "++", "4="})
+  public void shouldError_givenInvalidMinOrMax(String invalidArgument) {

Review comment:
       Small point, but the arguments for ZREMRANGEBYRANK are named "start" and 
"stop", not "min" and "max", so this and other tests should probably be renamed 
to reflect this.




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to