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/04/09 00:27:14 UTC

[GitHub] [geode] DonalEvans commented on a diff in pull request #7558: GEODE-10157: Add Radish Delta Unit Tests

DonalEvans commented on code in PR #7558:
URL: https://github.com/apache/geode/pull/7558#discussion_r846533646


##########
geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java:
##########
@@ -178,7 +178,8 @@ public boolean equals(Object o) {
     });
   }
 
-  protected synchronized MemberAddResult memberAdd(byte[] memberToAdd, double scoreToAdd) {
+  @VisibleForTesting
+  public synchronized MemberAddResult memberAdd(byte[] memberToAdd, double scoreToAdd) {

Review Comment:
   We can avoid increasing the visibility of this internal method by making the constructor on `RedisSortedSet` public and changing `AbstractRedisDeltaUnitTest.makeRedisSortedSet()` to :
   ```
     protected RedisSortedSet makeRedisSortedSet() {
       List<byte[]> members = Arrays.asList("alpha".getBytes(),"beta".getBytes(), "gamma".getBytes());
       double[] scores = {1.0d, 2.0d, 4.0d};
       return new RedisSortedSet(members, scores);
     }
   ```



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/ReplaceByteArrayAtOffsetDeltaUnitTest.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.REPLACE_BYTE_ARRAY_AT_OFFSET;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisList;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class ReplaceByteArrayAtOffsetDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final String payload = "something amazing I guess";
+
+  @Test
+  public void testReplaceByteArrayAtOffsetDeltaForRedisString() throws Exception {
+    DataInputStream dis = getDataInputStream(3);
+    String original = "0123456789";
+    RedisString redisString = new RedisString(original.getBytes());
+
+    redisString.fromDelta(dis);
+
+    assertThat(new String(redisString.get())).isEqualTo(original.substring(0, 3) + payload);

Review Comment:
   It might be worth adding another test for this delta with Strings where the payload is small, since we have different logic in the `applyReplaceByteArrayAtOffsetDelta()` method if we're just replacing a subset of the String rather than extending the whole thing. It could look something like this:
   ```
     @Test
     public void testReplaceByteArrayAtOffsetDelta_forRedisString_withSmallByteArray() throws Exception {
       String payload = "small";
       DataInputStream dis = getDataInputStream(3, payload);
       String original = "0123456789";
       RedisString redisString = new RedisString(original.getBytes());
   
       redisString.fromDelta(dis);
   
       assertThat(new String(redisString.get())).isEqualTo("012small89");
     }
   ```
   and would require making `getDataInputStream()` take a String argument to allow the payload to be different in different tests.



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/AddByteArrayDoublePairsDeltaUnitTest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.ADD_BYTE_ARRAY_DOUBLE_PAIRS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisSortedSet;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class AddByteArrayDoublePairsDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final byte[] original = "omega".getBytes();
+
+  @Test
+  public void testAddByteArrayDoublePairsDelta() throws Exception {
+    DataInputStream dis = getDataInputStream();
+    RedisSortedSet redisSortedSet = makeRedisSortedSet();
+    redisSortedSet.fromDelta(dis);
+
+    assertThat(redisSortedSet.zcard()).isEqualTo(4);
+    assertThat(redisSortedSet.zrank(original)).isEqualTo(2L);
+  }
+
+  @Test
+  @Parameters(method = "getDataTypeInstances")
+  @TestCaseName("{method}: redisDataType:{0}")
+  public void unsupportedDataTypesThrowException_forAddByteArrayDoublePairsDelta(
+      RedisData redisData)
+      throws IOException {
+    final DataInputStream dis = getDataInputStream();
+
+    assertThatThrownBy(() -> redisData.fromDelta(dis)).isInstanceOf(
+        IllegalStateException.class)
+        .hasMessageContaining("unexpected " + ADD_BYTE_ARRAY_DOUBLE_PAIRS);
+  }
+
+  private DataInputStream getDataInputStream() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    AddByteArrayDoublePairs source = new AddByteArrayDoublePairs(1);
+    double score = 3.0;
+    source.add(original, score);
+    source.serializeTo(dos);
+
+    return new DataInputStream(new ByteArrayInputStream(baos.toByteArray()));
+  }
+
+  @SuppressWarnings("unused")
+  private Object[] getDataTypeInstances() {

Review Comment:
   This method (and the others named the same in other tests) would be a bit clearer if it were named something like "getUnsupportedDataTypesForDelta" or something like that.



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/AddByteArrayDoublePairsDeltaUnitTest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.ADD_BYTE_ARRAY_DOUBLE_PAIRS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisSortedSet;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class AddByteArrayDoublePairsDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final byte[] original = "omega".getBytes();
+
+  @Test
+  public void testAddByteArrayDoublePairsDelta() throws Exception {
+    DataInputStream dis = getDataInputStream();
+    RedisSortedSet redisSortedSet = makeRedisSortedSet();
+    redisSortedSet.fromDelta(dis);
+
+    assertThat(redisSortedSet.zcard()).isEqualTo(4);
+    assertThat(redisSortedSet.zrank(original)).isEqualTo(2L);

Review Comment:
   The assertions in this test can be made a little more robust by also asserting that the score is correct after the delta is applied. By making the `score` variable on line 69 a field of the class, we can then do:
   ```
       byte[] scoreBytes = redisSortedSet.zscore(original);
       assertThat(Coder.bytesToDouble(scoreBytes)).isEqualTo(score);
   ```



##########
geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSet.java:
##########
@@ -409,7 +409,7 @@ public void fromData(DataInput in) throws IOException, ClassNotFoundException {
   }
 
   @VisibleForTesting
-  synchronized boolean membersAdd(byte[] memberToAdd) {
+  public synchronized boolean membersAdd(byte[] memberToAdd) {

Review Comment:
   We can avoid having to increase the visibility of this method by making `AbstractRedisDeltaUnitTest.makeRedisSet()` use the constructor for `RedisSet` that takes a collection of byte arrays:
   ```
     protected RedisSet makeRedisSet() {
       Set<byte[]> bytes = new HashSet<>();
       bytes.add("zero".getBytes());
       bytes.add("one".getBytes());
       bytes.add("two".getBytes());
       return new RedisSet(bytes);
     }
   ```



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/AddByteArrayPairsDeltaUnitTest.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.ADD_BYTE_ARRAY_PAIRS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisHash;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class AddByteArrayPairsDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final byte[] original = "0123456789".getBytes();
+  private final byte[] payload = "something amazing I guess".getBytes();

Review Comment:
   These fields might be better named "fieldToAdd" and "valueToAdd"



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/AddByteArrayDoublePairsDeltaUnitTest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.ADD_BYTE_ARRAY_DOUBLE_PAIRS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisSortedSet;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class AddByteArrayDoublePairsDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final byte[] original = "omega".getBytes();
+
+  @Test
+  public void testAddByteArrayDoublePairsDelta() throws Exception {

Review Comment:
   For clarity, could all of the tests for data structures that support the delta type specify what data structure the test is for please? So in this case, it would be "testAddByteArrayDoublePairsDelta_forRedisSortedSet".



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/AddByteArrayDoublePairsDeltaUnitTest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.ADD_BYTE_ARRAY_DOUBLE_PAIRS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisSortedSet;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class AddByteArrayDoublePairsDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final byte[] original = "omega".getBytes();
+
+  @Test
+  public void testAddByteArrayDoublePairsDelta() throws Exception {
+    DataInputStream dis = getDataInputStream();
+    RedisSortedSet redisSortedSet = makeRedisSortedSet();
+    redisSortedSet.fromDelta(dis);
+
+    assertThat(redisSortedSet.zcard()).isEqualTo(4);
+    assertThat(redisSortedSet.zrank(original)).isEqualTo(2L);
+  }
+
+  @Test
+  @Parameters(method = "getDataTypeInstances")
+  @TestCaseName("{method}: redisDataType:{0}")
+  public void unsupportedDataTypesThrowException_forAddByteArrayDoublePairsDelta(
+      RedisData redisData)
+      throws IOException {
+    final DataInputStream dis = getDataInputStream();
+
+    assertThatThrownBy(() -> redisData.fromDelta(dis)).isInstanceOf(
+        IllegalStateException.class)
+        .hasMessageContaining("unexpected " + ADD_BYTE_ARRAY_DOUBLE_PAIRS);
+  }
+
+  private DataInputStream getDataInputStream() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    AddByteArrayDoublePairs source = new AddByteArrayDoublePairs(1);
+    double score = 3.0;
+    source.add(original, score);
+    source.serializeTo(dos);
+
+    return new DataInputStream(new ByteArrayInputStream(baos.toByteArray()));
+  }
+
+  @SuppressWarnings("unused")
+  private Object[] getDataTypeInstances() {
+    // Values are original, offset, expected result

Review Comment:
   This comment is incorrect.



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/AddByteArrayDoublePairsDeltaUnitTest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.ADD_BYTE_ARRAY_DOUBLE_PAIRS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisSortedSet;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class AddByteArrayDoublePairsDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final byte[] original = "omega".getBytes();
+
+  @Test
+  public void testAddByteArrayDoublePairsDelta() throws Exception {
+    DataInputStream dis = getDataInputStream();
+    RedisSortedSet redisSortedSet = makeRedisSortedSet();
+    redisSortedSet.fromDelta(dis);
+
+    assertThat(redisSortedSet.zcard()).isEqualTo(4);
+    assertThat(redisSortedSet.zrank(original)).isEqualTo(2L);
+  }
+
+  @Test
+  @Parameters(method = "getDataTypeInstances")
+  @TestCaseName("{method}: redisDataType:{0}")
+  public void unsupportedDataTypesThrowException_forAddByteArrayDoublePairsDelta(

Review Comment:
   The "forAddByteArrayDoublePairsDelta" on this test name is a bit redundant, since we're in the `AddByteArrayDoublePairsDeltaUnitTest` class. I think it should be fine to remove it. I noticed that the other test classes also have similar naming for this method, so those could be simplified too.



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/AddByteArrayDoublePairsDeltaUnitTest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.ADD_BYTE_ARRAY_DOUBLE_PAIRS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisSortedSet;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class AddByteArrayDoublePairsDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final byte[] original = "omega".getBytes();
+
+  @Test
+  public void testAddByteArrayDoublePairsDelta() throws Exception {
+    DataInputStream dis = getDataInputStream();
+    RedisSortedSet redisSortedSet = makeRedisSortedSet();
+    redisSortedSet.fromDelta(dis);
+
+    assertThat(redisSortedSet.zcard()).isEqualTo(4);
+    assertThat(redisSortedSet.zrank(original)).isEqualTo(2L);
+  }
+
+  @Test
+  @Parameters(method = "getDataTypeInstances")
+  @TestCaseName("{method}: redisDataType:{0}")
+  public void unsupportedDataTypesThrowException_forAddByteArrayDoublePairsDelta(
+      RedisData redisData)
+      throws IOException {
+    final DataInputStream dis = getDataInputStream();
+
+    assertThatThrownBy(() -> redisData.fromDelta(dis)).isInstanceOf(
+        IllegalStateException.class)
+        .hasMessageContaining("unexpected " + ADD_BYTE_ARRAY_DOUBLE_PAIRS);

Review Comment:
   This (and other similar assertions in other tests added) can be `.hasMessage()` to make the assertion stronger



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/AddByteArrayDoublePairsDeltaUnitTest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.ADD_BYTE_ARRAY_DOUBLE_PAIRS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisSortedSet;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class AddByteArrayDoublePairsDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final byte[] original = "omega".getBytes();

Review Comment:
   This field seems misnamed, since it's being added to the sorted set. Maybe "memberToAdd" would be better?



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/RemoveByteArraysDeltaUnitTest.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.REMOVE_BYTE_ARRAYS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisHash;
+import org.apache.geode.redis.internal.data.RedisSet;
+import org.apache.geode.redis.internal.data.RedisSortedSet;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class RemoveByteArraysDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  @Test
+  public void testRemoveByteArraysDeltaForRedisSet() throws Exception {
+    RemoveByteArrays source =
+        new RemoveByteArrays(Arrays.asList("zero".getBytes(), "two".getBytes()));
+    DataInputStream dis = getDataInputStream(source);
+    RedisSet redisSet = makeRedisSet();
+
+    redisSet.fromDelta(dis);
+
+    assertThat(redisSet.scard()).isEqualTo(1);
+    assertThat(redisSet.sismember("one".getBytes())).isTrue();
+  }
+
+  @Test
+  public void testRemoveByteArraysDeltaForRedisSortedSet() throws Exception {
+    RemoveByteArrays source =
+        new RemoveByteArrays(Arrays.asList("alpha".getBytes(), "gamma".getBytes()));
+    DataInputStream dis = getDataInputStream(source);
+    RedisSortedSet redisSortedSet = makeRedisSortedSet();
+
+    redisSortedSet.fromDelta(dis);
+
+    assertThat(redisSortedSet.zcard()).isEqualTo(1);
+    assertThat(redisSortedSet.zrank("beta".getBytes())).isEqualTo(0L);
+  }
+
+  @Test
+  public void testRemoveByteArraysDeltaForRedisHash() throws Exception {
+    RemoveByteArrays source =
+        new RemoveByteArrays(Collections.singletonList("zero".getBytes()));

Review Comment:
   This test is a little inconsistent with the other two in this class, since they remove two elements and then confirm that the remaining element is correct, but this removes one element and then confirms that it was removed. It might be best to assert on both the presence and absence of elements in all the tests just to cover our bases.



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/SetTimestampDeltaUnitTest.java:
##########
@@ -23,18 +23,19 @@
 import java.io.DataOutputStream;
 
 import org.junit.Test;
+import org.junit.runner.RunWith;
 
-import org.apache.geode.redis.internal.data.delta.ReplaceByteArrayAtOffset;
-
-public class DeltaClassesJUnitTest {
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
 
+@RunWith(GeodeParamsRunner.class)
+public class SetTimestampDeltaUnitTest {
   @Test
-  public void testReplaceByteArrayAtOffsetForRedisString() throws Exception {
+  public void testSetTimestampDelta() throws Exception {

Review Comment:
   Since it should be possible to set the timestamp on any class that extends `AbstractRedisData`, we should probably have tests for each of them. It's a bit tedious and duplicates some test code, but better safe than sorry.



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/AddByteArrayDoublePairsDeltaUnitTest.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.ADD_BYTE_ARRAY_DOUBLE_PAIRS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisSortedSet;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class AddByteArrayDoublePairsDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final byte[] original = "omega".getBytes();
+
+  @Test
+  public void testAddByteArrayDoublePairsDelta() throws Exception {
+    DataInputStream dis = getDataInputStream();
+    RedisSortedSet redisSortedSet = makeRedisSortedSet();
+    redisSortedSet.fromDelta(dis);
+
+    assertThat(redisSortedSet.zcard()).isEqualTo(4);
+    assertThat(redisSortedSet.zrank(original)).isEqualTo(2L);
+  }
+
+  @Test
+  @Parameters(method = "getDataTypeInstances")
+  @TestCaseName("{method}: redisDataType:{0}")
+  public void unsupportedDataTypesThrowException_forAddByteArrayDoublePairsDelta(
+      RedisData redisData)
+      throws IOException {
+    final DataInputStream dis = getDataInputStream();
+
+    assertThatThrownBy(() -> redisData.fromDelta(dis)).isInstanceOf(
+        IllegalStateException.class)
+        .hasMessageContaining("unexpected " + ADD_BYTE_ARRAY_DOUBLE_PAIRS);
+  }
+
+  private DataInputStream getDataInputStream() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    DataOutputStream dos = new DataOutputStream(baos);
+    AddByteArrayDoublePairs source = new AddByteArrayDoublePairs(1);
+    double score = 3.0;
+    source.add(original, score);
+    source.serializeTo(dos);
+
+    return new DataInputStream(new ByteArrayInputStream(baos.toByteArray()));
+  }
+
+  @SuppressWarnings("unused")
+  private Object[] getDataTypeInstances() {
+    // Values are original, offset, expected result
+    return new Object[] {
+        new Object[] {makeRedisHash()},
+        new Object[] {makeRedisList()},
+        new Object[] {makeRedisSet()},
+        new Object[] {new RedisString()}

Review Comment:
   Is there a reason there isn't a `makeRedisString()` method that we can could call here (and in other similar methods in the other tests)? It would make things more consistent and prevent any possible weird behaviour due to a `RedisString` with a null `value` field.



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/SetTimestampDeltaUnitTest.java:
##########
@@ -23,18 +23,19 @@
 import java.io.DataOutputStream;
 
 import org.junit.Test;
+import org.junit.runner.RunWith;
 
-import org.apache.geode.redis.internal.data.delta.ReplaceByteArrayAtOffset;
-
-public class DeltaClassesJUnitTest {
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
 
+@RunWith(GeodeParamsRunner.class)

Review Comment:
   This test is not parameterised, so this annotation can be removed.



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/SetByteArrayAndTimestampDeltaUnitTest.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.SET_BYTE_ARRAY_AND_TIMESTAMP;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class SetByteArrayAndTimestampDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final String payload = "something amazing I guess";
+
+  @Test
+  public void testSetByteArrayAndTimestampDelta() throws Exception {
+    DataInputStream dis = getDataInputStream();
+    String original = "0123456789";
+    RedisString redisString = new RedisString(original.getBytes());
+
+    redisString.fromDelta(dis);
+
+    assertThat(new String(redisString.get())).isEqualTo(payload);
+    assertThat(redisString.getExpirationTimestamp()).isEqualTo(2);

Review Comment:
   To make it a little clearer where this 2 is coming from, could it be extracted to a field, similar to the payload?



##########
geode-for-redis/src/test/java/org/apache/geode/redis/internal/data/delta/ReplaceByteArrayDoublePairsDeltaUnitTest.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.REPLACE_BYTE_ARRAY_DOUBLE_PAIRS;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import junitparams.Parameters;
+import junitparams.naming.TestCaseName;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.redis.internal.data.RedisData;
+import org.apache.geode.redis.internal.data.RedisSortedSet;
+import org.apache.geode.redis.internal.data.RedisString;
+import org.apache.geode.test.junit.runners.GeodeParamsRunner;
+
+@RunWith(GeodeParamsRunner.class)
+public class ReplaceByteArrayDoublePairsDeltaUnitTest extends AbstractRedisDeltaUnitTest {
+  private final byte[] deltaBytes = "delta".getBytes();
+
+  @Test
+  public void testReplaceByteArrayDoublePairsDelta() throws Exception {
+    DataInputStream dis = getDataInputStream();
+    RedisSortedSet redisSortedSet = makeRedisSortedSet();
+
+    redisSortedSet.fromDelta(dis);
+
+    assertThat(redisSortedSet.zcard()).isEqualTo(1);
+    assertThat(redisSortedSet.zrank(deltaBytes)).isEqualTo(0L);

Review Comment:
   Similar to the test in `AddByteArrayDoublePairsDeltaUnitTest`, we can beef up the assertions here a little by also asserting that the score for the replaced member is correct by making the hard-coded score on line 70 a field of the class and adding:
   ```
       byte[] scoreBytes = redisSortedSet.zscore(deltaBytes);
       assertThat(Coder.bytesToDouble(scoreBytes)).isEqualTo(score);
   ```



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