You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2022/02/23 03:13:10 UTC

[GitHub] [geode] Kris-10-0 opened a new pull request #7389: GEODE-9950: Add LRANGE command

Kris-10-0 opened a new pull request #7389:
URL: https://github.com/apache/geode/pull/7389


   This implements a version of the Redis LRANGE command, which is used for list data types. Associated tests were also added.
   
   Given a key, it returns a list of elements in the range (inclusive) for the specified start index and stop index. Both the start and stop indexes are zero based, which starts at the head of the list. Negative indexes start at the tail of the list.
   
   LINDEX was modified to use a new method created in this commit. The method converts negative indexes to the corresponding positive index.
   
   <!-- Thank you for submitting a contribution to Apache Geode. -->
   
   <!-- In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken: 
   -->
   
   ### For all changes:
   - [x] Is there a JIRA ticket associated with this PR? Is it referenced in the commit message?
   
   - [x] Has your PR been rebased against the latest commit within the target branch (typically `develop`)?
   
   - [x] Is your initial contribution a single, squashed commit?
   
   - [x] Does `gradlew build` run cleanly?
   
   - [x] Have you written or updated unit tests to verify your changes?
   
   - [NA] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   
   <!-- Note:
   Please ensure that once the PR is submitted, check Concourse for build issues and
   submit an update to your PR as soon as possible. If you need help, please send an
   email to dev@geode.apache.org.
   -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] Kris-10-0 commented on a change in pull request #7389: GEODE-9950: Add LRANGE command

