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 2021/12/03 21:36:06 UTC

[GitHub] [geode] DonalEvans commented on a change in pull request #7105: GEODE-9772: use delta for all redis modify data ops

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



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/AbstractRedisData.java
##########
@@ -39,13 +45,9 @@
 import org.apache.geode.internal.serialization.KnownVersion;
 import org.apache.geode.internal.serialization.SerializationContext;
 import org.apache.geode.logging.internal.log4j.api.LogService;
-import org.apache.geode.redis.internal.data.delta.AddsDeltaInfo;
-import org.apache.geode.redis.internal.data.delta.AppendDeltaInfo;
 import org.apache.geode.redis.internal.data.delta.DeltaInfo;
 import org.apache.geode.redis.internal.data.delta.DeltaType;
-import org.apache.geode.redis.internal.data.delta.RemsDeltaInfo;
 import org.apache.geode.redis.internal.data.delta.TimestampDeltaInfo;
-import org.apache.geode.redis.internal.data.delta.ZAddsDeltaInfo;
 import org.apache.geode.redis.internal.services.RegionProvider;
 
 public abstract class AbstractRedisData implements RedisData {

Review comment:
       While this class is being modified, there is a logger on line 57 that is never used and should probably be removed.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/delta/HAddsDeltaInfo.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.redis.internal.data.delta;
+
+import static org.apache.geode.redis.internal.data.delta.DeltaType.HADDS;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.internal.InternalDataSerializer;
+
+public class HAddsDeltaInfo implements DeltaInfo {
+  private final ArrayList<byte[]> deltas;

Review comment:
       The type of `deltas` can be changed to the interface `List<Byte[]` rather than the concrete `ArrayList<byte[]>`

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/delta/AddsDeltaInfo.java
##########
@@ -24,14 +24,11 @@
 import java.util.List;
 
 import org.apache.geode.DataSerializer;
+import org.apache.geode.internal.InternalDataSerializer;
 
 public class AddsDeltaInfo implements DeltaInfo {

Review comment:
       Could this class and the Enum constant associated with it be renamed to "SAddDeltaInfo" and "SADD" respectively?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/delta/HAddsDeltaInfo.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.redis.internal.data.delta;
+
+import static org.apache.geode.redis.internal.data.delta.DeltaType.HADDS;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.internal.InternalDataSerializer;
+
+public class HAddsDeltaInfo implements DeltaInfo {
+  private final ArrayList<byte[]> deltas;
+
+  public HAddsDeltaInfo(int size) {
+    this.deltas = new ArrayList<>(size);
+  }
+
+  public HAddsDeltaInfo(byte[] fieldName, byte[] fieldValue) {
+    this(2);
+    add(fieldName);
+    add(fieldValue);

Review comment:
       IntelliJ complains about the use of an overridable method in the constructor here, which can be fixed by either making the class `final`, making the `add()` method final, or replacing these lines with: 
   ```
       deltas.add(fieldName);
       deltas.add(fieldValue);
   ```

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/delta/RemsDeltaInfo.java
##########
@@ -16,14 +16,16 @@
 
 package org.apache.geode.redis.internal.data.delta;
 
-import static org.apache.geode.redis.internal.data.delta.DeltaType.REMS;
+import static org.apache.geode.redis.internal.data.delta.DeltaType.REMOVES;
 
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.internal.InternalDataSerializer;
 
 public class RemsDeltaInfo implements DeltaInfo {

Review comment:
       Could this be renamed to `RemoveDeltaInfo` and the enum associated with it `REMOVE`?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/delta/HAddsDeltaInfo.java
##########
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
+ * agreements. See the NOTICE file distributed with this work for additional information regarding
+ * copyright ownership. The ASF licenses this file to You under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License. You may obtain a
+ * copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License
+ * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
+ * or implied. See the License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.geode.redis.internal.data.delta;
+
+import static org.apache.geode.redis.internal.data.delta.DeltaType.HADDS;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.geode.DataSerializer;
+import org.apache.geode.internal.InternalDataSerializer;
+
+public class HAddsDeltaInfo implements DeltaInfo {

Review comment:
       Could this be `HAddDeltaInfo` instead, and the enum constant associated with it `HADD`?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/delta/RemsDeltaInfo.java
##########
@@ -16,14 +16,16 @@
 
 package org.apache.geode.redis.internal.data.delta;
 
-import static org.apache.geode.redis.internal.data.delta.DeltaType.REMS;
+import static org.apache.geode.redis.internal.data.delta.DeltaType.REMOVES;
 
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.geode.DataSerializer;
+import org.apache.geode.annotations.VisibleForTesting;
+import org.apache.geode.internal.InternalDataSerializer;
 
 public class RemsDeltaInfo implements DeltaInfo {
   private final ArrayList<byte[]> deltas;

Review comment:
       The type of `deltas` can be changed to the interface `List<Byte[]` rather than the concrete `ArrayList<byte[]>`

##########
File path: geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/RedisStringTest.java
##########
@@ -113,6 +113,82 @@ public void appendStoresStableDelta() {
     assertThat(stringOne.hasDelta()).isFalse();
   }
 
+  @Test
+  public void setStoresStableDelta() {
+    Region<RedisKey, RedisData> region = uncheckedCast(mock(Region.class));
+    final byte[] baseBytes = {'0', '1'};
+    final byte[] bytesToSet = {'2', '3'};
+
+    when(region.put(any(), any()))
+        .thenAnswer(invocation -> validateDeltaSerialization(baseBytes, invocation));
+    RedisString stringOne = new RedisString(baseBytes);
+
+    stringOne.set(region, null, bytesToSet, null);
+
+    verify(region).put(any(), any());
+    assertThat(stringOne.hasDelta()).isFalse();
+  }
+
+  @Test
+  public void incrStoresStableDelta() {
+    Region<RedisKey, RedisData> region = uncheckedCast(mock(Region.class));
+    final byte[] baseBytes = {'1'};
+
+    when(region.put(any(), any()))
+        .thenAnswer(invocation -> validateDeltaSerialization(baseBytes, invocation));
+    RedisString stringOne = new RedisString(baseBytes);
+
+    stringOne.incr(region, null);
+
+    verify(region).put(any(), any());
+    assertThat(stringOne.hasDelta()).isFalse();
+  }
+
+  @Test
+  public void incrbyStoresStableDelta() {
+    Region<RedisKey, RedisData> region = uncheckedCast(mock(Region.class));
+    final byte[] baseBytes = {'1'};
+
+    when(region.put(any(), any()))
+        .thenAnswer(invocation -> validateDeltaSerialization(baseBytes, invocation));
+    RedisString stringOne = new RedisString(baseBytes);
+
+    stringOne.incrby(region, null, 3);
+
+    verify(region).put(any(), any());
+    assertThat(stringOne.hasDelta()).isFalse();
+  }
+
+  @Test
+  public void incrbyfloatStoresStableDelta() {
+    Region<RedisKey, RedisData> region = uncheckedCast(mock(Region.class));
+    final byte[] baseBytes = {'1'};
+
+    when(region.put(any(), any()))
+        .thenAnswer(invocation -> validateDeltaSerialization(baseBytes, invocation));
+    RedisString stringOne = new RedisString(baseBytes);
+
+    stringOne.incrbyfloat(region, null, new BigDecimal(3.0));

Review comment:
       IntelliJ has a warning here about a double value being passed into the BigDecimal constructor. While it's not really relevant in this test, it can be fixed by passing in a String instead. The same warning is present on line 246.

##########
File path: geode-for-redis/src/distributedTest/java/org/apache/geode/redis/internal/data/PartitionedRegionStatsUpdateTest.java
##########
@@ -75,115 +75,166 @@ public void should_showIncreaseInDatastoreBytesInUse_givenStringValueSizeIncreas
     String LONG_APPEND_VALUE = String.valueOf(Integer.MAX_VALUE);
     jedis.set(STRING_KEY, "value");
 
-    long initialDataStoreBytesInUse =
+    long initialDataStoreBytesInUse1 =
         clusterStartUpRule.getDataStoreBytesInUseForDataRegion(server1);
 
     for (int i = 0; i < 1000; i++) {
       jedis.append(STRING_KEY, LONG_APPEND_VALUE);
     }
 
-    long finalDataStoreBytesInUse = clusterStartUpRule.getDataStoreBytesInUseForDataRegion(server1);
+    long finalDataStoreBytesInUse1 =
+        clusterStartUpRule.getDataStoreBytesInUseForDataRegion(server1);
 
-    assertThat(finalDataStoreBytesInUse).isGreaterThan(initialDataStoreBytesInUse);
+    assertThat(finalDataStoreBytesInUse1).isGreaterThan(initialDataStoreBytesInUse1);
     long server2finalDataStoreBytesInUse =
         clusterStartUpRule.getDataStoreBytesInUseForDataRegion(server2);
-    assertThat(server2finalDataStoreBytesInUse).isEqualTo(finalDataStoreBytesInUse);
+    assertThat(server2finalDataStoreBytesInUse).isEqualTo(finalDataStoreBytesInUse1);

Review comment:
       Is there also supposed to be an `initialDataStoreBytesInUse2` in this test?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/delta/ZAddsDeltaInfo.java
##########
@@ -24,6 +24,7 @@
 import java.util.List;
 
 import org.apache.geode.DataSerializer;
+import org.apache.geode.internal.InternalDataSerializer;
 
 public class ZAddsDeltaInfo implements DeltaInfo {

Review comment:
       Could this be renamed to `ZAddDeltaInfo` and the enum associated with it to `ZADD`?

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/delta/AddsDeltaInfo.java
##########
@@ -42,10 +39,9 @@ public void add(byte[] delta) {
 
   public void serializeTo(DataOutput out) throws IOException {
     DataSerializer.writeEnum(ADDS, out);
-    DataSerializer.writeArrayList(deltas, out);
-  }
-
-  public List<byte[]> getAdds() {
-    return deltas;
+    InternalDataSerializer.writeArrayLength(deltas.size(), out);
+    for (byte[] bytes : deltas) {
+      DataSerializer.writeByteArray(bytes, out);
+    }

Review comment:
       With this change, the type of `deltas` can be changed to the interface `List<Byte[]` rather than the concrete `ArrayList<byte[]>`




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