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/01/14 20:52:30 UTC

[GitHub] [geode] DonalEvans commented on a change in pull request #7261: Geode 9892 create infrastructure for redis lists

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



##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractLPopArgumentCountIntegrationTest.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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 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.RedisIntegrationTest;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+
+// We only need this test because LPop has a different argument count between Redis 5.x
+// and Redis 6.2+. But the pipeline is testing against Redis 6.2.x. Once the "COUNT"
+// subcommand is implemented for LPOP, this test (and LPopArgumentCountIntegrationTest)
+// should be terminated with prejudice

Review comment:
       It shouldn't be necessary to have a separate test class for this, because of the way we structure our tests. We can instead have a test case in the `LpopIntegrationTest` (not the Abstract parent class) that validates the behaviour we expect from geode-for-redis without that test then being run against native Redis

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractListsIntegrationTest.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.assertAtLeastNArgs;
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+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 redis.clients.jedis.exceptions.JedisDataException;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+
+public abstract class AbstractListsIntegrationTest implements RedisIntegrationTest {

Review comment:
       Rather than having multiple commands tested in one test class, can we follow the convention of having a separate test class for each command? I realize that it's a little weird here, because it's not really possible to test LLEN and LPOP without also calling LPUSH, but it would make things neater and more consistent to have test classes separated by command.

##########
File path: geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LPopDUnitTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+
+public class LPopDUnitTest {
+
+  @ClassRule
+  public static RedisClusterStartupRule clusterStartUp = new RedisClusterStartupRule(4);
+
+  private static final String LOCAL_HOST = "127.0.0.1";
+  private static final int LIST_SIZE = 1000;
+  private static final int JEDIS_TIMEOUT =

Review comment:
       The `LOCAL_HOST` and `JEDIS_TIMEOUT` constants here should be removed and the equivalent constants from the `RedisClusterStartupRule` class used instead.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractListsIntegrationTest.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.assertAtLeastNArgs;
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+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 redis.clients.jedis.exceptions.JedisDataException;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+
+public abstract class AbstractListsIntegrationTest implements RedisIntegrationTest {
+
+  public static final String KEY = "key";
+  public static final String PREEXISTING_VALUE = "preexistingValue";
+  private static final int REDIS_CLIENT_TIMEOUT =
+      Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  private JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort("localhost", getPort()), REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    flushAll();
+    jedis.close();
+  }
+
+  @Test
+  public void lpushErrors_givenTooFewArguments() {
+    assertAtLeastNArgs(jedis, Protocol.Command.LPUSH, 2);
+  }
+
+  @Test
+  public void lpushErrors_withExistingKey_ofWrongType() {
+    String[] elementValue = new String[1];
+    elementValue[0] = "list element value that should never get added";
+
+    jedis.set(KEY, PREEXISTING_VALUE);
+    assertThatThrownBy(() -> jedis.lpush(KEY, elementValue))
+        .hasMessageContaining("Operation against a key holding the wrong kind of value");
+  }
+
+  @Test
+  public void lpush_withExistingKey_ofWrongType_shouldNotOverWriteExistingKey() {
+    String[] elementValue = new String[1];
+    elementValue[0] = "set value that should never get added";
+
+    jedis.set(KEY, PREEXISTING_VALUE);
+
+    assertThatThrownBy(() -> jedis.lpush(KEY, elementValue)).isInstanceOf(JedisDataException.class);
+
+    String result = jedis.get(KEY);
+
+    assertThat(result).isEqualTo(PREEXISTING_VALUE);
+  }
+
+  @Test
+  public void lpush_canStoreBinaryData() {

Review comment:
       Just for simplicity/readability, could this test use a String instead of byte arrays? The type of data used to call the Jedis command is irrelevant to testing server-side behaviour, since the format of the data sent to the server is the responsibility of the client rather than the server. Regardless of the type of the arguments we use here, they should be serialized into a format that's type-agnostic (byte arrays in the RESP2 protocol) and all handled by the server in the same way, so we can make the test a little more straightforward by just pushing and popping a String here.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/list/LPopExecutor.java
##########
@@ -0,0 +1,47 @@
+/*
+ * 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 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.commands.parameters.RedisParametersMismatchException;
+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 LPopExecutor implements CommandExecutor {
+
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
+    Region<RedisKey, RedisData> region = context.getRegion();
+    RedisKey key = command.getKey();
+    // Needed until we implement 'count' arg (Redis 6.2+)
+    if (command.getProcessedCommand().size() > 2) {
+      throw new RedisParametersMismatchException(command.wrongNumberOfArgumentsErrorMessage());
+    }

Review comment:
       This is unnecessary if the parameter specification in `RedisCommandType` for LPOP is changed to `exact(2)`.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/RedisCommandType.java
##########
@@ -286,6 +289,12 @@
   ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
       new Parameter().min(4).flags(WRITE, DENYOOM, MOVABLEKEYS)),
 
+  /************** Lists *****************/
+
+  LLEN(new LLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(READONLY, FAST)),
+  LPOP(new LPopExecutor(), SUPPORTED, new Parameter().min(2).flags(WRITE, FAST)),

Review comment:
       This should be `exact(2)`, since we do not accept LPOP with the count option. Since the `CommandIntegrationTest` will fail with this change (because it explicitly checks against the output from Redis 6.2.6) the `commandReturnsResultsMatchingNativeRedis()` test there should be modified to account for the difference, as we shouldn't be asserting that the info for the command matches behaviour that we haven't implemented.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {
+      storeChanges(region, key, new AddByteArrays(elementsToAdd));
+    }
+    return elementsAdded;
+  }
+
+  /**
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the element actually popped
+   */
+  public byte[] lpop(Region<RedisKey, RedisData> region, RedisKey key) {
+    int originalSize = elements.size();
+    if (originalSize == 0) {
+      return null;
+    }
+
+    byte[] popped = elements.remove(0);
+    if (popped != null) {
+      RemoveByteArrays removed = new RemoveByteArrays();
+      removed.add(popped);
+      storeChanges(region, key, removed);
+    }
+    return popped;
+  }
+
+  /**
+   * @return the number of elements in the list
+   */
+  public int llen(Region<RedisKey, RedisData> region, RedisKey key) {
+    return elements.size();
+  }
+
+  @Override
+  public void applyAddByteArrayDelta(byte[] bytes) {
+    prependElement(bytes);
+  }
+
+  @Override
+  public void applyRemoveByteArrayDelta(byte[] bytes) {
+    elementRemove(bytes);
+  }
+
+  /**
+   * Since GII (getInitialImage) can come in and call toData while other threads
+   * are modifying this object, the striped executor will not protect toData.
+   * So any methods that modify "members" needs to be thread safe with toData.
+   */
+
+  @Override
+  public synchronized void toData(DataOutput out, SerializationContext context) throws IOException {
+    super.toData(out, context);
+    DataSerializer.writePrimitiveInt(elements.size(), out);
+    for (byte[] member : elements) {
+      DataSerializer.writeByteArray(member, out);
+    }
+  }
+
+  @Override
+  public void fromData(DataInput in, DeserializationContext context)
+      throws IOException, ClassNotFoundException {
+    super.fromData(in, context);
+    int size = DataSerializer.readPrimitiveInt(in);
+    elements = new ElementList(Collections.emptyList()); // TODO: zero arg constructor!!
+    for (int i = 0; i < size; ++i) {
+      elements.add(DataSerializer.readByteArray(in));
+    }
+  }
+
+  @Override
+  public int getDSFID() {
+    return REDIS_LIST_ID;
+  }
+
+  @VisibleForTesting
+  synchronized boolean appendElement(byte[] elementToAdd) {
+    return elements.add(elementToAdd);
+  }
+
+  @VisibleForTesting
+  synchronized boolean prependElement(byte[] elementToAdd) {
+    elements.add(0, elementToAdd);
+    return true;
+  }
+
+  @VisibleForTesting
+  synchronized boolean elementRemove(byte[] memberToRemove) {
+    return elements.remove(memberToRemove);
+  }

Review comment:
       The `@VisibleForTesting` annotation seems unnecessary, since these methods are never called from a test. As such, these methods should be made `private`.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableList.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.data.collections;
+
+
+import static org.apache.geode.internal.JvmSizeUtils.getObjectHeaderSize;
+import static org.apache.geode.internal.JvmSizeUtils.getReferenceSize;
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.internal.JvmSizeUtils.roundUpSize;
+
+import java.util.Collection;
+import java.util.LinkedList;
+
+import org.apache.geode.internal.size.Sizeable;
+
+public abstract class SizeableList<K> extends LinkedList<K> implements Sizeable {
+  private static final long serialVersionUID = -8255284217236712060L;
+  private static final int SIZEABLE_LIST_OVERHEAD =
+      memoryOverhead(SizeableList.class);
+  private static final int NODE_OVERHEAD =
+      roundUpSize(getObjectHeaderSize() + 3 * getReferenceSize());
+
+  private int memberOverhead;
+
+  public SizeableList(Collection<K> collection) {
+    for (K element : collection) {
+      add(0, element);
+    }
+  }
+
+  public void add(int index, K k) {

Review comment:
       This method and the other add and remove methods should be overriding methods on `LinkedList`

##########
File path: geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/collections/SizeableListTest.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.data.collections;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.geode.cache.util.ObjectSizer;
+import org.apache.geode.internal.size.ReflectionObjectSizer;
+import org.apache.geode.redis.internal.data.RedisList;
+
+public class SizeableListTest {
+  private final ObjectSizer sizer = ReflectionObjectSizer.getInstance();
+
+  private int expectedSize(SizeableList list) {
+    return sizer.sizeof(list);
+  }
+
+  private RedisList.ElementList createTestElementList(int size) {
+    List<byte[]> initialElements = new ArrayList<>(size);
+    for (int i = 0; i < size; ++i) {
+      initialElements.add(new byte[] {(byte) i});
+    }
+    RedisList.ElementList list = new RedisList.ElementList(initialElements);
+    return list;
+  }
+
+  @Test
+  public void getSizeInBytesForEmptyElementList() {
+    RedisList.ElementList list = createTestElementList(0);
+    assertThat(list.getSizeInBytes()).isEqualTo(expectedSize(list));
+  }
+
+  @Test
+  public void getSizeInBytesIsAccurateForByteArrays() {
+    List<byte[]> initialElements = new ArrayList<>();
+    int initialNumberOfElements = 20;
+    int elementsToAdd = 100;
+    for (int i = 0; i < initialNumberOfElements; ++i) {
+      initialElements.add(new byte[] {(byte) i});
+    }
+    // Create a set with an initial size and confirm that it correctly reports its size
+    RedisList.ElementList list = new RedisList.ElementList(initialElements);
+    assertThat(list.getSizeInBytes()).isEqualTo(expectedSize(list));
+
+    // Add enough members to force a resize and assert that the size is correct after each add

Review comment:
       LinkedLists do not resize a backing array based on the number of elements, so this comment is misleading.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractListsIntegrationTest.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.assertAtLeastNArgs;
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+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 redis.clients.jedis.exceptions.JedisDataException;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+
+public abstract class AbstractListsIntegrationTest implements RedisIntegrationTest {
+
+  public static final String KEY = "key";
+  public static final String PREEXISTING_VALUE = "preexistingValue";
+  private static final int REDIS_CLIENT_TIMEOUT =
+      Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  private JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort("localhost", getPort()), REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    flushAll();
+    jedis.close();
+  }
+
+  @Test
+  public void lpushErrors_givenTooFewArguments() {
+    assertAtLeastNArgs(jedis, Protocol.Command.LPUSH, 2);
+  }
+
+  @Test
+  public void lpushErrors_withExistingKey_ofWrongType() {
+    String[] elementValue = new String[1];
+    elementValue[0] = "list element value that should never get added";
+
+    jedis.set(KEY, PREEXISTING_VALUE);
+    assertThatThrownBy(() -> jedis.lpush(KEY, elementValue))
+        .hasMessageContaining("Operation against a key holding the wrong kind of value");

Review comment:
       Can this String be the `ERROR_WRONG_TYPE` constant instead?

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractListsIntegrationTest.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.assertAtLeastNArgs;
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+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 redis.clients.jedis.exceptions.JedisDataException;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+
+public abstract class AbstractListsIntegrationTest implements RedisIntegrationTest {
+
+  public static final String KEY = "key";
+  public static final String PREEXISTING_VALUE = "preexistingValue";
+  private static final int REDIS_CLIENT_TIMEOUT =
+      Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  private JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort("localhost", getPort()), REDIS_CLIENT_TIMEOUT);

Review comment:
       Rather than using a hard-coded "localhost" here, please use the `BIND_ADDRESS` constant in `RedisClusterStartupRule`, similar to what other tests are doing.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractListsIntegrationTest.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.assertAtLeastNArgs;
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+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 redis.clients.jedis.exceptions.JedisDataException;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+
+public abstract class AbstractListsIntegrationTest implements RedisIntegrationTest {
+
+  public static final String KEY = "key";
+  public static final String PREEXISTING_VALUE = "preexistingValue";
+  private static final int REDIS_CLIENT_TIMEOUT =
+      Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());

Review comment:
       This constant is already defined in `RedisClusterStartupRule` so it's not necessary to redefine it here.

##########
File path: geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LPopDUnitTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+
+public class LPopDUnitTest {
+
+  @ClassRule
+  public static RedisClusterStartupRule clusterStartUp = new RedisClusterStartupRule(4);
+
+  private static final String LOCAL_HOST = "127.0.0.1";
+  private static final int LIST_SIZE = 1000;
+  private static final int JEDIS_TIMEOUT =
+      Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  private static JedisCluster jedis;
+
+  private static MemberVM locator;
+  private static MemberVM server1;
+  private static MemberVM server2;
+  private static MemberVM server3;
+
+  @BeforeClass
+  public static void classSetup() {
+    locator = clusterStartUp.startLocatorVM(0);
+    server1 = clusterStartUp.startRedisVM(1, locator.getPort());
+    server2 = clusterStartUp.startRedisVM(2, locator.getPort());
+    server3 = clusterStartUp.startRedisVM(3, locator.getPort());
+
+    int redisServerPort = clusterStartUp.getRedisPort(1);
+    jedis = new JedisCluster(new HostAndPort(LOCAL_HOST, redisServerPort), JEDIS_TIMEOUT);
+  }
+
+  @Before
+  public void testSetup() {
+    clusterStartUp.flushAll();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    jedis.close();
+
+    server1.stop();
+    server2.stop();
+    server3.stop();

Review comment:
       It's not necessary to manually stop the servers, as the `RedisClusterStartupRule` takes care of that for you.

##########
File path: geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LPushDUnitTest.java
##########
@@ -0,0 +1,146 @@
+/*
+ * 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.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+
+public class LPushDUnitTest {

Review comment:
       See comments on `LPopDUnitTest`

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {
+      storeChanges(region, key, new AddByteArrays(elementsToAdd));
+    }
+    return elementsAdded;
+  }
+
+  /**
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the element actually popped
+   */
+  public byte[] lpop(Region<RedisKey, RedisData> region, RedisKey key) {
+    int originalSize = elements.size();
+    if (originalSize == 0) {
+      return null;
+    }
+
+    byte[] popped = elements.remove(0);
+    if (popped != null) {
+      RemoveByteArrays removed = new RemoveByteArrays();
+      removed.add(popped);
+      storeChanges(region, key, removed);
+    }
+    return popped;
+  }
+
+  /**
+   * @return the number of elements in the list
+   */
+  public int llen(Region<RedisKey, RedisData> region, RedisKey key) {

Review comment:
       The signature of this method doesn't need to take any arguments. It can be just `public int llen() {`

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractListsIntegrationTest.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.assertAtLeastNArgs;
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+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 redis.clients.jedis.exceptions.JedisDataException;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+
+public abstract class AbstractListsIntegrationTest implements RedisIntegrationTest {
+
+  public static final String KEY = "key";
+  public static final String PREEXISTING_VALUE = "preexistingValue";
+  private static final int REDIS_CLIENT_TIMEOUT =
+      Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  private JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort("localhost", getPort()), REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    flushAll();
+    jedis.close();
+  }
+
+  @Test
+  public void lpushErrors_givenTooFewArguments() {
+    assertAtLeastNArgs(jedis, Protocol.Command.LPUSH, 2);
+  }
+
+  @Test
+  public void lpushErrors_withExistingKey_ofWrongType() {
+    String[] elementValue = new String[1];
+    elementValue[0] = "list element value that should never get added";
+
+    jedis.set(KEY, PREEXISTING_VALUE);
+    assertThatThrownBy(() -> jedis.lpush(KEY, elementValue))
+        .hasMessageContaining("Operation against a key holding the wrong kind of value");
+  }
+
+  @Test
+  public void lpush_withExistingKey_ofWrongType_shouldNotOverWriteExistingKey() {
+    String[] elementValue = new String[1];
+    elementValue[0] = "set value that should never get added";

Review comment:
       It's not necessary to create an array here, the call to lpush can just be:
   ```
   jedis.lpush(KEY, "value")
   ```

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {

Review comment:
       It shouldn't be possible for `elementsAdded` to be 0, since calling LPUSH without any elements will result in an exception in the executor before we ever get here.

##########
File path: geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LPopDUnitTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+
+public class LPopDUnitTest {

Review comment:
       None of the tests in this class are testing distributed behaviour, as the keys in them remain on the same server throughout the test and are independent in the test where there are multiple keys. The behaviour covered in this test is already covered in full in the LPOP integration tests.
   
   In order to be testing distributed behaviour, the test would need to move buckets and/or crash servers, but given the issues with flakiness we've seen in numerous other Redis-related DUnit tests, it might be more trouble than it's worth to try to add such tests.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractListsIntegrationTest.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.assertAtLeastNArgs;
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+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 redis.clients.jedis.exceptions.JedisDataException;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+
+public abstract class AbstractListsIntegrationTest implements RedisIntegrationTest {
+
+  public static final String KEY = "key";
+  public static final String PREEXISTING_VALUE = "preexistingValue";
+  private static final int REDIS_CLIENT_TIMEOUT =
+      Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  private JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort("localhost", getPort()), REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    flushAll();
+    jedis.close();
+  }
+
+  @Test
+  public void lpushErrors_givenTooFewArguments() {
+    assertAtLeastNArgs(jedis, Protocol.Command.LPUSH, 2);
+  }
+
+  @Test
+  public void lpushErrors_withExistingKey_ofWrongType() {
+    String[] elementValue = new String[1];
+    elementValue[0] = "list element value that should never get added";
+
+    jedis.set(KEY, PREEXISTING_VALUE);
+    assertThatThrownBy(() -> jedis.lpush(KEY, elementValue))
+        .hasMessageContaining("Operation against a key holding the wrong kind of value");
+  }
+
+  @Test
+  public void lpush_withExistingKey_ofWrongType_shouldNotOverWriteExistingKey() {
+    String[] elementValue = new String[1];
+    elementValue[0] = "set value that should never get added";
+
+    jedis.set(KEY, PREEXISTING_VALUE);
+
+    assertThatThrownBy(() -> jedis.lpush(KEY, elementValue)).isInstanceOf(JedisDataException.class);
+
+    String result = jedis.get(KEY);
+
+    assertThat(result).isEqualTo(PREEXISTING_VALUE);
+  }
+
+  @Test
+  public void lpush_canStoreBinaryData() {
+    byte[] blob = new byte[256];
+    for (int i = 0; i < 256; i++) {
+      blob[i] = (byte) i;
+    }
+
+    jedis.lpush(KEY.getBytes(), blob, blob);
+
+    byte[] result = jedis.lpop(KEY.getBytes());
+    assertThat(result).containsExactly(blob);
+    result = jedis.lpop(KEY.getBytes());
+    assertThat(result).containsExactly(blob);
+  }
+
+  // not checking LPOP argument count here, see AbstractLPopArgumentCountIntegrationTest
+
+  @Test
+  public void lpop_withStringFails() {
+    jedis.set("string", PREEXISTING_VALUE);
+    assertThatThrownBy(() -> jedis.lpop("string")).hasMessageContaining("WRONGTYPE");
+  }
+
+  @Test
+  public void lpop_withNonExistentKey_returnsNull() {
+    assertThat(jedis.lpop("nonexistent")).isNull();
+  }
+
+  @Test
+  public void lpop_returnsOneMember() {
+    jedis.lpush(KEY, "e1", "e2");
+    String result = jedis.lpop(KEY);
+    assertThat(result).isIn("e1", "e2");

Review comment:
       This test should be asserting that the returned element is equal to "e2" rather than one of "e1" or "e2", since we expect that LPOP returns the leftmost element in the list.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/list/LPopExecutor.java
##########
@@ -0,0 +1,47 @@
+/*
+ * 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 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.commands.parameters.RedisParametersMismatchException;
+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 LPopExecutor implements CommandExecutor {
+
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
+    Region<RedisKey, RedisData> region = context.getRegion();
+    RedisKey key = command.getKey();
+    // Needed until we implement 'count' arg (Redis 6.2+)
+    if (command.getProcessedCommand().size() > 2) {
+      throw new RedisParametersMismatchException(command.wrongNumberOfArgumentsErrorMessage());
+    }
+    byte[] popped = context.listLockedExecute(key, false,
+        list -> list.lpop(region, key));
+
+    if (popped == null) {
+      return RedisResponse.nil();
+    }
+
+    return RedisResponse.bulkString(popped);

Review comment:
       This can be simplified to just `return RedisResponse.bulkString(popped);`, since the `bulkString()` method handles the case that the String is null.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractListsIntegrationTest.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.assertAtLeastNArgs;
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+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 redis.clients.jedis.exceptions.JedisDataException;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+
+public abstract class AbstractListsIntegrationTest implements RedisIntegrationTest {
+
+  public static final String KEY = "key";
+  public static final String PREEXISTING_VALUE = "preexistingValue";
+  private static final int REDIS_CLIENT_TIMEOUT =
+      Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  private JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort("localhost", getPort()), REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    flushAll();
+    jedis.close();
+  }
+
+  @Test
+  public void lpushErrors_givenTooFewArguments() {
+    assertAtLeastNArgs(jedis, Protocol.Command.LPUSH, 2);
+  }
+
+  @Test
+  public void lpushErrors_withExistingKey_ofWrongType() {
+    String[] elementValue = new String[1];
+    elementValue[0] = "list element value that should never get added";
+
+    jedis.set(KEY, PREEXISTING_VALUE);
+    assertThatThrownBy(() -> jedis.lpush(KEY, elementValue))
+        .hasMessageContaining("Operation against a key holding the wrong kind of value");
+  }
+
+  @Test
+  public void lpush_withExistingKey_ofWrongType_shouldNotOverWriteExistingKey() {
+    String[] elementValue = new String[1];
+    elementValue[0] = "set value that should never get added";
+
+    jedis.set(KEY, PREEXISTING_VALUE);
+
+    assertThatThrownBy(() -> jedis.lpush(KEY, elementValue)).isInstanceOf(JedisDataException.class);
+
+    String result = jedis.get(KEY);
+
+    assertThat(result).isEqualTo(PREEXISTING_VALUE);
+  }
+
+  @Test
+  public void lpush_canStoreBinaryData() {
+    byte[] blob = new byte[256];
+    for (int i = 0; i < 256; i++) {
+      blob[i] = (byte) i;
+    }
+
+    jedis.lpush(KEY.getBytes(), blob, blob);
+
+    byte[] result = jedis.lpop(KEY.getBytes());
+    assertThat(result).containsExactly(blob);
+    result = jedis.lpop(KEY.getBytes());
+    assertThat(result).containsExactly(blob);
+  }
+
+  // not checking LPOP argument count here, see AbstractLPopArgumentCountIntegrationTest
+
+  @Test
+  public void lpop_withStringFails() {
+    jedis.set("string", PREEXISTING_VALUE);
+    assertThatThrownBy(() -> jedis.lpop("string")).hasMessageContaining("WRONGTYPE");

Review comment:
       Could this error string be the `ERROR_WRONG_TYPE` constant instead please?

##########
File path: geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/commands/executor/list/LPopDUnitTest.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+
+public class LPopDUnitTest {
+
+  @ClassRule
+  public static RedisClusterStartupRule clusterStartUp = new RedisClusterStartupRule(4);
+
+  private static final String LOCAL_HOST = "127.0.0.1";
+  private static final int LIST_SIZE = 1000;
+  private static final int JEDIS_TIMEOUT =
+      Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  private static JedisCluster jedis;
+
+  private static MemberVM locator;

Review comment:
       `locator` can be a local variable in the `classSetup()` method.

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/list/AbstractListsIntegrationTest.java
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.assertAtLeastNArgs;
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertExactNumberOfArgs;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+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 redis.clients.jedis.exceptions.JedisDataException;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+import org.apache.geode.test.awaitility.GeodeAwaitility;
+
+public abstract class AbstractListsIntegrationTest implements RedisIntegrationTest {
+
+  public static final String KEY = "key";
+  public static final String PREEXISTING_VALUE = "preexistingValue";
+  private static final int REDIS_CLIENT_TIMEOUT =
+      Math.toIntExact(GeodeAwaitility.getTimeout().toMillis());
+  private JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort("localhost", getPort()), REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    flushAll();
+    jedis.close();
+  }
+
+  @Test
+  public void lpushErrors_givenTooFewArguments() {
+    assertAtLeastNArgs(jedis, Protocol.Command.LPUSH, 2);
+  }
+
+  @Test
+  public void lpushErrors_withExistingKey_ofWrongType() {
+    String[] elementValue = new String[1];
+    elementValue[0] = "list element value that should never get added";

Review comment:
       It's not necessary to create an array here, the call to lpush can just be:
   ```
   jedis.lpush(KEY, "value")
   ```

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {
+      storeChanges(region, key, new AddByteArrays(elementsToAdd));
+    }
+    return elementsAdded;
+  }
+
+  /**
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the element actually popped
+   */
+  public byte[] lpop(Region<RedisKey, RedisData> region, RedisKey key) {
+    int originalSize = elements.size();
+    if (originalSize == 0) {
+      return null;
+    }
+
+    byte[] popped = elements.remove(0);
+    if (popped != null) {

Review comment:
       It's not possible for the popped element here to be null, because we've already established that `elements` is not empty.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {
+      storeChanges(region, key, new AddByteArrays(elementsToAdd));
+    }
+    return elementsAdded;
+  }
+
+  /**
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the element actually popped
+   */
+  public byte[] lpop(Region<RedisKey, RedisData> region, RedisKey key) {
+    int originalSize = elements.size();
+    if (originalSize == 0) {
+      return null;
+    }
+
+    byte[] popped = elements.remove(0);
+    if (popped != null) {
+      RemoveByteArrays removed = new RemoveByteArrays();
+      removed.add(popped);
+      storeChanges(region, key, removed);
+    }
+    return popped;
+  }
+
+  /**
+   * @return the number of elements in the list
+   */
+  public int llen(Region<RedisKey, RedisData> region, RedisKey key) {
+    return elements.size();
+  }
+
+  @Override
+  public void applyAddByteArrayDelta(byte[] bytes) {
+    prependElement(bytes);
+  }
+
+  @Override
+  public void applyRemoveByteArrayDelta(byte[] bytes) {
+    elementRemove(bytes);
+  }
+
+  /**
+   * Since GII (getInitialImage) can come in and call toData while other threads
+   * are modifying this object, the striped executor will not protect toData.
+   * So any methods that modify "members" needs to be thread safe with toData.
+   */
+
+  @Override
+  public synchronized void toData(DataOutput out, SerializationContext context) throws IOException {
+    super.toData(out, context);
+    DataSerializer.writePrimitiveInt(elements.size(), out);
+    for (byte[] member : elements) {
+      DataSerializer.writeByteArray(member, out);
+    }
+  }
+
+  @Override
+  public void fromData(DataInput in, DeserializationContext context)
+      throws IOException, ClassNotFoundException {
+    super.fromData(in, context);
+    int size = DataSerializer.readPrimitiveInt(in);
+    elements = new ElementList(Collections.emptyList()); // TODO: zero arg constructor!!

Review comment:
       This TODO should be removed and the zero-arg constructor used.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {
+      storeChanges(region, key, new AddByteArrays(elementsToAdd));
+    }
+    return elementsAdded;
+  }
+
+  /**
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the element actually popped
+   */
+  public byte[] lpop(Region<RedisKey, RedisData> region, RedisKey key) {
+    int originalSize = elements.size();
+    if (originalSize == 0) {
+      return null;
+    }

Review comment:
       It's not possible for `originalSize` to be 0 here, since if the list becomes empty, it is removed from the region.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {
+      storeChanges(region, key, new AddByteArrays(elementsToAdd));
+    }
+    return elementsAdded;
+  }
+
+  /**
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the element actually popped
+   */
+  public byte[] lpop(Region<RedisKey, RedisData> region, RedisKey key) {
+    int originalSize = elements.size();
+    if (originalSize == 0) {
+      return null;
+    }
+
+    byte[] popped = elements.remove(0);
+    if (popped != null) {
+      RemoveByteArrays removed = new RemoveByteArrays();
+      removed.add(popped);
+      storeChanges(region, key, removed);
+    }
+    return popped;
+  }
+
+  /**
+   * @return the number of elements in the list
+   */
+  public int llen(Region<RedisKey, RedisData> region, RedisKey key) {
+    return elements.size();
+  }
+
+  @Override
+  public void applyAddByteArrayDelta(byte[] bytes) {
+    prependElement(bytes);
+  }
+
+  @Override
+  public void applyRemoveByteArrayDelta(byte[] bytes) {
+    elementRemove(bytes);
+  }
+
+  /**
+   * Since GII (getInitialImage) can come in and call toData while other threads
+   * are modifying this object, the striped executor will not protect toData.
+   * So any methods that modify "members" needs to be thread safe with toData.

Review comment:
       This comment refers to "members" rather than "elements". Also, despite what this comment says, the `lpop()`method (which modifies `elements`) is not synchronized, and therefore not threadsafe with `toData()`.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableList.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.data.collections;
+
+
+import static org.apache.geode.internal.JvmSizeUtils.getObjectHeaderSize;
+import static org.apache.geode.internal.JvmSizeUtils.getReferenceSize;
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.internal.JvmSizeUtils.roundUpSize;
+
+import java.util.Collection;
+import java.util.LinkedList;
+
+import org.apache.geode.internal.size.Sizeable;
+
+public abstract class SizeableList<K> extends LinkedList<K> implements Sizeable {
+  private static final long serialVersionUID = -8255284217236712060L;
+  private static final int SIZEABLE_LIST_OVERHEAD =
+      memoryOverhead(SizeableList.class);
+  private static final int NODE_OVERHEAD =
+      roundUpSize(getObjectHeaderSize() + 3 * getReferenceSize());
+
+  private int memberOverhead;
+
+  public SizeableList(Collection<K> collection) {
+    for (K element : collection) {
+      add(0, element);
+    }

Review comment:
       Calling `add()` here is potentially unsafe, as the `add()` method is public and could potentially be overridden, causing unspecified behaviour in the constructor. I'm not sure what the best solution here is, since calling the super constructor would require overriding `addAll(Collection<? extends E> collection)` to handle the sizing correctly, but that would mean iterating through the collection twice, which isn't ideal.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {
+      storeChanges(region, key, new AddByteArrays(elementsToAdd));
+    }
+    return elementsAdded;
+  }
+
+  /**
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the element actually popped
+   */
+  public byte[] lpop(Region<RedisKey, RedisData> region, RedisKey key) {
+    int originalSize = elements.size();
+    if (originalSize == 0) {
+      return null;
+    }
+
+    byte[] popped = elements.remove(0);
+    if (popped != null) {
+      RemoveByteArrays removed = new RemoveByteArrays();
+      removed.add(popped);
+      storeChanges(region, key, removed);
+    }
+    return popped;
+  }
+
+  /**
+   * @return the number of elements in the list
+   */
+  public int llen(Region<RedisKey, RedisData> region, RedisKey key) {
+    return elements.size();
+  }
+
+  @Override
+  public void applyAddByteArrayDelta(byte[] bytes) {
+    prependElement(bytes);
+  }
+
+  @Override
+  public void applyRemoveByteArrayDelta(byte[] bytes) {
+    elementRemove(bytes);
+  }
+
+  /**
+   * Since GII (getInitialImage) can come in and call toData while other threads
+   * are modifying this object, the striped executor will not protect toData.
+   * So any methods that modify "members" needs to be thread safe with toData.
+   */
+
+  @Override
+  public synchronized void toData(DataOutput out, SerializationContext context) throws IOException {
+    super.toData(out, context);
+    DataSerializer.writePrimitiveInt(elements.size(), out);
+    for (byte[] member : elements) {
+      DataSerializer.writeByteArray(member, out);
+    }
+  }
+
+  @Override
+  public void fromData(DataInput in, DeserializationContext context)
+      throws IOException, ClassNotFoundException {
+    super.fromData(in, context);
+    int size = DataSerializer.readPrimitiveInt(in);
+    elements = new ElementList(Collections.emptyList()); // TODO: zero arg constructor!!
+    for (int i = 0; i < size; ++i) {
+      elements.add(DataSerializer.readByteArray(in));
+    }
+  }
+
+  @Override
+  public int getDSFID() {
+    return REDIS_LIST_ID;
+  }
+
+  @VisibleForTesting
+  synchronized boolean appendElement(byte[] elementToAdd) {
+    return elements.add(elementToAdd);
+  }
+
+  @VisibleForTesting
+  synchronized boolean prependElement(byte[] elementToAdd) {
+    elements.add(0, elementToAdd);
+    return true;
+  }
+
+  @VisibleForTesting
+  synchronized boolean elementRemove(byte[] memberToRemove) {
+    return elements.remove(memberToRemove);
+  }
+
+  @Override
+  public RedisDataType getType() {
+    return REDIS_LIST;
+  }
+
+  @Override
+  protected boolean removeFromRegion() {
+    return elements.isEmpty();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof RedisList)) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    RedisList redisList = (RedisList) o;
+
+    if (redisList.elements.size() != elements.size()) {
+      return false;
+    }
+    for (byte[] element : elements) {
+      if (!redisList.elements.contains(element)) {
+        return false;
+      }

Review comment:
       This equals method is incorrect, as lists require not just that the contents of the collections are the same, but that they're in the same order too. This should iterate both lists and check that the element at each index is the same (using `Arrays.equals()`) in both.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {
+      storeChanges(region, key, new AddByteArrays(elementsToAdd));
+    }
+    return elementsAdded;
+  }
+
+  /**
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the element actually popped
+   */
+  public byte[] lpop(Region<RedisKey, RedisData> region, RedisKey key) {
+    int originalSize = elements.size();
+    if (originalSize == 0) {
+      return null;
+    }
+
+    byte[] popped = elements.remove(0);
+    if (popped != null) {
+      RemoveByteArrays removed = new RemoveByteArrays();
+      removed.add(popped);
+      storeChanges(region, key, removed);
+    }
+    return popped;
+  }
+
+  /**
+   * @return the number of elements in the list
+   */
+  public int llen(Region<RedisKey, RedisData> region, RedisKey key) {
+    return elements.size();
+  }
+
+  @Override
+  public void applyAddByteArrayDelta(byte[] bytes) {
+    prependElement(bytes);
+  }
+
+  @Override
+  public void applyRemoveByteArrayDelta(byte[] bytes) {
+    elementRemove(bytes);
+  }
+
+  /**
+   * Since GII (getInitialImage) can come in and call toData while other threads
+   * are modifying this object, the striped executor will not protect toData.
+   * So any methods that modify "members" needs to be thread safe with toData.
+   */
+
+  @Override
+  public synchronized void toData(DataOutput out, SerializationContext context) throws IOException {
+    super.toData(out, context);
+    DataSerializer.writePrimitiveInt(elements.size(), out);
+    for (byte[] member : elements) {
+      DataSerializer.writeByteArray(member, out);
+    }
+  }
+
+  @Override
+  public void fromData(DataInput in, DeserializationContext context)
+      throws IOException, ClassNotFoundException {
+    super.fromData(in, context);
+    int size = DataSerializer.readPrimitiveInt(in);
+    elements = new ElementList(Collections.emptyList()); // TODO: zero arg constructor!!
+    for (int i = 0; i < size; ++i) {
+      elements.add(DataSerializer.readByteArray(in));
+    }
+  }
+
+  @Override
+  public int getDSFID() {
+    return REDIS_LIST_ID;
+  }
+
+  @VisibleForTesting
+  synchronized boolean appendElement(byte[] elementToAdd) {
+    return elements.add(elementToAdd);
+  }
+
+  @VisibleForTesting
+  synchronized boolean prependElement(byte[] elementToAdd) {

Review comment:
       The return value of this method is never used, and it always returns true, so it should probably just be a `void` method.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {
+      storeChanges(region, key, new AddByteArrays(elementsToAdd));
+    }
+    return elementsAdded;
+  }
+
+  /**
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the element actually popped
+   */
+  public byte[] lpop(Region<RedisKey, RedisData> region, RedisKey key) {
+    int originalSize = elements.size();
+    if (originalSize == 0) {
+      return null;
+    }
+
+    byte[] popped = elements.remove(0);
+    if (popped != null) {
+      RemoveByteArrays removed = new RemoveByteArrays();

Review comment:
       The `RemoveByteArrays` Delta class is not sufficient for use with lists, because it assumes that only one instance of each byte array is possible in the collection (since it's used with Hash, Set and SortedSet, which do not allow duplicate entries). A new `DeltaInfo` class will be required which removes list elements by index, starting from the largest index so that the indexes of subsequent elements to remove aren't changed by removing other elements first.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisList.java
##########
@@ -0,0 +1,223 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_LIST;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.serialization.DeserializationContext;
+import org.apache.geode.internal.serialization.KnownVersion;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.redis.internal.data.collections.SizeableList;
+import org.apache.geode.redis.internal.data.delta.AddByteArrays;
+import org.apache.geode.redis.internal.data.delta.RemoveByteArrays;
+
+public class RedisList extends AbstractRedisData {
+  protected static final int REDIS_LIST_OVERHEAD = memoryOverhead(RedisList.class);
+  private ElementList elements;
+
+  public RedisList(Collection<byte[]> collection) {
+    this.elements = new ElementList(collection);
+  }
+
+  /**
+   * For deserialization only.
+   */
+  public RedisList() {}
+
+  /**
+   * @param elementsToAdd members to add to this set; NOTE this list may by
+   *        modified by this call
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the number of members actually added
+   */
+  public long lpush(List<byte[]> elementsToAdd, Region<RedisKey, RedisData> region, RedisKey key) {
+    for (byte[] element : elementsToAdd) {
+      prependElement(element);
+    }
+    int elementsAdded = elementsToAdd.size();
+    if (elementsAdded != 0) {
+      storeChanges(region, key, new AddByteArrays(elementsToAdd));
+    }
+    return elementsAdded;
+  }
+
+  /**
+   * @param region the region this instance is stored in
+   * @param key the name of the set to add to
+   * @return the element actually popped
+   */
+  public byte[] lpop(Region<RedisKey, RedisData> region, RedisKey key) {
+    int originalSize = elements.size();
+    if (originalSize == 0) {
+      return null;
+    }
+
+    byte[] popped = elements.remove(0);
+    if (popped != null) {
+      RemoveByteArrays removed = new RemoveByteArrays();
+      removed.add(popped);
+      storeChanges(region, key, removed);
+    }
+    return popped;
+  }
+
+  /**
+   * @return the number of elements in the list
+   */
+  public int llen(Region<RedisKey, RedisData> region, RedisKey key) {
+    return elements.size();
+  }
+
+  @Override
+  public void applyAddByteArrayDelta(byte[] bytes) {
+    prependElement(bytes);
+  }
+
+  @Override
+  public void applyRemoveByteArrayDelta(byte[] bytes) {
+    elementRemove(bytes);
+  }
+
+  /**
+   * Since GII (getInitialImage) can come in and call toData while other threads
+   * are modifying this object, the striped executor will not protect toData.
+   * So any methods that modify "members" needs to be thread safe with toData.
+   */
+
+  @Override
+  public synchronized void toData(DataOutput out, SerializationContext context) throws IOException {
+    super.toData(out, context);
+    DataSerializer.writePrimitiveInt(elements.size(), out);
+    for (byte[] member : elements) {
+      DataSerializer.writeByteArray(member, out);
+    }
+  }
+
+  @Override
+  public void fromData(DataInput in, DeserializationContext context)
+      throws IOException, ClassNotFoundException {
+    super.fromData(in, context);
+    int size = DataSerializer.readPrimitiveInt(in);
+    elements = new ElementList(Collections.emptyList()); // TODO: zero arg constructor!!
+    for (int i = 0; i < size; ++i) {
+      elements.add(DataSerializer.readByteArray(in));
+    }
+  }
+
+  @Override
+  public int getDSFID() {
+    return REDIS_LIST_ID;
+  }
+
+  @VisibleForTesting
+  synchronized boolean appendElement(byte[] elementToAdd) {
+    return elements.add(elementToAdd);
+  }
+
+  @VisibleForTesting
+  synchronized boolean prependElement(byte[] elementToAdd) {
+    elements.add(0, elementToAdd);
+    return true;
+  }
+
+  @VisibleForTesting
+  synchronized boolean elementRemove(byte[] memberToRemove) {
+    return elements.remove(memberToRemove);
+  }
+
+  @Override
+  public RedisDataType getType() {
+    return REDIS_LIST;
+  }
+
+  @Override
+  protected boolean removeFromRegion() {
+    return elements.isEmpty();
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof RedisList)) {
+      return false;
+    }
+    if (!super.equals(o)) {
+      return false;
+    }
+    RedisList redisList = (RedisList) o;
+
+    if (redisList.elements.size() != elements.size()) {
+      return false;
+    }
+    for (byte[] element : elements) {
+      if (!redisList.elements.contains(element)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), elements);
+  }
+
+  @Override
+  public String toString() {
+    return "RedisList{" + super.toString() + ", " + "size=" + elements.size() + '}';
+  }
+
+  @Override
+  public KnownVersion[] getSerializationVersions() {
+    return null;
+  }
+
+  @Override
+  public int getSizeInBytes() {
+    return REDIS_LIST_OVERHEAD + elements.getSizeInBytes();
+  }
+
+  public static class ElementList extends SizeableList<byte[]> {
+    public ElementList(Collection<byte[]> initialElements) {
+      super(initialElements);
+    }
+
+    public ElementList() {
+      super(Collections.emptyList());

Review comment:
       Rather than calling the super constructor with an empty collection, it would be neater to add a zero-arg constructor to the super class and call that.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableList.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.data.collections;
+
+
+import static org.apache.geode.internal.JvmSizeUtils.getObjectHeaderSize;
+import static org.apache.geode.internal.JvmSizeUtils.getReferenceSize;
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.internal.JvmSizeUtils.roundUpSize;
+
+import java.util.Collection;
+import java.util.LinkedList;
+
+import org.apache.geode.internal.size.Sizeable;
+
+public abstract class SizeableList<K> extends LinkedList<K> implements Sizeable {

Review comment:
       Could the generic type for this class be named `E` rather than `K` please? `K` is usually used for keys in maps, whereas `E` is for elements in lists and sets.

##########
File path: geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/collections/SizeableListTest.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.data.collections;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.geode.cache.util.ObjectSizer;
+import org.apache.geode.internal.size.ReflectionObjectSizer;
+import org.apache.geode.redis.internal.data.RedisList;
+
+public class SizeableListTest {
+  private final ObjectSizer sizer = ReflectionObjectSizer.getInstance();
+
+  private int expectedSize(SizeableList list) {
+    return sizer.sizeof(list);
+  }
+
+  private RedisList.ElementList createTestElementList(int size) {

Review comment:
       This method is only used once, so it should probably just be inlined.

##########
File path: geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/RedisListTest.java
##########
@@ -0,0 +1,499 @@
+/*
+ * 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.data;
+
+import static org.apache.geode.redis.internal.data.AbstractRedisData.NO_EXPIRATION;
+import static org.apache.geode.redis.internal.data.NullRedisDataStructures.NULL_REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.data.RedisSet.setOpStoreResult;
+import static org.apache.geode.redis.internal.netty.Coder.stringToBytes;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import it.unimi.dsi.fastutil.bytes.ByteArrays;
+import it.unimi.dsi.fastutil.objects.ObjectOpenCustomHashSet;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.cache.Region;
+import org.apache.geode.internal.HeapDataOutputStream;
+import org.apache.geode.internal.serialization.ByteArrayDataInput;
+import org.apache.geode.internal.serialization.SerializationContext;
+import org.apache.geode.internal.size.ReflectionObjectSizer;
+import org.apache.geode.redis.internal.netty.Coder;
+import org.apache.geode.redis.internal.services.RegionProvider;
+import org.apache.geode.test.junit.rules.ExecutorServiceRule;
+
+public class RedisListTest {

Review comment:
       Every test in this class is testing `RedisSet` rather than `RedisList`. When writing tests, especially unit tests, it's best not to copy/paste code.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableList.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.data.collections;
+
+
+import static org.apache.geode.internal.JvmSizeUtils.getObjectHeaderSize;
+import static org.apache.geode.internal.JvmSizeUtils.getReferenceSize;
+import static org.apache.geode.internal.JvmSizeUtils.memoryOverhead;
+import static org.apache.geode.internal.JvmSizeUtils.roundUpSize;
+
+import java.util.Collection;
+import java.util.LinkedList;
+
+import org.apache.geode.internal.size.Sizeable;
+
+public abstract class SizeableList<K> extends LinkedList<K> implements Sizeable {
+  private static final long serialVersionUID = -8255284217236712060L;
+  private static final int SIZEABLE_LIST_OVERHEAD =
+      memoryOverhead(SizeableList.class);
+  private static final int NODE_OVERHEAD =
+      roundUpSize(getObjectHeaderSize() + 3 * getReferenceSize());
+
+  private int memberOverhead;
+
+  public SizeableList(Collection<K> collection) {
+    for (K element : collection) {
+      add(0, element);
+    }
+  }
+
+  public void add(int index, K k) {
+    super.add(index, k);
+    memberOverhead += sizeElement(k) + NODE_OVERHEAD;
+  }
+
+  public boolean add(K k) {
+    boolean added = super.add(k);
+    if (added) {
+      memberOverhead += sizeElement(k) + NODE_OVERHEAD;
+    }
+    return added;
+  }
+
+  @SuppressWarnings("unchecked")

Review comment:
       This suppression is unnecessary.

##########
File path: geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/collections/SizeableListTest.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.data.collections;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.geode.cache.util.ObjectSizer;
+import org.apache.geode.internal.size.ReflectionObjectSizer;
+import org.apache.geode.redis.internal.data.RedisList;
+
+public class SizeableListTest {
+  private final ObjectSizer sizer = ReflectionObjectSizer.getInstance();
+
+  private int expectedSize(SizeableList list) {
+    return sizer.sizeof(list);
+  }

Review comment:
       Extracting this to its own method doesn't really gain us much in terms of simplification, so I'd prefer to inline it and remove this method.

##########
File path: geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/collections/SizeableListTest.java
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.data.collections;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.geode.cache.util.ObjectSizer;
+import org.apache.geode.internal.size.ReflectionObjectSizer;
+import org.apache.geode.redis.internal.data.RedisList;
+
+public class SizeableListTest {
+  private final ObjectSizer sizer = ReflectionObjectSizer.getInstance();
+
+  private int expectedSize(SizeableList list) {
+    return sizer.sizeof(list);
+  }
+
+  private RedisList.ElementList createTestElementList(int size) {
+    List<byte[]> initialElements = new ArrayList<>(size);
+    for (int i = 0; i < size; ++i) {
+      initialElements.add(new byte[] {(byte) i});
+    }
+    RedisList.ElementList list = new RedisList.ElementList(initialElements);
+    return list;
+  }
+
+  @Test
+  public void getSizeInBytesForEmptyElementList() {
+    RedisList.ElementList list = createTestElementList(0);

Review comment:
       This can be simplified to just `RedisList.ElementList list = new RedisList.ElementList();`




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