DonalEvans commented on a change in pull request #7403: URL: https://github.com/apache/geode/pull/7403#discussion_r837847372
########## File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java ########## @@ -252,6 +253,63 @@ public void lset(Region<RedisKey, RedisData> region, RedisKey key, int index, by storeChanges(region, key, new ReplaceByteArrayAtOffset(index, value)); } + /** + * @param start the index of the first element to retain + * @param end the index of the last element to retain + * @param region the region this instance is stored in + * @param key the name of the list to pop from + * @return the element actually popped + */ + public byte[] ltrim(long start, long end, Region<RedisKey, RedisData> region, + RedisKey key) { + int length = elementList.size(); + int boundedStart = getBoundedStartIndex(start, length); + int boundedEnd = getBoundedEndIndex(end, length); + + if (boundedStart > boundedEnd || boundedStart == length) { + // Remove everything + region.remove(key); + return null; + } + + if (boundedStart == 0 && boundedEnd == length) { + // No-op, return without modifying the list + return null; + } + + RetainElementsByIndexRange retainElementsByRange; + synchronized (this) { + if (boundedEnd < length) { + // trim stuff at end of list + elementList.clearSublist(boundedEnd + 1, length); + } + if (boundedStart > 0) { + // trim stuff at start of list + elementList.clearSublist(0, boundedStart); + } Review comment: To prevent code duplication, this should probably be replaced with `elementsRetainByIndexRange(boundedStart, boundedEnd);` ########## File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java ########## @@ -252,6 +253,63 @@ public void lset(Region<RedisKey, RedisData> region, RedisKey key, int index, by storeChanges(region, key, new ReplaceByteArrayAtOffset(index, value)); } + /** + * @param start the index of the first element to retain + * @param end the index of the last element to retain + * @param region the region this instance is stored in + * @param key the name of the list to pop from + * @return the element actually popped + */ + public byte[] ltrim(long start, long end, Region<RedisKey, RedisData> region, + RedisKey key) { + int length = elementList.size(); + int boundedStart = getBoundedStartIndex(start, length); + int boundedEnd = getBoundedEndIndex(end, length); + + if (boundedStart > boundedEnd || boundedStart == length) { + // Remove everything + region.remove(key); + return null; Review comment: It might be worth adding an implementation of `ltrim()` to `NullRedisList` that just returns immediately, since then we can avoid doing these steps. I'm not sure how expensive a call to `region.remove()` for a non-existent key is, but it's probably worth avoiding if we can. ########## File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java ########## @@ -252,6 +253,63 @@ public void lset(Region<RedisKey, RedisData> region, RedisKey key, int index, by storeChanges(region, key, new ReplaceByteArrayAtOffset(index, value)); } + /** + * @param start the index of the first element to retain + * @param end the index of the last element to retain + * @param region the region this instance is stored in + * @param key the name of the list to pop from + * @return the element actually popped + */ + public byte[] ltrim(long start, long end, Region<RedisKey, RedisData> region, + RedisKey key) { + int length = elementList.size(); + int boundedStart = getBoundedStartIndex(start, length); + int boundedEnd = getBoundedEndIndex(end, length); + + if (boundedStart > boundedEnd || boundedStart == length) { + // Remove everything + region.remove(key); + return null; + } + + if (boundedStart == 0 && boundedEnd == length) { + // No-op, return without modifying the list + return null; + } + + RetainElementsByIndexRange retainElementsByRange; + synchronized (this) { + if (boundedEnd < length) { + // trim stuff at end of list + elementList.clearSublist(boundedEnd + 1, length); + } + if (boundedStart > 0) { + // trim stuff at start of list + elementList.clearSublist(0, boundedStart); + } + retainElementsByRange = + new RetainElementsByIndexRange(incrementAndGetVersion(), boundedStart, boundedEnd); + } + storeChanges(region, key, retainElementsByRange); + return null; + } + + private int getBoundedStartIndex(long index, int size) { + if (index >= 0L) { + return (int) Math.min(index, size); + } else { + return (int) Math.max(index + size, 0); + } + } + + private int getBoundedEndIndex(long index, int size) { + if (index >= 0L) { + return (int) Math.min(index, size); + } else { + return (int) Math.max(index + size, -1); + } Review comment: This method is still inconsistent, since for a list of size `n`, passing in `n` as the index returns a different value than passing in `-1`, when they should be equivalent. This can lead to us failing to hit the early return in `ltrim()` when the operation is a no-op and sending a delta that we don't need to. This method should return the exclusive index to be used in `clearSublist()` so that we can be consistent with how Java deals with indexes and not have to add 1 to the value before passing it into other methods, i.e. it should return `size -1` if the index passed to it is greater than or equal to `size`, or if the index is `-1`. ########## File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLTrimIntegrationTest.java ########## @@ -0,0 +1,160 @@ +/* + * 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.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import junitparams.Parameters; +import junitparams.naming.TestCaseName; +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.ConcurrentLoopingThreads; +import org.apache.geode.redis.RedisIntegrationTest; +import org.apache.geode.test.junit.runners.GeodeParamsRunner; + +@RunWith(GeodeParamsRunner.class) +public abstract class AbstractLTrimIntegrationTest implements RedisIntegrationTest { + public static final String KEY = "key"; + public static final String PREEXISTING_VALUE = "preexistingValue"; Review comment: This is only used in once place in the test and never referenced elsewhere, so it can be inlined. ########## File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLTrimIntegrationTest.java ########## @@ -0,0 +1,160 @@ +/* + * 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.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import junitparams.Parameters; +import junitparams.naming.TestCaseName; +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.ConcurrentLoopingThreads; +import org.apache.geode.redis.RedisIntegrationTest; +import org.apache.geode.test.junit.runners.GeodeParamsRunner; + +@RunWith(GeodeParamsRunner.class) +public abstract class AbstractLTrimIntegrationTest implements RedisIntegrationTest { + public static final String KEY = "key"; + public static final String PREEXISTING_VALUE = "preexistingValue"; + 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 givenWrongNumOfArgs_returnsError() { + assertExactNumberOfArgs(jedis, Protocol.Command.LTRIM, 3); + } + + @Test + public void withNonListKey_Fails() { + jedis.set("string", PREEXISTING_VALUE); + assertThatThrownBy(() -> jedis.ltrim("string", 0, -1)) + .hasMessage(ERROR_WRONG_TYPE); + } + + @Test + public void withNonExistentKey_returnsOK() { + assertThat(jedis.ltrim("nonexistent", 0, -1)).isEqualTo("OK"); + } + + @Test + public void withNonIntegerRangeSpecifier_Fails() { + jedis.lpush(KEY, "e1", "e2", "e3", "e4"); + + assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.LTRIM, KEY, + "0", "not-an-integer")) + .hasMessage(ERROR_NOT_INTEGER); + assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.LTRIM, KEY, + "not-an-integer", "-1")) + .hasMessage(ERROR_NOT_INTEGER); + assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.LTRIM, KEY, + "not-an-integer", "not-an-integer")) + .hasMessage(ERROR_NOT_INTEGER); + } + + @Test + @Parameters(method = "getValidRanges") + @TestCaseName("{method}: start:{0}, end:{1}, expected:{2}") + public void trimsToSpecifiedRange_givenValidRange(long start, long end, String[] expected) { + initializeTestList(); + + jedis.ltrim(KEY, start, end); + assertThat(jedis.lrange(KEY, 0, -1)).containsExactly(expected); + } + + // @Parameterized.Parameters(name = "start:{0}, end:{1}, expected:{2}") + @SuppressWarnings("unused") + private Object[] getValidRanges() { + // Values are start, end, expected result + // For initial list of {e4, e3, e2, e1} + return new Object[] { + new Object[] {0L, 0L, new String[] {"e4"}}, + new Object[] {0L, 1L, new String[] {"e4", "e3"}}, + new Object[] {0L, 2L, new String[] {"e4", "e3", "e2"}}, + new Object[] {1L, 2L, new String[] {"e3", "e2"}}, + new Object[] {1L, -1L, new String[] {"e3", "e2", "e1"}}, + new Object[] {1L, -2L, new String[] {"e3", "e2"}}, + new Object[] {-2L, -1L, new String[] {"e2", "e1"}}, + new Object[] {-1L, -1L, new String[] {"e1"}}, + new Object[] {0L, 3L, new String[] {"e4", "e3", "e2", "e1"}}, + new Object[] {2L, 3L, new String[] {"e2", "e1"}}, + new Object[] {3L, 4L, new String[] {"e1"}}, + new Object[] {0L, 4L, new String[] {"e4", "e3", "e2", "e1"}}, + new Object[] {0L, 10L, new String[] {"e4", "e3", "e2", "e1"}}, + new Object[] {-5L, -1L, new String[] {"e4", "e3", "e2", "e1"}}, + new Object[] {-10L, 10L, new String[] {"e4", "e3", "e2", "e1"}} + }; + } + + private void initializeTestList() { + jedis.del(KEY); Review comment: This line is not necessary, as we call `flushAll()` in the `@Before` method, which gets called before each iteration of the test, which means that this line can be removed and the method can be inlined. ########## File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLTrimIntegrationTest.java ########## @@ -0,0 +1,160 @@ +/* + * 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.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import junitparams.Parameters; +import junitparams.naming.TestCaseName; +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.ConcurrentLoopingThreads; +import org.apache.geode.redis.RedisIntegrationTest; +import org.apache.geode.test.junit.runners.GeodeParamsRunner; + +@RunWith(GeodeParamsRunner.class) +public abstract class AbstractLTrimIntegrationTest implements RedisIntegrationTest { + public static final String KEY = "key"; + public static final String PREEXISTING_VALUE = "preexistingValue"; + 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 givenWrongNumOfArgs_returnsError() { + assertExactNumberOfArgs(jedis, Protocol.Command.LTRIM, 3); + } + + @Test + public void withNonListKey_Fails() { + jedis.set("string", PREEXISTING_VALUE); + assertThatThrownBy(() -> jedis.ltrim("string", 0, -1)) + .hasMessage(ERROR_WRONG_TYPE); + } + + @Test + public void withNonExistentKey_returnsOK() { + assertThat(jedis.ltrim("nonexistent", 0, -1)).isEqualTo("OK"); + } + + @Test + public void withNonIntegerRangeSpecifier_Fails() { + jedis.lpush(KEY, "e1", "e2", "e3", "e4"); + + assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.LTRIM, KEY, + "0", "not-an-integer")) + .hasMessage(ERROR_NOT_INTEGER); + assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.LTRIM, KEY, + "not-an-integer", "-1")) + .hasMessage(ERROR_NOT_INTEGER); + assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.LTRIM, KEY, + "not-an-integer", "not-an-integer")) + .hasMessage(ERROR_NOT_INTEGER); + } + + @Test + @Parameters(method = "getValidRanges") + @TestCaseName("{method}: start:{0}, end:{1}, expected:{2}") + public void trimsToSpecifiedRange_givenValidRange(long start, long end, String[] expected) { + initializeTestList(); + + jedis.ltrim(KEY, start, end); + assertThat(jedis.lrange(KEY, 0, -1)).containsExactly(expected); + } + + // @Parameterized.Parameters(name = "start:{0}, end:{1}, expected:{2}") + @SuppressWarnings("unused") + private Object[] getValidRanges() { + // Values are start, end, expected result + // For initial list of {e4, e3, e2, e1} + return new Object[] { + new Object[] {0L, 0L, new String[] {"e4"}}, + new Object[] {0L, 1L, new String[] {"e4", "e3"}}, + new Object[] {0L, 2L, new String[] {"e4", "e3", "e2"}}, + new Object[] {1L, 2L, new String[] {"e3", "e2"}}, + new Object[] {1L, -1L, new String[] {"e3", "e2", "e1"}}, + new Object[] {1L, -2L, new String[] {"e3", "e2"}}, + new Object[] {-2L, -1L, new String[] {"e2", "e1"}}, + new Object[] {-1L, -1L, new String[] {"e1"}}, + new Object[] {0L, 3L, new String[] {"e4", "e3", "e2", "e1"}}, + new Object[] {2L, 3L, new String[] {"e2", "e1"}}, + new Object[] {3L, 4L, new String[] {"e1"}}, + new Object[] {0L, 4L, new String[] {"e4", "e3", "e2", "e1"}}, + new Object[] {0L, 10L, new String[] {"e4", "e3", "e2", "e1"}}, + new Object[] {-5L, -1L, new String[] {"e4", "e3", "e2", "e1"}}, + new Object[] {-10L, 10L, new String[] {"e4", "e3", "e2", "e1"}} + }; + } + + private void initializeTestList() { + jedis.del(KEY); + jedis.lpush(KEY, "e1", "e2", "e3", "e4"); + } + + @Test + public void removesKey_whenAllElementsTrimmed() { + initializeTestList(); + + jedis.ltrim(KEY, 0, -5); Review comment: This test could be expanded to "removesKey_whenRangeIsEmpty()" and parameterized to use various ranges that do not contain any elements e.g. both start and end are past the end of the list, both start and end are before the start of the list etc. ########## File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java ########## @@ -221,6 +222,63 @@ public int llen() { return elementList.size(); } + /** + * @param start the index of the first element to retain + * @param end the index of the last element to retain + * @param region the region this instance is stored in + * @param key the name of the list to pop from + * @return the element actually popped + */ + public byte[] ltrim(long start, long end, Region<RedisKey, RedisData> region, Review comment: I think the method return type being `Void` here would still be a nice improvement, since it's misleading to imply that a `byte[]` is returned when this method always returns `null`. Also, `start` and `end` are now passed in as `int`, so the method signature (and the signatures of `getBoundedStartIndex()` and `getBoundedEndIndex()`) should be changed to reflect that. ########## File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableByteArrayList.java ########## @@ -83,6 +83,39 @@ return indexesRemoved; } + public void clearSublist(int fromIndex, int toIndex) { Review comment: Since this is a custom implementation rather than just an overridden one that calls the `super()` method, we should have comprehensive unit tests for this method. It's not something that should be addressed as part of this PR, but we're also missing unit tests for several other public methods on this class, which should definitely be added. It would be good to file a JIRA ticket to track that testing gap. ########## File path: geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/collections/SizeableByteArrayListTest.java ########## @@ -58,6 +58,24 @@ public void getSizeInBytesIsAccurate_ForSizeableByteArrayListElements() { assertThat(list.size()).isEqualTo(0); } + @Test + public void clearSublist_getSizeInBytesIsAccurate() { + // Create a list with an initial size and confirm that it correctly reports its size + SizeableByteArrayList list = createList(); + assertThat(list.getSizeInBytes()).isEqualTo(sizer.sizeof(list)); + + // Remove subset of elements and assert that the size is correct after each remove + Random rand = new Random(); + while (list.size() > 3) { + int fromIndex = rand.nextInt(list.size() / 2); + int toIndex = rand.nextInt(list.size() / 2) + fromIndex; Review comment: This test doesn't need to use randomness; that just complicates things and could potentially lead to a more difficult to diagnose bug if the test does fail. For this test, it would be enough to just create a list, remove some sublist from the middle once, then assert that the size is correct. ########## File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableByteArrayList.java ########## @@ -83,6 +83,39 @@ return indexesRemoved; } + public void clearSublist(int fromIndex, int toIndex) { + if (fromIndex < size() / 2) { + clearFromBeginning(fromIndex, toIndex); + } else { + clearFromEnd(fromIndex, toIndex); + } + } + + public void clearFromBeginning(int fromIndex, int toIndex) { Review comment: This method should be private. ########## File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLTrimIntegrationTest.java ########## @@ -0,0 +1,160 @@ +/* + * 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.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import junitparams.Parameters; +import junitparams.naming.TestCaseName; +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.ConcurrentLoopingThreads; +import org.apache.geode.redis.RedisIntegrationTest; +import org.apache.geode.test.junit.runners.GeodeParamsRunner; + +@RunWith(GeodeParamsRunner.class) +public abstract class AbstractLTrimIntegrationTest implements RedisIntegrationTest { Review comment: To help catch potential edge cases, could we have a parameterized test for the behaviour when the list contains only one element? I added some trace logging and played around a bit and noticed that there was some inconsistent/unexpected behaviour when the list contains only one element that it would be good to explicitly test. ########## File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableByteArrayList.java ########## @@ -83,6 +83,39 @@ return indexesRemoved; } + public void clearSublist(int fromIndex, int toIndex) { + if (fromIndex < size() / 2) { + clearFromBeginning(fromIndex, toIndex); + } else { + clearFromEnd(fromIndex, toIndex); + } + } + + public void clearFromBeginning(int fromIndex, int toIndex) { + ListIterator<byte[]> iterator = listIterator(fromIndex); + int removeCount = toIndex - fromIndex; + int count = 0; + + while (iterator.hasNext() && count < removeCount) { + byte[] element = iterator.next(); + iterator.remove(); + count++; + memberOverhead -= calculateByteArrayOverhead(element); + } + } + + private void clearFromEnd(int fromIndex, int toIndex) { + ListIterator<byte[]> descendingIterator = listIterator(toIndex); + int removedCount = toIndex - fromIndex; + + while (descendingIterator.hasPrevious() && removedCount > 0) { + byte[] element = descendingIterator.previous(); + descendingIterator.remove(); + removedCount--; + memberOverhead -= calculateByteArrayOverhead(element); + } + } Review comment: For consistency, could this method be modified to match the implementation of `clearFromBeginning()`: ``` private void clearFromEnd(int fromIndex, int toIndex) { ListIterator<byte[]> descendingIterator = listIterator(toIndex); int removeCount = toIndex - fromIndex; int count = 0; while (descendingIterator.hasPrevious() && count < removeCount) { byte[] element = descendingIterator.previous(); descendingIterator.remove(); count++; memberOverhead -= calculateByteArrayOverhead(element); } ``` ########## File path: geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LTrimDUnitTest.java ########## @@ -0,0 +1,181 @@ +/* + * 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.List; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.junit.After; +import org.junit.Before; +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 LTrimDUnitTest { + public static final int INITIAL_LIST_SIZE = 5_000; + + @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("ltrim", 1)); + List<String> elementList = makeElementList(key, INITIAL_LIST_SIZE); + lpushPerformAndVerify(key, elementList); + + // Remove all but last element + jedis.ltrim(key, INITIAL_LIST_SIZE - 1, INITIAL_LIST_SIZE); + + clusterStartUp.crashVM(1); // kill primary server + + assertThat(jedis.lindex(key, 0)).isEqualTo(elementList.get(0)); + jedis.ltrim(key, 0, -2); + assertThat(jedis.exists(key)).isFalse(); + } + + @Test + public void givenBucketsMoveDuringLtrim_thenOperationsAreNotLost() throws Exception { + AtomicBoolean isRunning = new AtomicBoolean(true); + List<String> listHashtags = makeListHashtags(); + List<String> keys = makeListKeys(listHashtags); + + List<String> elementList1 = makeElementList(keys.get(0), INITIAL_LIST_SIZE); + List<String> elementList2 = makeElementList(keys.get(1), INITIAL_LIST_SIZE); + List<String> elementList3 = makeElementList(keys.get(2), INITIAL_LIST_SIZE); + + Runnable task1 = + () -> ltrimPerformAndVerify(keys.get(0), isRunning, elementList1); + Runnable task2 = + () -> ltrimPerformAndVerify(keys.get(1), isRunning, elementList2); + Runnable task3 = + () -> ltrimPerformAndVerify(keys.get(2), isRunning, elementList3); + + Future<Void> future1 = executor.runAsync(task1); + Future<Void> future2 = executor.runAsync(task2); + Future<Void> future3 = executor.runAsync(task3); + + for (int i = 0; i < 100; i++) { + clusterStartUp.moveBucketForKey(listHashtags.get(i % listHashtags.size())); + Thread.sleep(200); + } + + isRunning.set(false); + + future1.get(); + future2.get(); + future3.get(); + } + + private List<String> makeListHashtags() { + List<String> listHashtags = new ArrayList<>(); + listHashtags.add(clusterStartUp.getKeyOnServer("ltrim", 1)); + listHashtags.add(clusterStartUp.getKeyOnServer("ltrim", 2)); + listHashtags.add(clusterStartUp.getKeyOnServer("ltrim", 3)); + return listHashtags; + } + + private List<String> makeListKeys(List<String> listHashtags) { + List<String> keys = new ArrayList<>(); + keys.add(makeListKeyWithHashtag(1, listHashtags.get(0))); + keys.add(makeListKeyWithHashtag(2, listHashtags.get(1))); + keys.add(makeListKeyWithHashtag(3, listHashtags.get(2))); + return keys; + } + + private void lpushPerformAndVerify(String key, List<String> elementList) { + jedis.lpush(key, elementList.toArray(new String[] {})); + + Long listLength = jedis.llen(key); + assertThat(listLength).as("Initial list lengths not equal for key %s'", key) + .isEqualTo(elementList.size()); + } + + private void ltrimPerformAndVerify(String key, + AtomicBoolean isRunning, + List<String> elementList) { + while (isRunning.get()) { + lpushPerformAndVerify(key, elementList); + + for (int i = 1; i <= INITIAL_LIST_SIZE / 2 && isRunning.get(); i++) { + long lastIndex = jedis.llen(key) - 2; + try { + assertThat(jedis.lindex(key, 0)) + .as("lpush head verification failed at iteration " + i) + .isEqualTo(makeElementString(key, INITIAL_LIST_SIZE - i)); + assertThat(jedis.lindex(key, lastIndex)) + .as("lpush tail verification failed at iteration " + i) + .isEqualTo(makeElementString(key, i)); + jedis.ltrim(key, 1, lastIndex); + assertThat(jedis.llen(key)).as("Key: %s ", key).isEqualTo(lastIndex); + } catch (Throwable ex) { + isRunning.set(false); // test is over + throw ex; + } + } + if (isRunning.get()) { + assertThat(jedis.exists(key)).isFalse(); + } Review comment: This can be simplified somewhat by using negative indexes and doing the assertions after performing LTRIM (since we already assert on the contents of the list as part of `lpushPerformAndVerify()` so we should never see the list be wrong before we've even done an LTRIM): ``` for (int i = 1; i < INITIAL_LIST_SIZE / 2 && isRunning.get(); i++) { try { jedis.ltrim(key, 1, -2); assertThat(jedis.llen(key)).as("Key: %s ", key).isEqualTo(INITIAL_LIST_SIZE - (i * 2L)); assertThat(jedis.lindex(key, 0)) .as("LTRIM head verification failed at iteration " + i) .isEqualTo(makeElementString(key, INITIAL_LIST_SIZE - 1 - i)); assertThat(jedis.lindex(key, -1)) .as("LTRIM tail verification failed at iteration " + i) .isEqualTo(makeElementString(key, i)); } catch (Throwable ex) { isRunning.set(false); // test is over throw ex; } } if (isRunning.get()) { jedis.ltrim(key, 1, -2); assertThat(jedis.exists(key)).isFalse(); } ``` ########## File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableByteArrayList.java ########## @@ -83,6 +83,39 @@ return indexesRemoved; } + public void clearSublist(int fromIndex, int toIndex) { + if (fromIndex < size() / 2) { Review comment: This attempt at optimization is not working the way you intend, I think. For a list of 100 elements with `fromIndex = 49`, `toIndex = 100`, you end up starting at the beginning of the list and iterating all 100 elements, rather than starting at the end and only having to iterate 51. The correct optimization would be to check which of `fromIndex` and `toIndex` is closer to its respective end of the list, and then clear from that side. ########## File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java ########## @@ -252,6 +253,63 @@ public void lset(Region<RedisKey, RedisData> region, RedisKey key, int index, by storeChanges(region, key, new ReplaceByteArrayAtOffset(index, value)); } + /** + * @param start the index of the first element to retain + * @param end the index of the last element to retain + * @param region the region this instance is stored in + * @param key the name of the list to pop from + * @return the element actually popped + */ + public byte[] ltrim(long start, long end, Region<RedisKey, RedisData> region, + RedisKey key) { + int length = elementList.size(); + int boundedStart = getBoundedStartIndex(start, length); + int boundedEnd = getBoundedEndIndex(end, length); + + if (boundedStart > boundedEnd || boundedStart == length) { + // Remove everything + region.remove(key); + return null; + } + + if (boundedStart == 0 && boundedEnd == length) { + // No-op, return without modifying the list + return null; + } Review comment: It might be nice to add a test to `RedisListTest.java` to confirm that we hit this early return under various conditions, as I noticed that for lists with just one element, we were still sending a delta in some situations even though the list wasn't modified at all. ########## File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLTrimIntegrationTest.java ########## @@ -0,0 +1,160 @@ +/* + * 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.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; + +import junitparams.Parameters; +import junitparams.naming.TestCaseName; +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.ConcurrentLoopingThreads; +import org.apache.geode.redis.RedisIntegrationTest; +import org.apache.geode.test.junit.runners.GeodeParamsRunner; + +@RunWith(GeodeParamsRunner.class) +public abstract class AbstractLTrimIntegrationTest implements RedisIntegrationTest { + public static final String KEY = "key"; + public static final String PREEXISTING_VALUE = "preexistingValue"; + 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 givenWrongNumOfArgs_returnsError() { + assertExactNumberOfArgs(jedis, Protocol.Command.LTRIM, 3); + } + + @Test + public void withNonListKey_Fails() { + jedis.set("string", PREEXISTING_VALUE); + assertThatThrownBy(() -> jedis.ltrim("string", 0, -1)) + .hasMessage(ERROR_WRONG_TYPE); + } + + @Test + public void withNonExistentKey_returnsOK() { + assertThat(jedis.ltrim("nonexistent", 0, -1)).isEqualTo("OK"); + } + + @Test + public void withNonIntegerRangeSpecifier_Fails() { + jedis.lpush(KEY, "e1", "e2", "e3", "e4"); + + assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.LTRIM, KEY, + "0", "not-an-integer")) + .hasMessage(ERROR_NOT_INTEGER); + assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.LTRIM, KEY, + "not-an-integer", "-1")) + .hasMessage(ERROR_NOT_INTEGER); + assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.LTRIM, KEY, + "not-an-integer", "not-an-integer")) + .hasMessage(ERROR_NOT_INTEGER); + } + + @Test + @Parameters(method = "getValidRanges") + @TestCaseName("{method}: start:{0}, end:{1}, expected:{2}") + public void trimsToSpecifiedRange_givenValidRange(long start, long end, String[] expected) { + initializeTestList(); + + jedis.ltrim(KEY, start, end); + assertThat(jedis.lrange(KEY, 0, -1)).containsExactly(expected); + } + + // @Parameterized.Parameters(name = "start:{0}, end:{1}, expected:{2}") Review comment: This comment should be removed. ########## File path: geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LTrimDUnitTest.java ########## @@ -0,0 +1,181 @@ +/* + * 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.List; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.junit.After; +import org.junit.Before; +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 LTrimDUnitTest { + public static final int INITIAL_LIST_SIZE = 5_000; + + @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("ltrim", 1)); + List<String> elementList = makeElementList(key, INITIAL_LIST_SIZE); + lpushPerformAndVerify(key, elementList); + + // Remove all but last element + jedis.ltrim(key, INITIAL_LIST_SIZE - 1, INITIAL_LIST_SIZE); + + clusterStartUp.crashVM(1); // kill primary server + + assertThat(jedis.lindex(key, 0)).isEqualTo(elementList.get(0)); + jedis.ltrim(key, 0, -2); + assertThat(jedis.exists(key)).isFalse(); + } + + @Test + public void givenBucketsMoveDuringLtrim_thenOperationsAreNotLost() throws Exception { + AtomicBoolean isRunning = new AtomicBoolean(true); + List<String> listHashtags = makeListHashtags(); + List<String> keys = makeListKeys(listHashtags); + + List<String> elementList1 = makeElementList(keys.get(0), INITIAL_LIST_SIZE); + List<String> elementList2 = makeElementList(keys.get(1), INITIAL_LIST_SIZE); + List<String> elementList3 = makeElementList(keys.get(2), INITIAL_LIST_SIZE); + + Runnable task1 = + () -> ltrimPerformAndVerify(keys.get(0), isRunning, elementList1); + Runnable task2 = + () -> ltrimPerformAndVerify(keys.get(1), isRunning, elementList2); + Runnable task3 = + () -> ltrimPerformAndVerify(keys.get(2), isRunning, elementList3); + + Future<Void> future1 = executor.runAsync(task1); + Future<Void> future2 = executor.runAsync(task2); + Future<Void> future3 = executor.runAsync(task3); + + for (int i = 0; i < 100; i++) { Review comment: in order to have the test stop early if `ltrimPerformAndVerify()` encounters an exception, you need to also include a check for `isRunning.get()` in the for loop condition. -- 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