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



##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -67,14 +65,19 @@ public int getSizeInBytes() {
     return REDIS_SORTED_SET_OVERHEAD + members.getSizeInBytes() + 
scoreSet.getSizeInBytes();
   }
 
-  RedisSortedSet(List<byte[]> members) {
-    this.members = new MemberMap(members.size() / 2);
+  RedisSortedSet(List<byte[]> members, List<Double> scores) {
+    this.members = new MemberMap(members.size() + scores.size());

Review comment:
       The initial size of `members` should be just `members.size()`.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -91,6 +94,9 @@ protected void applyDelta(DeltaInfo deltaInfo) {
     if (deltaInfo instanceof AddsDeltaInfo) {
       AddsDeltaInfo addsDeltaInfo = (AddsDeltaInfo) deltaInfo;
       membersAddAll(addsDeltaInfo);

Review comment:
       With the changes to this class, `AddsDeltaInfo` should no longer ever be 
used in this class, as it assumes an interleaved list of member names and 
scores, which is no longer valid. This means that this if statement and the 
`membersAddAll()` method can be removed.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -67,14 +65,19 @@ public int getSizeInBytes() {
     return REDIS_SORTED_SET_OVERHEAD + members.getSizeInBytes() + 
scoreSet.getSizeInBytes();
   }
 
-  RedisSortedSet(List<byte[]> members) {
-    this.members = new MemberMap(members.size() / 2);
+  RedisSortedSet(List<byte[]> members, List<Double> scores) {
+    this.members = new MemberMap(members.size() + scores.size());
 
-    Iterator<byte[]> iterator = members.iterator();
+    if (members.size() != scores.size()) {

Review comment:
       This check is already made in the Executors that might call this 
constructor. `ZAddExecutor` and `ZIncryByExecutor` are the only two, I think, 
and both have logic in place to deal with member/score mismatches, so this 
check, and the similar one in `zadd()` can be safely removed.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -163,43 +169,56 @@ public int getDSFID() {
     return REDIS_SORTED_SET_ID;
   }
 
-  protected synchronized byte[] memberAdd(byte[] memberToAdd, byte[] 
scoreToAdd) {
-    OrderedSetEntry existingEntry = members.get(memberToAdd);
-    if (existingEntry == null) {
+  protected synchronized boolean memberAdd(byte[] memberToAdd, double 
scoreToAdd) {
+    OrderedSetEntry entry = members.get(memberToAdd);
+    if (entry == null) {
       OrderedSetEntry newEntry = new OrderedSetEntry(memberToAdd, scoreToAdd);
       members.put(memberToAdd, newEntry);
       scoreSet.add(newEntry);
-      return null;
+      return false;
     } else {
-      scoreSet.remove(existingEntry);
-      byte[] oldScore = existingEntry.scoreBytes;
-      existingEntry.updateScore(stripTrailingZeroFromDouble(scoreToAdd));
-      members.put(memberToAdd, existingEntry);
-      scoreSet.add(existingEntry);
-      return oldScore;
+      if (entry.score == scoreToAdd) {
+        return false;
+      }
+      updateScore(memberToAdd, scoreToAdd, entry);
     }
+    return true;
   }
 
-  synchronized byte[] memberRemove(byte[] member) {
-    byte[] oldValue = null;
+  private void updateScore(byte[] memberToAdd, double scoreToAdd, 
OrderedSetEntry entry) {
+    scoreSet.remove(entry);
+    entry.updateScore(scoreToAdd);
+    members.put(memberToAdd, entry);
+    scoreSet.add(entry);
+  }
+
+  synchronized boolean memberRemove(byte[] member) {
     OrderedSetEntry orderedSetEntry = members.remove(member);
     if (orderedSetEntry != null) {
       scoreSet.remove(orderedSetEntry);
-      oldValue = orderedSetEntry.getScoreBytes();
+      return true;
     }
 
-    return oldValue;
+    return false;
   }
 
   private synchronized void membersAddAll(AddsDeltaInfo addsDeltaInfo) {
     Iterator<byte[]> iterator = addsDeltaInfo.getAdds().iterator();
     while (iterator.hasNext()) {
       byte[] member = iterator.next();
-      byte[] score = iterator.next();
+      double score = Coder.bytesToDouble(iterator.next());
       memberAdd(member, score);
     }
   }
 
+  private synchronized void membersZAddAll(ZaddsDeltaInfo zaddsDeltaInfo) {

Review comment:
       With the removal of the now-defunct `membersAddAll()` method, this 
method could maybe be renamed to "membersAddAll" to keep things simpler, since 
it doesn't make a lot of sense to specify the "Z" part in the sorted set class, 
where it should be implicit.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -265,30 +292,27 @@ long zcount(SortedSetScoreRangeOptions rangeOptions) {
     return maxIndex - minIndex;
   }
 
-  byte[] zincrby(Region<RedisKey, RedisData> region, RedisKey key, byte[] 
increment,
-      byte[] member) {
+  double zincrby(Region<RedisKey, RedisData> region, RedisKey key, double 
incr, byte[] member) {

Review comment:
       For consistency, could either this method return a `byte[]`, or could 
other methods in this class that return a score also return it as `double`? The 
former is probably the most consistent and simplest, given we have range 
commands that return interleaved member and score `byte[]` if "WITHSCORES" is 
specified and it wouldn't be sensible to change those to return `double` rather 
than `byte[]` for the score now.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -213,34 +232,42 @@ private synchronized void membersRemoveAll(RemsDeltaInfo 
remsDeltaInfo) {
    * @return the number of members actually added OR incremented value if INCR 
option specified
    */
   Object zadd(Region<RedisKey, RedisData> region, RedisKey key, List<byte[]> 
membersToAdd,
-      ZAddOptions options) {
+      List<Double> scoresToAdd, ZAddOptions options) {
     if (options.isINCR()) {
-      return zaddIncr(region, key, membersToAdd, options);
+      // if INCR, only one score and member can be added
+      return zaddIncr(region, key, membersToAdd.get(0), scoresToAdd.get(0), 
options);
+    }
+
+    if (membersToAdd.size() != scoresToAdd.size()) {
+      throw new NumberFormatException("there must be the same number of 
members and scores");
     }
-    AddsDeltaInfo deltaInfo = null;
-    Iterator<byte[]> iterator = membersToAdd.iterator();
+
+    ZaddsDeltaInfo deltaInfo = null;
+    Iterator<byte[]> membersIterator = membersToAdd.iterator();
+    Iterator<Double> scoresIterator = scoresToAdd.iterator();
+
     int initialSize = scoreSet.size();
     int changesCount = 0;
-    while (iterator.hasNext()) {
-      byte[] score = iterator.next();
-      byte[] member = iterator.next();
+
+    while (membersIterator.hasNext()) {
+      double score = scoresIterator.next();
+      byte[] member = membersIterator.next();
       if (options.isNX() && members.containsKey(member)) {
         continue;
       }
       if (options.isXX() && !members.containsKey(member)) {
         continue;
       }
-      byte[] oldScore = memberAdd(member, score);
-      if (options.isCH() && oldScore != null
-          && !Arrays.equals(oldScore, stripTrailingZeroFromDouble(score))) {
+      boolean scoreChanged = memberAdd(member, score);
+      if (options.isCH() && scoreChanged) {
         changesCount++;
       }
 
       if (deltaInfo == null) {
-        deltaInfo = new AddsDeltaInfo(new ArrayList<>());
+        deltaInfo = new ZaddsDeltaInfo(member, score);
+      } else {
+        deltaInfo.add(member, score);
       }

Review comment:
       I think this block can be refactored slightly to avoid adding to the 
delta if we didn't actually update the score:
   ```
         if (scoreChanged) {
           if (options.isCH()) {
             changesCount++;
           }
           if (deltaInfo == null) {
             deltaInfo = new ZaddsDeltaInfo(member, score);
           } else {
             deltaInfo.add(member, score);
           }
         }
   ```

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -67,14 +65,19 @@ public int getSizeInBytes() {
     return REDIS_SORTED_SET_OVERHEAD + members.getSizeInBytes() + 
scoreSet.getSizeInBytes();
   }
 
-  RedisSortedSet(List<byte[]> members) {
-    this.members = new MemberMap(members.size() / 2);
+  RedisSortedSet(List<byte[]> members, List<Double> scores) {
+    this.members = new MemberMap(members.size() + scores.size());
 
-    Iterator<byte[]> iterator = members.iterator();
+    if (members.size() != scores.size()) {
+      throw new NumberFormatException("there must be the same number of 
members and scores");
+    }
 
-    while (iterator.hasNext()) {
-      byte[] score = iterator.next();
-      byte[] member = iterator.next();
+    for (int i = 0; i < members.size(); i++) {
+      double score = scores.get(i);
+      if (Double.isNaN(score)) {
+        throw new NumberFormatException(ERROR_NOT_A_VALID_FLOAT);
+      }

Review comment:
       With the changes to the `Coder.bytesToDouble()` method that treats "NaN" 
as an invalid float, it should not be possible to have `score` be NaN at this 
point, so this check can be removed. Sanitization of inputs should ideally be 
done in the Executor classes, I think, and both `ZAddExecutor` and 
`ZIncryByExecutor` (the only two executors that can result in a call to this 
constructor) have checks for invalid score values.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -648,10 +656,9 @@ public int compareScores(double score1, double score2) {
 
     public static final int ORDERED_SET_ENTRY_OVERHEAD = 
memoryOverhead(OrderedSetEntry.class);
 
-    public OrderedSetEntry(byte[] member, byte[] score) {
+    public OrderedSetEntry(byte[] member, double score) {
       this.member = member;
-      this.scoreBytes = stripTrailingZeroFromDouble(score);

Review comment:
       With the removal of this line, the `stripTrailingZeroFromDouble()` 
method is not used anywhere outside of tests, so it can be removed.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/delta/ZaddsDeltaInfo.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.delta;
+
+import static org.apache.geode.redis.internal.delta.DeltaType.ZADDS;
+
+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.internal.InternalDataSerializer;
+
+public class ZaddsDeltaInfo implements DeltaInfo {
+  private final ArrayList<byte[]> deltas = new ArrayList<>();
+  private final ArrayList<Double> scores = new ArrayList<>();
+
+  public ZaddsDeltaInfo() {}
+
+  public ZaddsDeltaInfo(List<byte[]> deltas, List<Double> scores) {
+    this.deltas.addAll(deltas);
+    this.scores.addAll(scores);
+  }
+
+  public ZaddsDeltaInfo(byte[] delta, Double score) {
+    this.deltas.add(delta);
+    this.scores.add(score);
+  }
+
+  public void add(byte[] delta, double score) {
+    deltas.add(delta);
+    this.scores.add(score);

Review comment:
       Inconsistent use of `this` here. It shouldn't be necessary in this 
method.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/delta/ZaddsDeltaInfo.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.delta;
+
+import static org.apache.geode.redis.internal.delta.DeltaType.ZADDS;
+
+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.internal.InternalDataSerializer;
+
+public class ZaddsDeltaInfo implements DeltaInfo {
+  private final ArrayList<byte[]> deltas = new ArrayList<>();
+  private final ArrayList<Double> scores = new ArrayList<>();

Review comment:
       These should be declared as the interface `List<>` rather that the 
concrete class, since we're never calling `DataSerializer.writeArrayList()` 
like in `AddsDeltaInfo`, which is the only reason the field in that class need 
to be an `ArrayList<>`. Doing this allows the fields to be set in the 
constructor directly rather than using `addAll()`.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
##########
@@ -433,18 +457,19 @@ private long 
iteratorRangeRemove(Iterator<AbstractOrderedSetEntry> scoresIterato
     return entriesRemoved;
   }
 
-  private byte[] zaddIncr(Region<RedisKey, RedisData> region, RedisKey key,
-      List<byte[]> membersToAdd, ZAddOptions options) {
+  private byte[] zaddIncr(Region<RedisKey, RedisData> region, RedisKey key, 
byte[] member,
+      double increment, ZAddOptions options) {
+    if (Double.isNaN(increment)) {
+      throw new NumberFormatException(ERROR_NOT_A_VALID_FLOAT);
+    }

Review comment:
       This check should be unnecessary, as the ZAddExecutor already checks for 
NaN increments (assuming the `Coder.bytesToDouble()` method keeps its NaN check 
that throws an Exception).

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/Coder.java
##########
@@ -404,9 +404,14 @@ public static double bytesToDouble(byte[] bytes) {
       return NEGATIVE_INFINITY;
     }
     if (isNaN(bytes)) {
-      return NaN;
+      throw new NumberFormatException();

Review comment:
       Internally, there is no situation in which NaN is ever a valid double 
value for us. If we ever get passed NaN by the user or would get NaN from a 
calculation, then we always return an error and never use the actual "NaN" 
value. As such, it makes sense to just hard-code it into our conversion method 
rather than adding checks for it all over the place, since there's no situation 
in which we would have NaN and be okay with it.

##########
File path: 
geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/collections/SizeableObject2ObjectOpenCustomHashMapWithCursorTest.java
##########
@@ -338,24 +338,22 @@ public void 
getSizeInBytesIsAccurateForOrderedSetEntryValues() {
     for (int i = initialNumberOfElements; i < totalNumberOfElements; ++i) {
       byte[] key = {(byte) i};
       byte[] member = key;
-      byte[] scoreBytes = String.valueOf(totalNumberOfElements - i).getBytes();
-      RedisSortedSet.OrderedSetEntry value = new 
RedisSortedSet.OrderedSetEntry(member, scoreBytes);
+      double score = totalNumberOfElements - i;
+      RedisSortedSet.OrderedSetEntry value = new 
RedisSortedSet.OrderedSetEntry(member, score);
       hash.put(key, value);
       assertThat(hash.getSizeInBytes()).isEqualTo(expectedSize(hash));
     }
 
     // Update values and confirm that size changes as expected
     for (int i = initialNumberOfElements; i < totalNumberOfElements; ++i) {
       byte[] key = {(byte) i};
-      byte[] member = key;
-      byte[] scoreBytes = String.valueOf(i).getBytes();
       RedisSortedSet.OrderedSetEntry value = hash.get(key);
-      byte[] oldScoreBytes = value.getScoreBytes();
-      int scoreDelta = memoryOverhead(scoreBytes)
-          - memoryOverhead(oldScoreBytes);
+      double oldScore = value.getScore();
+      int scoreDelta =
+          memoryOverhead(Coder.doubleToBytes(i)) - 
memoryOverhead(Coder.doubleToBytes(oldScore));
 
       int oldSize = hash.getSizeInBytes();
-      value.updateScore(scoreBytes);
+      value.updateScore(i);
       int sizeDelta = hash.getSizeInBytes() - oldSize;
 
       assertThat(sizeDelta).isEqualTo(scoreDelta);

Review comment:
       The scoreDelta here should be 0, since `double` is fixed size and 
updating its value does not change its size. This section can then be 
refactored to:
   ```
         RedisSortedSet.OrderedSetEntry value = hash.get(key);
   
         int oldSize = hash.getSizeInBytes();
         value.updateScore(i);
         int sizeDelta = hash.getSizeInBytes() - oldSize;
   
         assertThat(sizeDelta).isZero();
   ```

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/delta/ZaddsDeltaInfo.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.delta;
+
+import static org.apache.geode.redis.internal.delta.DeltaType.ZADDS;
+
+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.internal.InternalDataSerializer;
+
+public class ZaddsDeltaInfo implements DeltaInfo {
+  private final ArrayList<byte[]> deltas = new ArrayList<>();
+  private final ArrayList<Double> scores = new ArrayList<>();
+
+  public ZaddsDeltaInfo() {}

Review comment:
       This empty constructor is never used and can be removed, or reworked to 
have similar behaviour to the 0-arg constructor of `RemsDeltaInfo` or the 
`AddsDeltaInfo(int size)` constructor for `AddsDeltaInfo`.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/delta/ZaddsDeltaInfo.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.delta;
+
+import static org.apache.geode.redis.internal.delta.DeltaType.ZADDS;
+
+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.internal.InternalDataSerializer;
+
+public class ZaddsDeltaInfo implements DeltaInfo {
+  private final ArrayList<byte[]> deltas = new ArrayList<>();
+  private final ArrayList<Double> scores = new ArrayList<>();
+
+  public ZaddsDeltaInfo() {}
+
+  public ZaddsDeltaInfo(List<byte[]> deltas, List<Double> scores) {
+    this.deltas.addAll(deltas);
+    this.scores.addAll(scores);
+  }
+
+  public ZaddsDeltaInfo(byte[] delta, Double score) {

Review comment:
       Rather than this being its own method, which basically just reimplements 
`add()` it might be better to do what `AddsDeltaInfo` and `RemsDeltaInfo` do, 
and call a constructor that does not need data, then call `add()`.

##########
File path: 
geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZAddExecutor.java
##########
@@ -89,6 +101,9 @@ private int findAndValidateZAddOptions(Command command, 
Iterator<byte[]> command
         optionsFoundCount++;
       } else if (isInfinity(subCommand)) {
         scoreFound = true;
+      } else if (isNaN(subCommand)) {
+        executorState.exceptionMessage = ERROR_NOT_A_VALID_FLOAT;
+        return 0;

Review comment:
       With the call to `Coder.bytesToDouble()` on line 60, this check 
shouldn't be needed, as the modified `bytesToDouble()` method will now throw an 
exception with the correct message when we get to that line. We would do a 
couple of unnecessary allocations if a user has passed NaN as a score with this 
removed, but we would avoid doing a byte array comparison on every argument in 
the more likely case where they haven't passed NaN, so it's a net performance 
improvement, I think.




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to