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