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 22:25:56 UTC

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

dschneider-pivotal commented on a change in pull request #7261:
URL: https://github.com/apache/geode/pull/7261#discussion_r785181426



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/GeodeRedisService.java
##########
@@ -60,6 +60,7 @@ public void register(DataSerializableFixedIdRegistrar registrar) {
     registrar.register(DataSerializableFixedID.REDIS_KEY, RedisKey.class);
     registrar.register(DataSerializableFixedID.PUBLISH_REQUEST,
         Publisher.PublishRequest.class);
+    registrar.register(DataSerializableFixedID.REDIS_LIST_ID, RedisSet.class);

Review comment:
       This should be RedisList.class instead of RedisSet.class

##########
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) {

Review comment:
       It seems wrong that given a collection we will add that collection in reverse order. I know that is what LPUSH needs but what happens when we implement RPUSH? It seems like it would be cleaner to get rid of these constructors that take a Collection and instead just have a zero-arg constructor. Given that this is a LinkedList we don't lose anything by not telling the constructor an initial size. Once everyone uses a zero-arg constructor then the caller who constructs it will take care of adding its elements in the order it desires. You should probably just have something like addHead and addTail for LPUSH and RPUSH.

##########
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:
       Also we don't want to ship the entire element removed by pop to the secondary. An index sounds good

##########
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);

Review comment:
       this is okay for now but will have a problem in the future when RPUSH is implemented. Given a bunch of byte arrays to add you will need some extra info about the order to add them and/or the index to add them

##########
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()) {

Review comment:
       consider adding equals and hashCode on ElementList. The size and element comparison would be the job of ElementList.equals

##########
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);

Review comment:
       consider having private synchronized methods (like you did with prependElement and appendElement) for all places that will modify "elements". Or maybe these should be methods on ElementList. This class actually exposes a lot of stuff it inherits from LinkedList. It might be better for it to own a LinkedList instead of subclassing. Also consider making ElementList and SizableList a single class.




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