You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2021/09/01 00:43:39 UTC

[GitHub] [geode] DonalEvans commented on a change in pull request #6773: GEODE-9434: add ZREMRANGEBYLEX command

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



##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/server/AbstractHitsMissesIntegrationTest.java
##########
@@ -715,4 +720,17 @@ private void runCommandAndAssertNoStatUpdates(String key,
     assertThat(info.get(HITS)).isEqualTo(currentHits);
     assertThat(info.get(MISSES)).isEqualTo(currentMisses);
   }
+
+  private void runCommandAndAssertNoStatUpdates(String key, String min, String max,

Review comment:
       This method is not used and can be removed.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRemRangeByLexExecutor.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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 ZRemRangeByLexExecutor
+    extends AbstractSortedSetRangeExecutor<SortedSetLexRangeOptions> {
+  @Override
+  public boolean isRev() {
+    return false;
+  }
+
+  @Override
+  public SortedSetLexRangeOptions createRangeOptions(List<byte[]> commandElements) {
+    return new SortedSetLexRangeOptions(commandElements, isRev());
+  }
+
+  @Override
+  public RedisResponse getEmptyResponse() {
+    return RedisResponse.emptyArray();

Review comment:
       The empty response for this class should be `RedisResponse.integer(0);` since a no-op range remove returns 0 rather than an empty array.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSetCommandsFunctionExecutor.java
##########
@@ -100,6 +100,12 @@ public long zrem(RedisKey key, List<byte[]> membersToRemove) {
         () -> getRedisSortedSet(key, false).zrem(getRegion(), key, membersToRemove));
   }
 
+  @Override
+  public int zremrangebylex(RedisKey key, SortedSetLexRangeOptions rangeOptions) {

Review comment:
       To match the other methods in this class, this should return a `long`.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/NullRedisSortedSet.java
##########
@@ -107,6 +107,13 @@ long zlexcount(SortedSetLexRangeOptions rangeOptions) {
     return Collections.emptyList();
   }
 
+  @Override
+  int zremrangebylex(

Review comment:
       Minor nitpick, but it looks like spA did something weird to the formatting of this method.

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

Review comment:
       To match the other methods in this class, this should return a `long`.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -355,6 +355,26 @@ long zrem(Region<RedisKey, RedisData> region, RedisKey key, List<byte[]> members
     return membersRemoved;
   }
 
+  int zremrangebylex(Region<RedisKey, RedisData> region, RedisKey key,
+      SortedSetLexRangeOptions rangeOptions) {
+    if (scoreSet.isEmpty()) {
+      return 0;
+    }
+
+    int minIndex = rangeOptions.getRangeIndex(scoreSet, true);
+    if (minIndex >= scoreSet.size()) {
+      return 0;
+    }
+
+    int maxIndex = rangeOptions.getRangeIndex(scoreSet, false);
+
+    if (minIndex == maxIndex) {
+      return 0;
+    }
+
+    return removeEntriesByRange(region, key, minIndex, maxIndex);

Review comment:
       This should be able to be replaced with a call to `removeRange()`, I think. That method handles all of the same stuff as this method and `removeEntriesByRange()` so it's not necessary to reimplement them here.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRemRangeByLexExecutor.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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 ZRemRangeByLexExecutor
+    extends AbstractSortedSetRangeExecutor<SortedSetLexRangeOptions> {

Review comment:
       This class can extend `ZRangeByLexExecutor` instead, which means that it doesn't need to override the `isRev()` and `createRangeOptions()` methods.

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRemRangeByLexIntegrationTest.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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_MIN_MAX_NOT_A_VALID_STRING;
+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 java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.commons.lang3.StringUtils;
+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 AbstractZRemRangeByLexIntegrationTest 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.ZREMRANGEBYLEX, 3);
+  }
+
+  @Test
+  @Parameters({"a", "--", "++", "4"})
+  public void shouldError_givenInvalidMinOrMax(String invalidArgument) {
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", invalidArgument, "+"))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", "-", invalidArgument))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", invalidArgument, invalidArgument))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+  }
+
+  @Test
+  public void shouldReturnZero_givenNonExistentKey() {
+    jedis.zadd(KEY, SCORE, "member1");
+    assertThat(jedis.zremrangeByLex("fakeKey", "-", "+")).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMinGreaterThanMax() {
+    jedis.zadd(KEY, SCORE, "member");
+
+    // Range + <= member name <= -
+    assertThat(jedis.zrangeByLex(KEY, "+", "-")).isEmpty();
+    // Range z <= member name <= a
+    assertThat(jedis.zrangeByLex(KEY, "[z", "[a")).isEmpty();
+  }
+
+  @Test
+  public void shouldReturnOne_givenOneMemberAndMemberNameInRangeInclusiveMtoN() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range m <= member name <= n
+    assertThat(jedis.zremrangeByLex(KEY, "[m", "[n")).isEqualTo(1);

Review comment:
       Very minor thing, but `isEqualTo(1)` can be simplified to `isOne()` throughout this class.

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRemRangeByLexIntegrationTest.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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_MIN_MAX_NOT_A_VALID_STRING;
+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 java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.commons.lang3.StringUtils;
+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 AbstractZRemRangeByLexIntegrationTest 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.ZREMRANGEBYLEX, 3);
+  }
+
+  @Test
+  @Parameters({"a", "--", "++", "4"})
+  public void shouldError_givenInvalidMinOrMax(String invalidArgument) {
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", invalidArgument, "+"))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", "-", invalidArgument))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", invalidArgument, invalidArgument))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+  }
+
+  @Test
+  public void shouldReturnZero_givenNonExistentKey() {
+    jedis.zadd(KEY, SCORE, "member1");
+    assertThat(jedis.zremrangeByLex("fakeKey", "-", "+")).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMinGreaterThanMax() {
+    jedis.zadd(KEY, SCORE, "member");
+
+    // Range + <= member name <= -
+    assertThat(jedis.zrangeByLex(KEY, "+", "-")).isEmpty();
+    // Range z <= member name <= a
+    assertThat(jedis.zrangeByLex(KEY, "[z", "[a")).isEmpty();

Review comment:
       These should be `jedis.zremrangeByLex()` and should be asserting `isZero()`.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -228,6 +229,7 @@
   ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
       .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
   ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),

Review comment:
       Could this be added to the list between `ZREM` and `ZREMRANGEBYSCORE` instead, to preserve alphabetical ordering?

##########
File path: geode-apis-compatible-with-redis/src/distributedTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRemRangeByLexDUnitTest.java
##########
@@ -0,0 +1,280 @@
+/*
+ * 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 java.util.stream.Collectors;
+
+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 ZRemRangeByLexDUnitTest {
+  @Rule
+  public ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  @Rule
+  public RedisClusterStartupRule clusterStartUp = new RedisClusterStartupRule(4);
+
+  private JedisCluster jedis1;
+  private JedisCluster jedis2;
+  private List<MemberVM> servers;
+  private static final String sortedSetKey = "key";
+  private final int setSize = 500;
+
+  @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);
+    servers = new ArrayList<>();
+    servers.add(server1);
+    servers.add(server2);
+
+    int redisServerPort1 = clusterStartUp.getRedisPort(1);
+    int redisServerPort2 = clusterStartUp.getRedisPort(2);
+
+    jedis1 =
+        new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort1), REDIS_CLIENT_TIMEOUT);
+    jedis2 =
+        new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort2), REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    jedis1.close();

Review comment:
       `jedis2` should also be closed here. Alternatively, this test could use just one client, as every other DUnit test uses only one (besides `ZRemRangeByScoreDUnitTest` which I'm modifying as part of the GEODE-9435 PR). Then again, this might just be a huge oversight in our testing, so maybe leave it in? I think this is something the team needs to talk about.

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRemRangeByLexIntegrationTest.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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_MIN_MAX_NOT_A_VALID_STRING;
+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 java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.commons.lang3.StringUtils;
+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 AbstractZRemRangeByLexIntegrationTest 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.ZREMRANGEBYLEX, 3);
+  }
+
+  @Test
+  @Parameters({"a", "--", "++", "4"})
+  public void shouldError_givenInvalidMinOrMax(String invalidArgument) {
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", invalidArgument, "+"))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", "-", invalidArgument))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", invalidArgument, invalidArgument))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+  }
+
+  @Test
+  public void shouldReturnZero_givenNonExistentKey() {
+    jedis.zadd(KEY, SCORE, "member1");
+    assertThat(jedis.zremrangeByLex("fakeKey", "-", "+")).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMinGreaterThanMax() {
+    jedis.zadd(KEY, SCORE, "member");
+
+    // Range + <= member name <= -
+    assertThat(jedis.zrangeByLex(KEY, "+", "-")).isEmpty();
+    // Range z <= member name <= a
+    assertThat(jedis.zrangeByLex(KEY, "[z", "[a")).isEmpty();
+  }
+
+  @Test
+  public void shouldReturnOne_givenOneMemberAndMemberNameInRangeInclusiveMtoN() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range m <= member name <= n
+    assertThat(jedis.zremrangeByLex(KEY, "[m", "[n")).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnOne_givenOneMemberAndMemberNameInRangeMinusToN() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range -infinity <= member name <= n
+    assertThat(jedis.zremrangeByLex(KEY, "-", "[n")).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnOne_givenOneMemberAndMemberNameInRangeInclusiveMToPlus() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range m <= member name <= +infinity
+    assertThat(jedis.zremrangeByLex(KEY, "[m", "+")).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnOne_givenOneMemberInRange_MinEqualToMemberNameAndMinInclusive() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range member <= member name <= n
+    assertThat(jedis.zremrangeByLex(KEY, "[" + memberName, "[n")).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnOne_givenMaxEqualToMemberNameAndMaxInclusive() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range a <= member name <= member
+    assertThat(jedis.zremrangeByLex(KEY, "[a", "[" + memberName)).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnOne_givenMinAndMaxEqualToMemberNameAndMinAndMaxInclusive() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    assertThat(jedis.zremrangeByLex(KEY, "[" + memberName, "[" + memberName)).isEqualTo(1);
+  }
+
+  @Test
+  @Parameters({"[", "(", "", "-", "+"})
+  public void shouldReturnOne_givenMemberNameIsSpecialCharacterInRange(String memberName) {
+    jedis.zadd(KEY, SCORE, memberName);
+
+    assertThat(jedis.zremrangeByLex(KEY, "[" + memberName, "[" + memberName)).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMinEqualToMemberNameAndMinExclusive() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range member < member name <= n
+    assertThat(jedis.zremrangeByLex(KEY, "(" + memberName, "[n")).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMaxEqualToMemberNameAndMaxExclusive() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range a <= member name < member
+    assertThat(jedis.zremrangeByLex(KEY, "[a", "(" + memberName)).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMemberNotInRange() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range member name <= n <= o
+    assertThat(jedis.zremrangeByLex(KEY, "[n", "[o")).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnNumMembersRemoved_givenMultipleMembersInRange_withInclusiveMinAndMax() {
+    List<String> members = populateSortedSet();
+
+    int minLength = 3;
+    int maxLength = 6;
+    String min = StringUtils.repeat(BASE_MEMBER_NAME, minLength);
+    String max = StringUtils.repeat(BASE_MEMBER_NAME, maxLength);
+
+    List<String> sublist = new ArrayList<>(members.subList(minLength - 1, maxLength));
+    members.removeAll(sublist);
+
+    // Range (v * 3) <= member name <= (v * 6)
+    assertThat(jedis.zremrangeByLex(KEY, "[" + min, "[" + max)).isEqualTo(sublist.size());
+    assertThat(jedis.zrange(KEY, 0, -1)).hasSameElementsAs(members);
+  }
+
+  @Test
+  public void shouldReturnNumMembersRemoved_givenMultipleMembersInRange_withExclusiveMinAndMax() {
+    List<String> members = populateSortedSet();
+
+    int minLength = 1;
+    int maxLength = 7;
+    String min = StringUtils.repeat(BASE_MEMBER_NAME, minLength);
+    String max = StringUtils.repeat(BASE_MEMBER_NAME, maxLength);
+
+    List<String> sublist = new ArrayList<>(members.subList(minLength, maxLength - 1));
+    members.removeAll(sublist);
+
+    // Range (v * 1) < member name < (v * 7)
+    assertThat(jedis.zremrangeByLex(KEY, "(" + min, "(" + max)).isEqualTo(sublist.size());
+    assertThat(jedis.zrange(KEY, 0, -1)).hasSameElementsAs(members);
+  }
+
+  @Test
+  public void shouldReturnNumMembersRemoved_givenMultipleMembersInRange_withInclusiveMinAndExclusiveMax() {
+    List<String> members = populateSortedSet();
+
+    int minLength = 5;
+    int maxLength = 8;
+    String min = StringUtils.repeat(BASE_MEMBER_NAME, minLength);
+    String max = StringUtils.repeat(BASE_MEMBER_NAME, maxLength);
+
+    List<String> sublist = new ArrayList<>(members.subList(minLength - 1, maxLength - 1));
+    members.removeAll(sublist);
+
+    // Range (v * 5) <= member name < (v * 8)
+    assertThat(jedis.zremrangeByLex(KEY, "[" + min, "(" + max)).isEqualTo(sublist.size());
+    assertThat(jedis.zrange(KEY, 0, -1)).hasSameElementsAs(members);
+  }
+
+  @Test
+  public void shouldReturnNumMembersRemoved_givenMultipleMembersInRange_withExclusiveMinAndInclusiveMax() {
+    List<String> members = populateSortedSet();
+
+    int minLength = 2;
+    int maxLength = 5;
+    String min = StringUtils.repeat(BASE_MEMBER_NAME, minLength);
+    String max = StringUtils.repeat(BASE_MEMBER_NAME, maxLength);
+
+    List<String> sublist = new ArrayList<>(members.subList(minLength, maxLength));
+    members.removeAll(sublist);
+
+    // Range (v * 2) < member name <= (v * 5)
+    assertThat(jedis.zremrangeByLex(KEY, "(" + min, "[" + max)).isEqualTo(sublist.size());
+    assertThat(jedis.zrange(KEY, 0, -1)).hasSameElementsAs(members);
+  }
+
+  @Test
+  public void shouldReturnNumMembersRemoved_givenMinusAndInclusiveMaxRange() {
+    List<String> members = populateSortedSet();
+
+    int maxLength = 8;
+    String max = StringUtils.repeat(BASE_MEMBER_NAME, maxLength);
+
+    List<String> sublist = new ArrayList<>(members.subList(0, maxLength));
+    members.removeAll(sublist);
+
+    // Range -infinity <= member name <= (v * 8)
+    assertThat(jedis.zremrangeByLex(KEY, "-", "[" + max)).isEqualTo(sublist.size());
+    assertThat(jedis.zrange(KEY, 0, -1)).hasSameElementsAs(members);
+  }
+
+  @Test
+  public void shouldReturnNumMembersRemoved_givenInclusiveMinAndPlusRange() {
+    List<String> members = populateSortedSet();
+
+    int minLength = 4;
+    String min = StringUtils.repeat(BASE_MEMBER_NAME, minLength);
+
+    List<String> sublist = new ArrayList<>(members.subList(minLength - 1, members.size()));
+    members.removeAll(sublist);
+
+    // Range (v * 4) <= member name < +infinity
+    assertThat(jedis.zremrangeByLex(KEY, "[" + min, "+")).isEqualTo(sublist.size());
+    assertThat(jedis.zrange(KEY, 0, -1)).hasSameElementsAs(members);
+  }
+
+  @Test
+  public void shouldReturnNumMembersRemoved_givenMinusAndPlusRange() {
+    List<String> members = populateSortedSet();
+
+    // Range -infinity <= member name < +infinity
+    assertThat(jedis.zremrangeByLex(KEY, "-", "+")).isEqualTo(members.size());
+    assertThat(jedis.zcard(KEY)).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldRemoveMemberInRangeAndKey_givenOneMember() {
+    jedis.zadd(KEY, 1.0, "member");
+
+    assertThat(jedis.zremrangeByLex(KEY, "-", "+")).isEqualTo(1);
+    assertThat(jedis.exists(KEY)).isFalse();
+  }
+
+  @Test
+  public void shouldRemoveMembersInRange_givenMultipleMembers() {

Review comment:
       This behaviour is covered in the `shouldReturnNumMembersRemoved_givenMultipleMembersInRange_withInclusiveMinAndExclusiveMax()` test, so it shouldn't be necessary to repeat it here.

##########
File path: geode-apis-compatible-with-redis/src/distributedTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRemRangeByLexDUnitTest.java
##########
@@ -0,0 +1,280 @@
+/*
+ * 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 java.util.stream.Collectors;
+
+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 ZRemRangeByLexDUnitTest {
+  @Rule
+  public ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  @Rule
+  public RedisClusterStartupRule clusterStartUp = new RedisClusterStartupRule(4);
+
+  private JedisCluster jedis1;
+  private JedisCluster jedis2;
+  private List<MemberVM> servers;
+  private static final String sortedSetKey = "key";
+  private final int setSize = 500;
+
+  @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);
+    servers = new ArrayList<>();
+    servers.add(server1);
+    servers.add(server2);
+
+    int redisServerPort1 = clusterStartUp.getRedisPort(1);
+    int redisServerPort2 = clusterStartUp.getRedisPort(2);
+
+    jedis1 =
+        new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort1), REDIS_CLIENT_TIMEOUT);
+    jedis2 =
+        new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort2), REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    jedis1.close();
+  }
+
+  @Test
+  public void ZRemRangeByLex_removesMembersInRangeFromBothServers() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis1.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    List<String> expected = new ArrayList<>(memberScoreMap.keySet().stream().sorted()
+        .collect(Collectors.toList())).subList(4, setSize);
+
+    assertThat(jedis1.zrank(sortedSetKey, "vvvvv")).isEqualTo(4L);
+    assertThat(jedis2.zrank(sortedSetKey, "vvvvv")).isEqualTo(4L);
+
+    assertThat(jedis1.zremrangeByLex(sortedSetKey, "[" + "v", "[" + "vvvv")).isEqualTo(4);
+
+    assertThat(jedis1.zrank(sortedSetKey, "vvvvv")).isEqualTo(0L);
+    assertThat(jedis2.zrank(sortedSetKey, "vvvvv")).isEqualTo(0L);
+
+    assertThat(jedis1.zrange(sortedSetKey, 0, -1)).hasSameElementsAs(expected);
+    assertThat(jedis2.zrange(sortedSetKey, 0, -1)).hasSameElementsAs(expected);
+  }
+
+  @Test
+  public void ZRemRangeByLex_concurrentlyRemovesMembersInRangeFromBothServers() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis1.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    assertThat(jedis1.zrank(sortedSetKey, "vvvvv")).isEqualTo(4L);
+    assertThat(jedis2.zrank(sortedSetKey, "vvvvv")).isEqualTo(4L);
+
+    AtomicInteger totalRemoved = new AtomicInteger();
+    new ConcurrentLoopingThreads(setSize,
+        (i) -> doZRemRangeByLexOnMembers(i, totalRemoved),
+        (i) -> doZRemRangeByLexOnMembersInDifferentOrder(i, totalRemoved)).run();
+
+    assertThat(jedis1.exists(sortedSetKey)).isFalse();
+    assertThat(jedis2.exists(sortedSetKey)).isFalse();
+
+    assertThat(totalRemoved.get()).isEqualTo(setSize);
+  }
+
+  @Test
+  public void zRemRangeByLexRemovesMembersFromSortedSetAfterPrimaryShutsDown() {
+    Map<String, Double> memberScoreMap = makeMemberScoreMap(setSize);
+    jedis1.zadd(sortedSetKey, memberScoreMap);
+    verifyDataExists(memberScoreMap);
+
+    stopNodeWithPrimaryBucketOfTheKey(false);
+
+    doZRemRangeByLexWithRetries(memberScoreMap);
+
+    verifyDataDoesNotExist(memberScoreMap);
+    assertThat(jedis1.exists(sortedSetKey)).isFalse();
+    assertThat(jedis2.exists(sortedSetKey)).isFalse();
+  }
+
+  @Test
+  @Ignore("Fails due to GEODE-9310")

Review comment:
       This should be safe to un-ignore if a third server is added to the test. A working implementation of what this test is trying to do can be found in `ZRemRangeByScoreDUnitTest`.

##########
File path: geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/data/RedisSortedSetTest.java
##########
@@ -202,6 +202,30 @@ public void zremCanRemoveMembersToBeRemoved() {
     verify(sortedSet).storeChanges(eq(region), eq(key), any(RemsDeltaInfo.class));
   }
 
+  @Test
+  public void zremrangebylex_removesMembersInRange() {

Review comment:
       I'm not certain that this test is needed, since the behaviour should be covered extensively in the integration test.

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRemRangeByLexIntegrationTest.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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_MIN_MAX_NOT_A_VALID_STRING;
+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 java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.commons.lang3.StringUtils;
+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 AbstractZRemRangeByLexIntegrationTest 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.ZREMRANGEBYLEX, 3);
+  }
+
+  @Test
+  @Parameters({"a", "--", "++", "4"})
+  public void shouldError_givenInvalidMinOrMax(String invalidArgument) {
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", invalidArgument, "+"))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", "-", invalidArgument))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", invalidArgument, invalidArgument))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+  }
+
+  @Test
+  public void shouldReturnZero_givenNonExistentKey() {
+    jedis.zadd(KEY, SCORE, "member1");
+    assertThat(jedis.zremrangeByLex("fakeKey", "-", "+")).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMinGreaterThanMax() {
+    jedis.zadd(KEY, SCORE, "member");
+
+    // Range + <= member name <= -
+    assertThat(jedis.zrangeByLex(KEY, "+", "-")).isEmpty();
+    // Range z <= member name <= a
+    assertThat(jedis.zrangeByLex(KEY, "[z", "[a")).isEmpty();
+  }
+
+  @Test
+  public void shouldReturnOne_givenOneMemberAndMemberNameInRangeInclusiveMtoN() {

Review comment:
       In addition to asserting that the returned value is correct, could these tests also assert that the member is removed (or not removed if it shouldn't be) using a zrange or zscore call?

##########
File path: geode-apis-compatible-with-redis/src/distributedTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRemRangeByLexDUnitTest.java
##########
@@ -0,0 +1,280 @@
+/*
+ * 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 java.util.stream.Collectors;
+
+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 ZRemRangeByLexDUnitTest {
+  @Rule
+  public ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  @Rule
+  public RedisClusterStartupRule clusterStartUp = new RedisClusterStartupRule(4);
+
+  private JedisCluster jedis1;
+  private JedisCluster jedis2;
+  private List<MemberVM> servers;
+  private static final String sortedSetKey = "key";
+  private final int setSize = 500;
+
+  @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);

Review comment:
       To prevent test failures due to loss of quorum when a server crashes, three servers are required. Adding an extra one should allow the currently ignored test to pass (possibly with a little tweaking).

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRemRangeByLexIntegrationTest.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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_MIN_MAX_NOT_A_VALID_STRING;
+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 java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.commons.lang3.StringUtils;
+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 AbstractZRemRangeByLexIntegrationTest 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.ZREMRANGEBYLEX, 3);
+  }
+
+  @Test
+  @Parameters({"a", "--", "++", "4"})
+  public void shouldError_givenInvalidMinOrMax(String invalidArgument) {
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", invalidArgument, "+"))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", "-", invalidArgument))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+    assertThatThrownBy(() -> jedis.zremrangeByLex("fakeKey", invalidArgument, invalidArgument))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_VALID_STRING);
+  }
+
+  @Test
+  public void shouldReturnZero_givenNonExistentKey() {
+    jedis.zadd(KEY, SCORE, "member1");
+    assertThat(jedis.zremrangeByLex("fakeKey", "-", "+")).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMinGreaterThanMax() {
+    jedis.zadd(KEY, SCORE, "member");
+
+    // Range + <= member name <= -
+    assertThat(jedis.zrangeByLex(KEY, "+", "-")).isEmpty();
+    // Range z <= member name <= a
+    assertThat(jedis.zrangeByLex(KEY, "[z", "[a")).isEmpty();
+  }
+
+  @Test
+  public void shouldReturnOne_givenOneMemberAndMemberNameInRangeInclusiveMtoN() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range m <= member name <= n
+    assertThat(jedis.zremrangeByLex(KEY, "[m", "[n")).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnOne_givenOneMemberAndMemberNameInRangeMinusToN() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range -infinity <= member name <= n
+    assertThat(jedis.zremrangeByLex(KEY, "-", "[n")).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnOne_givenOneMemberAndMemberNameInRangeInclusiveMToPlus() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range m <= member name <= +infinity
+    assertThat(jedis.zremrangeByLex(KEY, "[m", "+")).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnOne_givenOneMemberInRange_MinEqualToMemberNameAndMinInclusive() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range member <= member name <= n
+    assertThat(jedis.zremrangeByLex(KEY, "[" + memberName, "[n")).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnOne_givenMaxEqualToMemberNameAndMaxInclusive() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range a <= member name <= member
+    assertThat(jedis.zremrangeByLex(KEY, "[a", "[" + memberName)).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnOne_givenMinAndMaxEqualToMemberNameAndMinAndMaxInclusive() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    assertThat(jedis.zremrangeByLex(KEY, "[" + memberName, "[" + memberName)).isEqualTo(1);
+  }
+
+  @Test
+  @Parameters({"[", "(", "", "-", "+"})
+  public void shouldReturnOne_givenMemberNameIsSpecialCharacterInRange(String memberName) {
+    jedis.zadd(KEY, SCORE, memberName);
+
+    assertThat(jedis.zremrangeByLex(KEY, "[" + memberName, "[" + memberName)).isEqualTo(1);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMinEqualToMemberNameAndMinExclusive() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range member < member name <= n
+    assertThat(jedis.zremrangeByLex(KEY, "(" + memberName, "[n")).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMaxEqualToMemberNameAndMaxExclusive() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range a <= member name < member
+    assertThat(jedis.zremrangeByLex(KEY, "[a", "(" + memberName)).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnZero_givenMemberNotInRange() {
+    String memberName = "member";
+    jedis.zadd(KEY, SCORE, memberName);
+
+    // Range member name <= n <= o
+    assertThat(jedis.zremrangeByLex(KEY, "[n", "[o")).isEqualTo(0);
+  }
+
+  @Test
+  public void shouldReturnNumMembersRemoved_givenMultipleMembersInRange_withInclusiveMinAndMax() {
+    List<String> members = populateSortedSet();
+
+    int minLength = 3;
+    int maxLength = 6;
+    String min = StringUtils.repeat(BASE_MEMBER_NAME, minLength);
+    String max = StringUtils.repeat(BASE_MEMBER_NAME, maxLength);
+
+    List<String> sublist = new ArrayList<>(members.subList(minLength - 1, maxLength));

Review comment:
       Could `sublist` be renamed "membersToRemove" or something along those lines, to make it clearer what it is?




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