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



##########
File path: 
geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/list/LRemExecutor.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.commands.executor.list;
+
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToLong;
+import static org.apache.geode.redis.internal.netty.Coder.narrowLongToInt;
+
+import java.util.List;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.redis.internal.commands.Command;
+import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
+import org.apache.geode.redis.internal.commands.executor.RedisResponse;
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisKey;
+import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+
+public class LRemExecutor implements CommandExecutor {
+
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext 
context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+    Region<RedisKey, RedisData> region = context.getRegion();
+    RedisKey key = command.getKey();

Review comment:
       Tiny performance optimization, but these can be moved to below where we 
check the count, since it's possible that we error out before needing to 
retrieve either of them.

##########
File path: 
geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/AbstractRedisData.java
##########
@@ -267,6 +269,9 @@ public void fromDelta(DataInput in) throws IOException, 
InvalidDeltaException {
       case REMOVE_ELEMENTS_BY_INDEX:
         RemoveElementsByIndex.deserializeFrom(in, this);
         break;
+      case REMOVE_ELEMENTS_BY_INDEX_REVERSE_ORDER:

Review comment:
       I think it might be better to have just one `REMOVE_ELEMENTS_BY_INDEX` 
Delta type, and make sure that when removing indexes, we always remove starting 
from the smallest index and moving forwards through the list. That way we only 
need one method to remove elements by index in `SizeableByteArrayList`.

##########
File path: 
geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -48,6 +49,33 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param count number of elements to remove.
+   *        A count that is 0 removes all matching elements in the list.
+   *        Positive count starts from the head and moves to the tail.
+   *        Negative count starts from the tail and moves to the head.
+   * @param element element to remove
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add
+   * @return amount of elements that were actually removed
+   */
+  public int lrem(int count, byte[] element, Region<RedisKey, RedisData> 
region, RedisKey key) {
+    List<Integer> removedIndexes;
+    if (0 <= count) {
+      removedIndexes = elementList.removeObjectsStartingAtHead(element, count);
+      if (!removedIndexes.isEmpty()) {
+        storeChanges(region, key, new RemoveElementsByIndex(removedIndexes));
+      }
+    } else {
+      removedIndexes = elementList.removeObjectsStartingAtTail(element, 
-count);
+      if (!removedIndexes.isEmpty()) {
+        storeChanges(region, key, new 
RemoveElementsByIndexReverseOrder(removedIndexes));
+      }

Review comment:
       This can be simplified if `RemoveElementsByIndex` is changed to always 
pass a list that's sorted from smallest to largest, since we can have the 
constructor do the sorting and know that what comes out on the other end will 
be in the right order to safely remove from the list by index. The single use 
of the no-arg constructor for `RemoveElementsByIndex` in `lpop()` could be 
removed and replaced by just creating the ArrayList in `lpop()` and passing 
that to the constructor rather than using an `add()` method.

##########
File path: 
geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LRemDUnitTest.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.commands.executor.list;
+
+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.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
+
+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 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 LRemDUnitTest {
+  private static final int MAX_MATCHING_ELEMENTS = 141; // Summation of 145 is 
10011
+  private static final String ELEMENT_TO_CHECK = "insertedValue";
+
+  @Rule
+  public RedisClusterStartupRule clusterStartUp = new 
RedisClusterStartupRule();
+
+  @Rule
+  public ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  private static JedisCluster jedis;
+
+  @Before
+  public void testSetup() {
+    MemberVM locator = clusterStartUp.startLocatorVM(0);
+    clusterStartUp.startRedisVM(1, locator.getPort());
+    clusterStartUp.startRedisVM(2, locator.getPort());
+    clusterStartUp.startRedisVM(3, locator.getPort());
+    int redisServerPort = clusterStartUp.getRedisPort(1);
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort), 
REDIS_CLIENT_TIMEOUT);
+    clusterStartUp.flushAll();
+  }
+
+  @After
+  public void tearDown() {
+    jedis.close();
+  }
+
+  @Test
+  public void shouldDistributeDataAmongCluster_andRetainDataAfterServerCrash() 
{
+    String key = makeListKeyWithHashtag(1, 
clusterStartUp.getKeyOnServer("lrem", 1));
+    List<String> elementList = makeListWithRepeatingElements(key);
+    elementList.add(ELEMENT_TO_CHECK);
+
+    lpushPerformAndVerify(key, elementList);
+
+    Random rand = new Random();
+
+    // Check for an element that doesn't exist in the list
+    Collections.reverse(elementList);
+    assertThat(jedis.lrem(key, getCount(rand, 1), 
"nonExistentValue")).isEqualTo(0);
+    assertThat(jedis.lrange(key, 0, 
elementList.size())).isEqualTo(elementList);
+
+    // Remove all elements except for ELEMENT_TO_CHECK
+    for (int i = MAX_MATCHING_ELEMENTS - 1; i >= 0; i--) {
+      assertThat(jedis.lrem(key, getCount(rand, i), makeElementString(key, 
i))).isEqualTo(i);
+    }
+
+    clusterStartUp.crashVM(1); // kill primary server
+
+    assertThat(jedis.lrem(key, getCount(rand, 1), 
ELEMENT_TO_CHECK)).isEqualTo(1);
+    assertThat(jedis.exists(key)).isFalse();
+  }
+
+  // TODO: Need to remove ignore and modify once GEODE-10108 is done
+  @Ignore
+  @Test
+  public void givenBucketsMoveDuringLrem_thenOperationsAreNotLost() throws 
Exception {
+    AtomicLong runningCount = new AtomicLong(3);
+    List<String> listHashtags = makeListHashtags();
+    List<String> keys = makeListKeys(listHashtags);
+
+    List<String> elementList1 = makeListWithRepeatingElements(keys.get(0));
+    List<String> elementList2 = makeListWithRepeatingElements(keys.get(1));
+    List<String> elementList3 = makeListWithRepeatingElements(keys.get(2));
+
+    lpushPerformAndVerify(keys.get(0), elementList1);
+    lpushPerformAndVerify(keys.get(1), elementList2);
+    lpushPerformAndVerify(keys.get(2), elementList3);
+
+    int initialListSize = elementList1.size();
+    Runnable task1 =
+        () -> lremPerformAndVerify(keys.get(0), runningCount, initialListSize);
+    Runnable task2 =
+        () -> lremPerformAndVerify(keys.get(1), runningCount, initialListSize);
+    Runnable task3 =
+        () -> lremPerformAndVerify(keys.get(2), runningCount, initialListSize);
+
+    Future<Void> future1 = executor.runAsync(task1);
+    Future<Void> future2 = executor.runAsync(task2);
+    Future<Void> future3 = executor.runAsync(task3);
+
+    for (int i = 0; i < 50 && runningCount.get() > 0; i++) {
+      clusterStartUp.moveBucketForKey(listHashtags.get(i % 
listHashtags.size()));
+      Thread.sleep(500);
+    }
+
+    runningCount.set(0);
+
+    future1.get();
+    future2.get();
+    future3.get();
+  }
+
+  private void lremPerformAndVerify(String key, AtomicLong runningCount, int 
listSize) {
+    assertThat(jedis.llen(key)).isEqualTo(listSize);
+
+    Random rand = new Random();
+    long elementCount = MAX_MATCHING_ELEMENTS - 1;
+    while (jedis.llen(key) > 0 && runningCount.get() > 0) {
+      String element = makeElementString(key, (int) elementCount);
+      assertThat(jedis.lrem(key, getCount(rand, elementCount), 
element)).isEqualTo(elementCount);
+      elementCount--;
+    }

Review comment:
       This test could be simplified a little without losing coverage, I think. 
What we're really wanting to test in this case is that we don't remove too many 
elements, or remove the wrong elements, when buckets are moving. Making the 
list complex and making what we're removing from it random just makes it harder 
to validate rather than actually providing meaningfully different behaviour vs. 
just simple, repetitive operations.
   
   It might be better to have a simpler list (a shuffled list of interleaved 
values of "keep" and "remove", maybe) and a fixed count that gets its sign 
flipped after each iteration, so we alternate removing from the head or the 
tail. After each execution of LREM, we should confirm both that what was 
returned matched what was expected (either the same as the value of count, 
assuming the number of matching elements is a multiple of the count, or 0 if 
we've removed all the matching elements) and also that the size of the list is 
what it should be, and the number of non-matching elements ("keep" in my 
example) is correct, using LRANGE. That way, we can fail as soon as we see 
either too many elements removed or the wrong elements removed.

##########
File path: 
geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableByteArrayList.java
##########
@@ -31,6 +33,57 @@
       roundUpSize(getObjectHeaderSize() + 3 * getReferenceSize());
   private int memberOverhead;
 
+  /**
+   * @param o element to remove from the list
+   * @param count number of elements that match object o to remove from the 
list.
+   *        Count that is equal to 0 removes all matching elements from the 
list.
+   * @return list of indexes that were removed in order.
+   */
+  public List<Integer> removeObjectsStartingAtHead(Object o, int count) {
+    int index = 0;
+    ListIterator<byte[]> iterator = this.listIterator(index);
+    List<Integer> indexesRemoved = 0 < count ? new ArrayList<>(count) : new 
ArrayList<>();

Review comment:
       Using the value of `count` to initialize this list is potentially 
unsafe, since a user could pass a very large value (larger than the size of the 
list) and cause OOM errors when we try to allocate an ArrayList of that size. 
It would be better to use whichever value is smaller out of `count` and 
`size()` to initialize the ArrayList here. Also, we could possibly use the 
value of `size()` to initialize it when `count` is zero, which would be faster 
in terms of never having to resize the array, but possibly produce more garbage 
since we'd almost always be creating an ArrayList that's a lot bigger than we 
need to.

##########
File path: 
geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LRemDUnitTest.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.commands.executor.list;
+
+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.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
+
+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 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 LRemDUnitTest {
+  private static final int MAX_MATCHING_ELEMENTS = 141; // Summation of 145 is 
10011

Review comment:
       This comment seems at odds with the value associated with this constant.

##########
File path: 
geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LRemDUnitTest.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.commands.executor.list;
+
+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.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
+
+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 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 LRemDUnitTest {
+  private static final int MAX_MATCHING_ELEMENTS = 141; // Summation of 145 is 
10011
+  private static final String ELEMENT_TO_CHECK = "insertedValue";
+
+  @Rule
+  public RedisClusterStartupRule clusterStartUp = new 
RedisClusterStartupRule();
+
+  @Rule
+  public ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  private static JedisCluster jedis;
+
+  @Before
+  public void testSetup() {
+    MemberVM locator = clusterStartUp.startLocatorVM(0);
+    clusterStartUp.startRedisVM(1, locator.getPort());
+    clusterStartUp.startRedisVM(2, locator.getPort());
+    clusterStartUp.startRedisVM(3, locator.getPort());
+    int redisServerPort = clusterStartUp.getRedisPort(1);
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort), 
REDIS_CLIENT_TIMEOUT);
+    clusterStartUp.flushAll();
+  }
+
+  @After
+  public void tearDown() {
+    jedis.close();
+  }
+
+  @Test
+  public void shouldDistributeDataAmongCluster_andRetainDataAfterServerCrash() 
{
+    String key = makeListKeyWithHashtag(1, 
clusterStartUp.getKeyOnServer("lrem", 1));
+    List<String> elementList = makeListWithRepeatingElements(key);
+    elementList.add(ELEMENT_TO_CHECK);
+
+    lpushPerformAndVerify(key, elementList);
+
+    Random rand = new Random();
+
+    // Check for an element that doesn't exist in the list
+    Collections.reverse(elementList);
+    assertThat(jedis.lrem(key, getCount(rand, 1), 
"nonExistentValue")).isEqualTo(0);
+    assertThat(jedis.lrange(key, 0, 
elementList.size())).isEqualTo(elementList);
+
+    // Remove all elements except for ELEMENT_TO_CHECK
+    for (int i = MAX_MATCHING_ELEMENTS - 1; i >= 0; i--) {
+      assertThat(jedis.lrem(key, getCount(rand, i), makeElementString(key, 
i))).isEqualTo(i);
+    }

Review comment:
       This approach seems a little convoluted. Instead of 141 different 
elements repeated various numbers of times, this test would be just as good 
with a list consisting of {"1", "2", "3"} repeated a few times, followed by one 
execution of LREM with a count of 0 for element "2" and one execution of LREM 
with a count of -(list size) for element "3", followed by crashing the server 
and confirming that we end up with what's expected (a list containing N repeats 
of "1") after failing over the secondary. Since we're only executing one 
operation at a time, and not doing any concurrent bucket/server stuff, we don't 
need to make this test too complicated.

##########
File path: 
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/RedisListTest.java
##########
@@ -110,6 +118,58 @@ public void equals_returnsTrue_givenDifferentEmptyLists() {
     assertThat(list2).isEqualTo(list1);
   }
 
+  @Test
+  public void lrem_storesStableDelta_inOrderRemove() {
+    Region<RedisKey, RedisData> region = 
uncheckedCast(mock(PartitionedRegion.class));
+    when(region.put(any(), 
any())).thenAnswer(this::validateDeltaSerialization);
+
+    byte[] element = new byte[] {1};
+    RedisList list = createRedisList();
+
+    list.lrem(2, element, region, null);
+
+    verify(region).put(any(), any());
+    assertThat(list.hasDelta()).isFalse();
+  }
+
+  @Test
+  public void lrem_storesStableDelta_reverseOrderRemove() {
+    Region<RedisKey, RedisData> region = 
uncheckedCast(mock(PartitionedRegion.class));
+    when(region.put(any(), 
any())).thenAnswer(this::validateDeltaSerialization);
+
+    byte[] element = new byte[] {1};
+    RedisList list = createRedisList();
+
+    list.lrem(-2, element, region, null);
+
+    verify(region).put(any(), any());
+    assertThat(list.hasDelta()).isFalse();
+  }
+
+  private Object validateDeltaSerialization(InvocationOnMock invocation) 
throws IOException {
+    RedisList value = invocation.getArgument(1, RedisList.class);
+    assertThat(value.hasDelta()).isTrue();
+    HeapDataOutputStream out = new HeapDataOutputStream(100);
+    value.toDelta(out);
+    ByteArrayDataInput in = new ByteArrayDataInput(out.toByteArray());
+    RedisList list2 = createRedisList();
+    assertThat(list2).isNotEqualTo(value);
+    list2.fromDelta(in);
+    assertThat(list2).isEqualTo(value);
+    return null;
+  }
+
+
+  private RedisList createRedisList() {

Review comment:
       To avoid confusion with the method below it, could this be renamed 
something like "createRedisListWithDuplicateElements"?

##########
File path: 
geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LRemDUnitTest.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.commands.executor.list;
+
+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.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicLong;
+
+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 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 LRemDUnitTest {
+  private static final int MAX_MATCHING_ELEMENTS = 141; // Summation of 145 is 
10011
+  private static final String ELEMENT_TO_CHECK = "insertedValue";
+
+  @Rule
+  public RedisClusterStartupRule clusterStartUp = new 
RedisClusterStartupRule();
+
+  @Rule
+  public ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  private static JedisCluster jedis;
+
+  @Before
+  public void testSetup() {
+    MemberVM locator = clusterStartUp.startLocatorVM(0);
+    clusterStartUp.startRedisVM(1, locator.getPort());
+    clusterStartUp.startRedisVM(2, locator.getPort());
+    clusterStartUp.startRedisVM(3, locator.getPort());
+    int redisServerPort = clusterStartUp.getRedisPort(1);
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort), 
REDIS_CLIENT_TIMEOUT);
+    clusterStartUp.flushAll();
+  }
+
+  @After
+  public void tearDown() {
+    jedis.close();
+  }
+
+  @Test
+  public void shouldDistributeDataAmongCluster_andRetainDataAfterServerCrash() 
{
+    String key = makeListKeyWithHashtag(1, 
clusterStartUp.getKeyOnServer("lrem", 1));
+    List<String> elementList = makeListWithRepeatingElements(key);
+    elementList.add(ELEMENT_TO_CHECK);
+
+    lpushPerformAndVerify(key, elementList);
+
+    Random rand = new Random();
+
+    // Check for an element that doesn't exist in the list
+    Collections.reverse(elementList);
+    assertThat(jedis.lrem(key, getCount(rand, 1), 
"nonExistentValue")).isEqualTo(0);
+    assertThat(jedis.lrange(key, 0, 
elementList.size())).isEqualTo(elementList);
+
+    // Remove all elements except for ELEMENT_TO_CHECK
+    for (int i = MAX_MATCHING_ELEMENTS - 1; i >= 0; i--) {
+      assertThat(jedis.lrem(key, getCount(rand, i), makeElementString(key, 
i))).isEqualTo(i);
+    }
+
+    clusterStartUp.crashVM(1); // kill primary server
+
+    assertThat(jedis.lrem(key, getCount(rand, 1), 
ELEMENT_TO_CHECK)).isEqualTo(1);
+    assertThat(jedis.exists(key)).isFalse();
+  }
+
+  // TODO: Need to remove ignore and modify once GEODE-10108 is done
+  @Ignore

Review comment:
       The `@Ignore` annotation lets you add a comment inline using this format:
   ```
   @Ignore("Remove once GEODE-10108 is resolved")
   ```
   but since GEODE-10108 has now been merged, this test and the implementation 
of this command should probably be modified to address the GII/delta bug.

##########
File path: 
geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLRemIntegrationTest.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.commands.executor.list;
+
+import static 
org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
+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.Arrays;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.Protocol;
+
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.redis.RedisIntegrationTest;
+
+public abstract class AbstractLRemIntegrationTest implements 
RedisIntegrationTest {
+  private static final String NON_EXISTENT_LIST_KEY = "{tag1}nonExistentKey";
+  private static final String LIST_KEY = "{tag1}listKey";
+  private static final String[] LIST_ELEMENTS =
+      {"pause", "cynic", "sugar", "skill", "pause", "pause", "pause", "aroma", 
"sugar", "pause",
+          "elder"};
+  private 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 lrem_wrongNumberOfArgs_returnsError() {
+    assertExactNumberOfArgs(jedis, Protocol.Command.LREM, 3);
+  }
+
+  @Test
+  public void lrem_withNonExistentList_returnsZero() {
+    assertThat(jedis.lrem(NON_EXISTENT_LIST_KEY, 2, "element")).isEqualTo(0);
+    assertThat(jedis.lrem(NON_EXISTENT_LIST_KEY, -2, "element")).isEqualTo(0);
+    assertThat(jedis.lrem(NON_EXISTENT_LIST_KEY, 0, "element")).isEqualTo(0);
+  }
+
+  @Test
+  public void lrem_withElementNotInList_returnsZero() {
+    jedis.lpush(LIST_KEY, LIST_ELEMENTS);
+    assertThat(jedis.lrem(LIST_KEY, 3, "magic")).isEqualTo(0);
+  }
+
+  @Test
+  public void lrem_withCountAsZero_returnsNumberOfElementsRemoved() {

Review comment:
       This test name would be a little better if it also stated that we expect 
a count of 0 to remove all matching elements, not just that we should return 
how many elements were removed, since we assert on both conditions.

##########
File path: 
geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLRemIntegrationTest.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.commands.executor.list;
+
+import static 
org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_TYPE;
+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.Arrays;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.Protocol;
+
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.redis.RedisIntegrationTest;
+
+public abstract class AbstractLRemIntegrationTest implements 
RedisIntegrationTest {
+  private static final String NON_EXISTENT_LIST_KEY = "{tag1}nonExistentKey";
+  private static final String LIST_KEY = "{tag1}listKey";
+  private static final String[] LIST_ELEMENTS =
+      {"pause", "cynic", "sugar", "skill", "pause", "pause", "pause", "aroma", 
"sugar", "pause",
+          "elder"};
+  private 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 lrem_wrongNumberOfArgs_returnsError() {
+    assertExactNumberOfArgs(jedis, Protocol.Command.LREM, 3);
+  }
+
+  @Test
+  public void lrem_withNonExistentList_returnsZero() {
+    assertThat(jedis.lrem(NON_EXISTENT_LIST_KEY, 2, "element")).isEqualTo(0);
+    assertThat(jedis.lrem(NON_EXISTENT_LIST_KEY, -2, "element")).isEqualTo(0);
+    assertThat(jedis.lrem(NON_EXISTENT_LIST_KEY, 0, "element")).isEqualTo(0);
+  }
+
+  @Test
+  public void lrem_withElementNotInList_returnsZero() {
+    jedis.lpush(LIST_KEY, LIST_ELEMENTS);
+    assertThat(jedis.lrem(LIST_KEY, 3, "magic")).isEqualTo(0);
+  }
+
+  @Test
+  public void lrem_withCountAsZero_returnsNumberOfElementsRemoved() {
+    jedis.lpush(LIST_KEY, LIST_ELEMENTS);
+
+    final String[] result =
+        {"elder", "sugar", "aroma", "skill", "sugar", "cynic"};
+    assertThat(jedis.lrem(LIST_KEY, 0, "pause")).isEqualTo(5);
+    assertThat(jedis.lrange(LIST_KEY, 0, -1)).containsExactly(result);
+  }
+
+  @Test
+  public void lrem_withPositiveCount_returnsNumberOfElementsRemoved() {
+    jedis.lpush(LIST_KEY, LIST_ELEMENTS);
+
+    // Amount of elements to remove is SMALLER than the amount in the list
+    final String[] result1 =
+        {"elder", "sugar", "aroma", "pause", "skill", "sugar", "cynic", 
"pause"};
+    assertThat(jedis.lrem(LIST_KEY, 3, "pause")).isEqualTo(3);
+    assertThat(jedis.lrange(LIST_KEY, 0, -1)).containsExactly(result1);
+
+    // Amount of elements to remove is GREATER than the amount in the list
+    final String[] result2 = {"elder", "aroma", "pause", "skill", "cynic", 
"pause"};
+    assertThat(jedis.lrem(LIST_KEY, 10, "sugar")).isEqualTo(2);
+    assertThat(jedis.lrange(LIST_KEY, 0, -1)).containsExactly(result2);
+  }
+
+  @Test
+  public void lrem_withNegativeCount_returnsNumberOfElementsRemoved() {
+    jedis.lpush(LIST_KEY, LIST_ELEMENTS);
+
+    // Amount of elements to remove is SMALLER than the amount in the list
+    final String[] result1 =
+        {"elder", "pause", "sugar", "aroma", "pause", "pause", "skill", 
"sugar", "cynic"};
+    assertThat(jedis.lrem(LIST_KEY, -2, "pause")).isEqualTo(2);
+    assertThat(jedis.lrange(LIST_KEY, 0, -1)).containsExactly(result1);
+
+    // Amount of elements to remove is GREATER than the amount in the list
+    final String[] result2 = {"elder", "sugar", "aroma", "skill", "sugar", 
"cynic"};
+    assertThat(jedis.lrem(LIST_KEY, -10, "pause")).isEqualTo(3);
+    assertThat(jedis.lrange(LIST_KEY, 0, -1)).containsExactly(result2);
+  }
+
+  @Test
+  public void lrem_withInvalidIndex_returnsErrorNotInteger() {

Review comment:
       This should be "withInvalidCount"




-- 
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-unsubscr...@geode.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to