Posted by GitBox <gi...@apache.org>.
Kris-10-0 commented on a change in pull request #7389:
URL: https://github.com/apache/geode/pull/7389#discussion_r815176258



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();
+    int curIndex;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      curIndex = 0;
+      ListIterator<byte[]> iterator = elementList.listIterator(curIndex);
+
+      while (iterator.hasNext() && curIndex <= stop) {
+        byte[] element = iterator.next();
+        if (start <= curIndex) {
+          result.add(element);
+        }
+        curIndex = iterator.nextIndex();

Review comment:
       It looks like calling a normal iterator just returns a list iterator.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] DonalEvans commented on a change in pull request #7389: GEODE-9950: Add LRANGE command

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on a change in pull request #7389:
URL: https://github.com/apache/geode/pull/7389#discussion_r816201891



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/list/LRangeExecutor.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.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.RedisKey;
+import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+
+public class LRangeExecutor implements CommandExecutor {
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+    RedisKey key = command.getKey();

Review comment:
       Very small nitpick, but it's a tiny bit more efficient to move this line to after the start and stop indexes are retrieved, as it's possible that we error out before ever needing to use the key. Definitely not a required change though.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLRangeIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+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 AbstractLRangeIntegrationTest 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 =
+      {"aardvark", "bats", "chameleon", "deer", "elephant", "flamingo", "goat"};
+  private static final String[] LIST_ELEMENTS_REVERSE =
+      {"goat", "flamingo", "elephant", "deer", "chameleon", "bats", "aardvark"};
+  private static final int LAST_INDEX = LIST_ELEMENTS.length;
+
+  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 lrange_wrongNumberOfArgs_returnsError() {
+    assertExactNumberOfArgs(jedis, Protocol.Command.LRANGE, 3);
+  }
+
+  @Test
+  public void lrange_withNonExistentList_withStartIndexLessThanStopIndex_returnsEmptyList() {
+    assertThat(jedis.lrange(NON_EXISTENT_LIST_KEY, -10, 10)).isEmpty();
+  }
+
+  @Test
+  public void lrange_withNonExistentList_withStartIndexGreaterThanStopIndex_returnsEmptyList() {
+    assertThat(jedis.lrange(NON_EXISTENT_LIST_KEY, 10, -10)).isEmpty();
+  }
+
+  @Test
+  public void lrange_withPositiveStartIndex_withPositiveStopIndex_returnsEmptyList() {

Review comment:
       A lot of the tests in this class are very similar, differing only in the specific values of start and stop they use. As a result, the names of the tests can get a bit imprecise and possibly misleading, such as this one, since we don't expect that LRANGE will return an empty list in ALL cases when start is positive and stop is positive.
   
   A better approach (that would reduce the amount of test code significantly) might be to follow the pattern in `AbstractZRevRangeIntegrationTest.shouldReturnEmptyCollection_givenInvalidRange()` and `AbstractZRevRangeIntegrationTest.zrevrange_withValidRanges()`, using "valid" and "invalid" ranges and parameterizing the tests. This would allow the test names to specify the range they're using exactly, rather than having to use imprecise descriptions like "positive" or "negative" which may not capture the behaviour we're actually interested in testing.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -38,28 +42,88 @@
   protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
   private final SizeableByteArrayList elementList;
 
+  private static final int INVALID_INDEX = -1;
+
   public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments

Review comment:
       Typos here, should be "elements"

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -38,28 +42,88 @@
   protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
   private final SizeableByteArrayList elementList;
 
+  private static final int INVALID_INDEX = -1;
+
   public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = normalizeStartIndex(start);
+    stop = normalizeStopIndex(stop);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    int resultLength = stop - start + 1;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      List<byte[]> result = new ArrayList<>(resultLength);
+      ListIterator<byte[]> iterator = elementList.listIterator(start);
+
+      for (int i = start; i <= stop; i++) {
+        byte[] element = iterator.next();
+        result.add(element);
+      }
+      return result;
+
+    } else {
+      // Starts at tail to access nodes at stop index then iterates backwards
+      byte[][] result = new byte[resultLength][];
+      ListIterator<byte[]> iterator = elementList.listIterator(stop + 1);
+
+      for (int i = resultLength - 1; i >= 0; i--) {
+        byte[] element = iterator.previous();
+        result[i] = element;
+      }
+      return Arrays.asList(result);
+    }
+  }
+
   /**
    * @param index index of desired element. Positive index starts at the head. Negative index starts
    *        at the tail.
    * @return element at index. Null if index is out of range.
    */
   public byte[] lindex(int index) {
-    if (index < 0) {
-      // Changes negative index to corresponding positive index to utilize get(int index)
-      index = elementList.size() + index;
-    }
+    index = getArrayIndex(index);
 
-    if (index < 0 || elementList.size() <= index) {
+    if (index == INVALID_INDEX || elementList.size() <= index) {
       return null;
     } else {
       return elementList.get(index);
     }
   }
 
+  private int normalizeStartIndex(int startIndex) {
+    return Math.max(0, getArrayIndex(startIndex));
+  }
+
+  private int normalizeStopIndex(int stopIndex) {
+    return Math.min(elementList.size() - 1, getArrayIndex(stopIndex));
+  }
+
+  /**
+   * Changes negative index to corresponding positive index.
+   * The index will still be negative if there is no corresponding positive index.
+   */
+  private int getArrayIndex(int listIndex) {
+    if (listIndex < 0) {
+      listIndex = elementList.size() + listIndex;
+      listIndex = Math.max(listIndex, INVALID_INDEX);
+    }
+    return listIndex;

Review comment:
       This might be slightly clearer as:
   ```
       if (listIndex < 0) {
         listIndex = elementList.size() + listIndex;
         if (listIndex < 0) {
           return INVALID_INDEX;
         }
       }
       return listIndex;
   ```
   Also, the comment on the method could be changed to say something like "Returns INVALID_INDEX if there is no corresponding positive index" to make it more specific.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] Kris-10-0 commented on a change in pull request #7389: GEODE-9950: Add LRANGE command

Posted by GitBox <gi...@apache.org>.
Kris-10-0 commented on a change in pull request #7389:
URL: https://github.com/apache/geode/pull/7389#discussion_r815167406



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {

Review comment:
       This checks if the start is greater than the element size. But I can move the normalization of stop before the check.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #7389: GEODE-9950: Add LRANGE command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #7389:
URL: https://github.com/apache/geode/pull/7389#discussion_r813329013



##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLRangeIntegrationTest.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+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 AbstractLRangeIntegrationTest 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 =
+      {"aardvark", "bats", "chameleon", "deer", "elephant", "flamingo", "goat"};
+  private static final String[] LIST_ELEMENTS_REVERSE =
+      {"goat", "flamingo", "elephant", "deer", "chameleon", "bats", "aardvark"};
+  private static final int LAST_INDEX = LIST_ELEMENTS.length;
+
+  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 lrange_wrongNumberOfArgs_returnsError() {
+    assertExactNumberOfArgs(jedis, Protocol.Command.LRANGE, 3);
+  }
+
+  @Test
+  public void lrange_withNonExistentSet_withStartIndexLessThanStopIndex_returnsEmptyList() {
+    assertThat(jedis.lrange(NON_EXISTENT_LIST_KEY, -10, 10)).isEmpty();
+  }
+
+  @Test
+  public void lrange_withNonExistentSet_withStartIndexGreaterThanStopIndex_returnsEmptyList() {

Review comment:
       Did you mean to have `List` in the method name instead of `Set`?
   
   

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLRangeIntegrationTest.java
##########
@@ -0,0 +1,272 @@
+/*
+ * 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.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+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 AbstractLRangeIntegrationTest 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 =
+      {"aardvark", "bats", "chameleon", "deer", "elephant", "flamingo", "goat"};
+  private static final String[] LIST_ELEMENTS_REVERSE =
+      {"goat", "flamingo", "elephant", "deer", "chameleon", "bats", "aardvark"};
+  private static final int LAST_INDEX = LIST_ELEMENTS.length;
+
+  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 lrange_wrongNumberOfArgs_returnsError() {
+    assertExactNumberOfArgs(jedis, Protocol.Command.LRANGE, 3);
+  }
+
+  @Test
+  public void lrange_withNonExistentSet_withStartIndexLessThanStopIndex_returnsEmptyList() {

Review comment:
       Did you mean to have `List` in the method name instead of `Set`?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +46,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = start < 0 ? 0 : start;

Review comment:
       Intellij suggests converting to `Math.max(0, start)`

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +46,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = start < 0 ? 0 : start;
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();
+    int curIndex;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      ListIterator<byte[]> iterator = elementList.listIterator();

Review comment:
       You can call `listIterator` with a start index for a small optimization.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #7389: GEODE-9950: Add LRANGE command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #7389:
URL: https://github.com/apache/geode/pull/7389#discussion_r815175719



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();
+    int curIndex;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      curIndex = 0;
+      ListIterator<byte[]> iterator = elementList.listIterator(curIndex);

Review comment:
       I thought that `listIterator(index)` worked a bit differently so there's really no optimization here. However, my original thought was to use `start` in this call which then allows you to remove the `if (start <= curIndex) ...` check below.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] jdeppe-pivotal merged pull request #7389: GEODE-9950: Add LRANGE command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal merged pull request #7389:
URL: https://github.com/apache/geode/pull/7389


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] Kris-10-0 commented on a change in pull request #7389: GEODE-9950: Add LRANGE command

Posted by GitBox <gi...@apache.org>.
Kris-10-0 commented on a change in pull request #7389:
URL: https://github.com/apache/geode/pull/7389#discussion_r813440408



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +46,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = start < 0 ? 0 : start;
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();
+    int curIndex;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      ListIterator<byte[]> iterator = elementList.listIterator();

Review comment:
       I changed it, but in what way does it create that small optimization? 
   
   I also modified the iterator for when it goes backwards since the default constructor for that is basically creating an listIterator with  an index that is the size of the list. It seemed similar to your suggestion here. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #7389: GEODE-9950: Add LRANGE command

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #7389:
URL: https://github.com/apache/geode/pull/7389#discussion_r815210408



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();
+    int curIndex;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      curIndex = 0;
+      ListIterator<byte[]> iterator = elementList.listIterator(curIndex);

Review comment:
       I think the code would be easier to understand if we passed "start" or "stop-1" to listIterator(index) and it would get rid of the if check on the loop.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #7389: GEODE-9950: Add LRANGE command

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #7389:
URL: https://github.com/apache/geode/pull/7389#discussion_r815076771



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -60,6 +112,11 @@ public RedisList() {
     }
   }
 
+  /** Changes negative index to corresponding positive index */
+  private int transformNegIndexToPosIndex(int index) {

Review comment:
       abbreviations are frowned upon these days (like Neg and Pos). Could this method just be named "normalizeIndex" or "getArrayIndex" and the parameter could be named "listIndex"? 

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();
+    int curIndex;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      curIndex = 0;
+      ListIterator<byte[]> iterator = elementList.listIterator(curIndex);
+
+      while (iterator.hasNext() && curIndex <= stop) {
+        byte[] element = iterator.next();
+        if (start <= curIndex) {
+          result.add(element);
+        }
+        curIndex = iterator.nextIndex();

Review comment:
       Could this just be "curIndex++"? Also could the block of code just use a normal iterator instead of a listIterator?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -60,6 +112,11 @@ public RedisList() {
     }
   }
 
+  /** Changes negative index to corresponding positive index */
+  private int transformNegIndexToPosIndex(int index) {

Review comment:
       Something I find confusing about this method is even though it says it will convert it to a positive index, it could still be negative (if size() is smaller than -index.
   It seems like you should have a normalizeStartIndex that also does the Math.max(0, start) check;
   and another normalizeStopIndex that does the elementSize <= stop ? elementSize - 1 : stop check.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();
+    int curIndex;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      curIndex = 0;
+      ListIterator<byte[]> iterator = elementList.listIterator(curIndex);
+
+      while (iterator.hasNext() && curIndex <= stop) {
+        byte[] element = iterator.next();
+        if (start <= curIndex) {
+          result.add(element);
+        }
+        curIndex = iterator.nextIndex();
+      }
+    } else {
+      // Starts at tail to access nodes at stop index then iterates backwards
+      curIndex = elementSize - 1;
+      ListIterator<byte[]> iterator = elementList.listIterator(elementSize);
+
+      while (iterator.hasPrevious() && start <= curIndex) {
+        byte[] element = iterator.previous();
+        if (curIndex <= stop) {
+          result.add(0, element); // LinkedList is used to add to head
+        }
+        curIndex = iterator.previousIndex();

Review comment:
       could this just be "curIndex--"?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();
+    int curIndex;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      curIndex = 0;
+      ListIterator<byte[]> iterator = elementList.listIterator(curIndex);
+
+      while (iterator.hasNext() && curIndex <= stop) {
+        byte[] element = iterator.next();
+        if (start <= curIndex) {
+          result.add(element);
+        }
+        curIndex = iterator.nextIndex();
+      }
+    } else {
+      // Starts at tail to access nodes at stop index then iterates backwards
+      curIndex = elementSize - 1;
+      ListIterator<byte[]> iterator = elementList.listIterator(elementSize);
+
+      while (iterator.hasPrevious() && start <= curIndex) {
+        byte[] element = iterator.previous();
+        if (curIndex <= stop) {
+          result.add(0, element); // LinkedList is used to add to head
+        }
+        curIndex = iterator.previousIndex();
+      }
+    }
+    return result;
+  }
+
   /**
    * @param index index of desired element. Positive index starts at the head. Negative index starts
    *        at the tail.
    * @return element at index. Null if index is out of range.
    */
   public byte[] lindex(int index) {
-    if (index < 0) {
-      // Changes negative index to corresponding positive index to utilize get(int index)
-      index = elementList.size() + index;
-    }
+    index = transformNegIndexToPosIndex(index);
 
     if (index < 0 || elementList.size() <= index) {

Review comment:
       if transformNegIndexToPosIndex checked to see if it was going to return something < 0 and instead returns Integer.MAX then this if statement could drop the "index < 0" check.
   Maybe not Integer.MAX just so we could support lists of size 2G. Are redis native lists also limited to <= 2G elements?
   But we could just have this method make sure the index it returns would be a valid list index [0...size-1]. If not then it could either throw an exception or return an exceptional value (for example we could define a constant int INVALID_INDEX = -1). Then the callers can just ask if the result is invalid and then return null/emptyList. Once lrange knows start and stop are both valid it just needs the additional check that start is not greator that stop.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {

Review comment:
       is it really correct to check if start > stop before you do the next "stop" check of it being out of range? I think if you normalized stop first then all you need to check here is start > stop since you know stop is < elementSize.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();

Review comment:
       given that you have figured out start and stop at this point it seems like you know exactly how many elements you will return. So I'd expect an array here instead of LinkedList.
   I know you have a case that you iterator the elements in reverse order but couldn't you then just add them to the array in that case at the last array index and work your way down to the 0 element?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();
+    int curIndex;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      curIndex = 0;
+      ListIterator<byte[]> iterator = elementList.listIterator(curIndex);
+
+      while (iterator.hasNext() && curIndex <= stop) {
+        byte[] element = iterator.next();

Review comment:
       It looks like "element" is only used inside the "if" so should this call be moved inside the if?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -42,16 +45,65 @@ public RedisList() {
     this.elementList = new SizeableByteArrayList();
   }
 
+  /**
+   * @param start start index of desired elments
+   * @param stop stop index of desired elments
+   * @return list of elements in the range (inclusive).
+   */
+  public List<byte[]> lrange(int start, int stop) {
+    start = transformNegIndexToPosIndex(start);
+    stop = transformNegIndexToPosIndex(stop);
+
+    // Out of range negative index changes to 0
+    start = Math.max(0, start);
+
+    int elementSize = elementList.size();
+    if (start > stop || elementSize <= start) {
+      return Collections.emptyList();
+    }
+
+    // Out of range positive stop index changes to last index available
+    stop = elementSize <= stop ? elementSize - 1 : stop;
+
+    List<byte[]> result = new LinkedList<>();
+    int curIndex;
+
+    // Finds the shortest distance to access nodes in range
+    if (start <= elementSize - stop - 1) {
+      // Starts at head to access nodes at start index then iterates forwards
+      curIndex = 0;
+      ListIterator<byte[]> iterator = elementList.listIterator(curIndex);
+
+      while (iterator.hasNext() && curIndex <= stop) {
+        byte[] element = iterator.next();
+        if (start <= curIndex) {
+          result.add(element);
+        }
+        curIndex = iterator.nextIndex();
+      }
+    } else {
+      // Starts at tail to access nodes at stop index then iterates backwards
+      curIndex = elementSize - 1;
+      ListIterator<byte[]> iterator = elementList.listIterator(elementSize);
+
+      while (iterator.hasPrevious() && start <= curIndex) {
+        byte[] element = iterator.previous();

Review comment:
       It looks like "element" is only used inside the "if" so should this call be moved inside the if?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@geode.apache.org

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