You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ri...@apache.org on 2021/07/20 15:51:13 UTC

[geode] branch develop updated: GEODE-9378: Implement ZRANGEBYSCORE (#6700)

This is an automated email from the ASF dual-hosted git repository.

ringles pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 0726d5a  GEODE-9378: Implement ZRANGEBYSCORE (#6700)
0726d5a is described below

commit 0726d5a445d218b3775902d3a4dc4da5308ae4ed
Author: Ray Ingles <ri...@pivotal.io>
AuthorDate: Tue Jul 20 11:50:07 2021 -0400

    GEODE-9378: Implement ZRANGEBYSCORE (#6700)
    
    GEODE-9378: Implements the ZRANGEBYSCORE command, including LIMITS option. Also fixes a formatting difference between Radish and Redis regarding Double values that represent integers.
---
 .../ZRangeByScoreNativeRedisAcceptanceTest.java}   |  32 +-
 .../AbstractZRangeByScoreIntegrationTest.java      | 339 +++++++++++++++++++++
 .../sortedset/ZRangeByScoreIntegrationTest.java}   |  31 +-
 .../geode/redis/internal/RedisCommandType.java     |   2 +
 .../redis/internal/data/NullRedisSortedSet.java    |   5 +
 .../geode/redis/internal/data/RedisSortedSet.java  |  47 ++-
 .../RedisSortedSetCommandsFunctionExecutor.java    |   7 +
 .../executor/sortedset/RedisSortedSetCommands.java |   2 +
 .../executor/sortedset/SortedSetRangeOptions.java  |  60 +++-
 .../executor/sortedset/ZCountExecutor.java         |  38 +--
 .../executor/sortedset/ZRangeByScoreExecutor.java  | 106 +++++++
 .../apache/geode/redis/internal/netty/Coder.java   |  11 +
 .../redis/internal/netty/StringBytesGlossary.java  |   5 +
 .../redis/internal/SupportedCommandsJUnitTest.java |   1 +
 .../redis/internal/data/RedisSortedSetTest.java    |   2 +-
 .../geode/redis/internal/netty/CoderTest.java      |  27 ++
 16 files changed, 626 insertions(+), 89 deletions(-)

diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/SortedSetRangeOptions.java b/geode-apis-compatible-with-redis/src/acceptanceTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreNativeRedisAcceptanceTest.java
similarity index 56%
copy from geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/SortedSetRangeOptions.java
copy to geode-apis-compatible-with-redis/src/acceptanceTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreNativeRedisAcceptanceTest.java
index ee748e1..f859a68 100644
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/SortedSetRangeOptions.java
+++ b/geode-apis-compatible-with-redis/src/acceptanceTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreNativeRedisAcceptanceTest.java
@@ -14,32 +14,22 @@
  */
 package org.apache.geode.redis.internal.executor.sortedset;
 
-public class SortedSetRangeOptions {
-  private final double minDouble;
-  private final boolean minExclusive;
-  private final double maxDouble;
-  private final boolean maxExclusive;
+import org.junit.ClassRule;
 
-  public SortedSetRangeOptions(double min, boolean minExclusive, double max, boolean maxExclusive) {
-    this.minDouble = min;
-    this.minExclusive = minExclusive;
-    this.maxDouble = max;
-    this.maxExclusive = maxExclusive;
-  }
+import org.apache.geode.redis.NativeRedisClusterTestRule;
 
-  public double getMinDouble() {
-    return minDouble;
-  }
+public class ZRangeByScoreNativeRedisAcceptanceTest extends AbstractZRangeByScoreIntegrationTest {
 
-  public boolean isMinExclusive() {
-    return minExclusive;
-  }
+  @ClassRule
+  public static NativeRedisClusterTestRule server = new NativeRedisClusterTestRule();
 
-  public double getMaxDouble() {
-    return maxDouble;
+  @Override
+  public int getPort() {
+    return server.getExposedPorts().get(0);
   }
 
-  public boolean isMaxExclusive() {
-    return maxExclusive;
+  @Override
+  public void flushAll() {
+    server.flushAll();
   }
 }
diff --git a/geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRangeByScoreIntegrationTest.java b/geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRangeByScoreIntegrationTest.java
new file mode 100644
index 0000000..d873e45
--- /dev/null
+++ b/geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/AbstractZRangeByScoreIntegrationTest.java
@@ -0,0 +1,339 @@
+/*
+ * 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.executor.sortedset;
+
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertAtLeastNArgs;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_MIN_MAX_NOT_A_FLOAT;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_SYNTAX;
+import static org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static org.apache.geode.test.dunit.rules.RedisClusterStartupRule.REDIS_CLIENT_TIMEOUT;
+import static org.apache.geode.util.internal.UncheckedUtils.uncheckedCast;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.JedisCluster;
+import redis.clients.jedis.Protocol;
+import redis.clients.jedis.Tuple;
+
+import org.apache.geode.redis.RedisIntegrationTest;
+
+public abstract class AbstractZRangeByScoreIntegrationTest implements RedisIntegrationTest {
+  public static final String KEY = "key";
+  private JedisCluster jedis;
+
+  @Before
+  public void setUp() {
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, getPort()), REDIS_CLIENT_TIMEOUT);
+  }
+
+  @After
+  public void tearDown() {
+    flushAll();
+    jedis.close();
+  }
+
+  @Test
+  public void shouldError_givenWrongNumberOfArguments() {
+    assertAtLeastNArgs(jedis, Protocol.Command.ZRANGEBYSCORE, 3);
+  }
+
+  @Test
+  public void shouldError_givenInvalidMinOrMax() {
+    assertThatThrownBy(() -> jedis.zrangeByScore("fakeKey", "notANumber", "1"))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_FLOAT);
+    assertThatThrownBy(() -> jedis.zrangeByScore("fakeKey", "1", "notANumber"))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_FLOAT);
+    assertThatThrownBy(() -> jedis.zrangeByScore("fakeKey", "notANumber", "notANumber"))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_FLOAT);
+    assertThatThrownBy(() -> jedis.zrangeByScore("fakeKey", "((", "1"))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_FLOAT);
+    assertThatThrownBy(() -> jedis.zrangeByScore("fakeKey", "1", "(("))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_FLOAT);
+    assertThatThrownBy(() -> jedis.zrangeByScore("fakeKey", "(a", "(b"))
+        .hasMessageContaining(ERROR_MIN_MAX_NOT_A_FLOAT);
+  }
+
+  @Test
+  public void shouldReturnZero_givenNonExistentKey() {
+    assertThat(jedis.zrangeByScore("fakeKey", "-inf", "inf")).isEmpty();
+  }
+
+  @Test
+  public void shouldReturnZero_givenMinGreaterThanMax() {
+    jedis.zadd(KEY, 1, "member");
+
+    // Range +inf <= score <= -inf
+    assertThat(jedis.zrangeByScore(KEY, "+inf", "-inf")).isEmpty();
+  }
+
+  @Test
+  public void shouldReturnElement_givenRangeIncludingScore() {
+    jedis.zadd(KEY, 1, "member");
+
+    // Range -inf <= score <= +inf
+    assertThat(jedis.zrangeByScore(KEY, "-inf", "inf"))
+        .containsExactly("member");
+  }
+
+  @Test
+  public void shouldReturnEmptyArray_givenRangeExcludingScore() {
+    int score = 1;
+    jedis.zadd(KEY, score, "member");
+
+    // Range 2 <= score <= 3
+    assertThat(jedis.zrangeByScore(KEY, score + 1, score + 2)).isEmpty();
+  }
+
+  @Test
+  public void shouldReturnRange_givenMinAndMaxEqualToScore() {
+    int score = 1;
+    jedis.zadd(KEY, score, "member");
+
+    // Range 1 <= score <= 1
+    assertThat(jedis.zrangeByScore(KEY, score, score))
+        .containsExactly("member");
+  }
+
+  @Test
+  public void shouldReturnRange_givenMultipleMembersWithDifferentScores() {
+    Map<String, Double> map = new HashMap<>();
+
+    map.put("member1", -10.0);
+    map.put("member2", 1.0);
+    map.put("member3", 10.0);
+
+    jedis.zadd(KEY, map);
+
+    // Range -5 <= score <= 15
+    assertThat(jedis.zrangeByScore(KEY, "-5", "15"))
+        .containsExactly("member2", "member3");
+  }
+
+  @Test
+  public void shouldReturnRange_givenMultipleMembersWithTheSameScoreAndMinAndMaxEqualToScore() {
+    Map<String, Double> map = new HashMap<>();
+    double score = 1;
+    map.put("member1", score);
+    map.put("member2", score);
+    map.put("member3", score);
+
+    jedis.zadd(KEY, map);
+
+    // Range 1 <= score <= 1
+    assertThat(jedis.zrangeByScore(KEY, score, score))
+        .containsExactly("member1", "member2", "member3");
+  }
+
+  @Test
+  public void shouldReturnRange_basicExclusivity() {
+    Map<String, Double> map = new HashMap<>();
+
+    map.put("member0", 0.0);
+    map.put("member1", 1.0);
+    map.put("member2", 2.0);
+    map.put("member3", 3.0);
+    map.put("member4", 4.0);
+
+    jedis.zadd(KEY, map);
+
+    assertThat(jedis.zrangeByScore(KEY, "(1.0", "(3.0"))
+        .containsExactly("member2");
+    assertThat(jedis.zrangeByScore(KEY, "(1.0", "3.0"))
+        .containsExactly("member2", "member3");
+    assertThat(jedis.zrangeByScore(KEY, "1.0", "(3.0"))
+        .containsExactly("member1", "member2");
+  }
+
+  @Test
+  public void shouldReturnRange_givenExclusiveMin() {
+    Map<String, Double> map = getExclusiveTestMap();
+
+    jedis.zadd(KEY, map);
+
+    // Range -inf < score <= +inf
+    assertThat(jedis.zrangeByScore(KEY, "(-inf", "+inf"))
+        .containsExactly("member2", "member3");
+  }
+
+  @Test
+  public void shouldReturnRange_givenExclusiveMax() {
+    Map<String, Double> map = getExclusiveTestMap();
+
+    jedis.zadd(KEY, map);
+
+    // Range -inf <= score < +inf
+    assertThat(jedis.zrangeByScore(KEY, "-inf", "(+inf"))
+        .containsExactly("member1", "member2");
+  }
+
+  @Test
+  public void shouldReturnRange_givenExclusiveMinAndMax() {
+    Map<String, Double> map = getExclusiveTestMap();
+
+    jedis.zadd(KEY, map);
+
+    // Range -inf < score < +inf
+    assertThat(jedis.zrangeByScore(KEY, "(-inf", "(+inf")).containsExactly("member2");
+  }
+
+
+  private Map<String, Double> getExclusiveTestMap() {
+    Map<String, Double> map = new HashMap<>();
+
+    map.put("member1", Double.NEGATIVE_INFINITY);
+    map.put("member2", 1.0);
+    map.put("member3", Double.POSITIVE_INFINITY);
+    return map;
+  }
+
+  @Test
+  public void shouldReturnZero_givenExclusiveMinAndMaxEqualToScore() {
+    double score = 1;
+    jedis.zadd(KEY, score, "member");
+
+    String scoreExclusive = "(" + score;
+    assertThat(jedis.zrangeByScore(KEY, scoreExclusive, scoreExclusive)).isEmpty();
+  }
+
+  @Test
+  // Using only "(" as either the min or the max is equivalent to "(0"
+  public void shouldReturnRange_givenLeftParenOnlyForMinOrMax() {
+    Map<String, Double> map = new HashMap<>();
+
+    map.put("slightlyLessThanZero", -0.01);
+    map.put("zero", 0.0);
+    map.put("slightlyMoreThanZero", 0.01);
+
+    jedis.zadd(KEY, map);
+
+    // Range 0 < score <= inf
+    assertThat(jedis.zrangeByScore(KEY, "(", "inf")).containsExactly("slightlyMoreThanZero");
+
+    // Range -inf <= score < 0
+    assertThat(jedis.zrangeByScore(KEY, "-inf", "(")).containsExactly("slightlyLessThanZero");
+  }
+
+  @Test
+  public void shouldReturnRange_boundedByLimit() {
+    createZSetRangeTestMap();
+
+    assertThat(jedis.zrangeByScore(KEY, "0", "10", 0, 2))
+        .containsExactly("b", "c");
+    assertThat(jedis.zrangeByScore(KEY, "0", "10", 2, 3))
+        .containsExactly("d", "e", "f");
+    assertThat(jedis.zrangeByScore(KEY, "0", "10", 2, 10))
+        .containsExactly("d", "e", "f");
+    assertThat(jedis.zrangeByScore(KEY, "0", "10", 20, 10)).isEmpty();
+  }
+
+  @Test
+  public void shouldReturnRange_withScores_boundedByLimit() {
+    createZSetRangeTestMap();
+
+    Set<Tuple> firstExpected = new LinkedHashSet<>();
+    firstExpected.add(new Tuple("b", 1d));
+    firstExpected.add(new Tuple("c", 2d));
+
+    Set<Tuple> secondExpected = new LinkedHashSet<>();
+    secondExpected.add(new Tuple("d", 3d));
+    secondExpected.add(new Tuple("e", 4d));
+    secondExpected.add(new Tuple("f", 5d));
+
+    assertThat(jedis.zrangeByScoreWithScores(KEY, "0", "10", 0, 0))
+        .isEmpty();
+    assertThat(jedis.zrangeByScoreWithScores(KEY, "0", "10", 0, 2))
+        .containsExactlyElementsOf(firstExpected);
+    assertThat(jedis.zrangeByScoreWithScores(KEY, "0", "10", 2, 3))
+        .containsExactlyElementsOf(secondExpected);
+    assertThat(jedis.zrangeByScoreWithScores(KEY, "0", "10", 2, 10))
+        .containsExactlyElementsOf(secondExpected);
+    assertThat(jedis.zrangeByScoreWithScores(KEY, "0", "10", 2, -1))
+        .containsExactlyElementsOf(secondExpected);
+  }
+
+  @Test
+  public void shouldReturnProperError_givenLimitWithWrongFormat() {
+    createZSetRangeTestMap();
+
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.ZRANGEBYSCORE, KEY, "0", "10", "LIMIT"))
+            .hasMessageContaining(ERROR_SYNTAX);
+    assertThatThrownBy(
+        () -> jedis.sendCommand(KEY, Protocol.Command.ZRANGEBYSCORE, KEY, "0", "10", "LIMIT", "0"))
+            .hasMessageContaining(ERROR_SYNTAX);
+    assertThatThrownBy(() -> jedis.sendCommand(KEY, Protocol.Command.ZRANGEBYSCORE, KEY, "0", "10",
+        "LIMIT", "0", "invalid"))
+            .hasMessageContaining(ERROR_NOT_INTEGER);
+  }
+
+  @Test
+  public void shouldReturnRange_givenMultipleCopiesOfWithscoresAndOrLimit() {
+    createZSetRangeTestMap();
+
+    List<byte[]> expectedWithScores = new ArrayList<>();
+    expectedWithScores.add("b".getBytes());
+    expectedWithScores.add("1".getBytes());
+    expectedWithScores.add("c".getBytes());
+    expectedWithScores.add("2".getBytes());
+
+    List<byte[]> expectedWithoutScores = new ArrayList<>();
+    expectedWithoutScores.add("b".getBytes());
+    expectedWithoutScores.add("c".getBytes());
+
+    List<byte[]> result = uncheckedCast(jedis.sendCommand(KEY, Protocol.Command.ZRANGEBYSCORE, KEY,
+        "0", "10",
+        "LIMIT", "0", "5",
+        "LIMIT", "0", "2"));
+    assertThat(result).containsExactlyElementsOf(expectedWithoutScores);
+    result = uncheckedCast(jedis.sendCommand(KEY, Protocol.Command.ZRANGEBYSCORE, KEY,
+        "1", "2",
+        "WITHSCORES",
+        "WITHSCORES"));
+    assertThat(result).containsExactlyElementsOf(expectedWithScores);
+    result = uncheckedCast(jedis.sendCommand(KEY, Protocol.Command.ZRANGEBYSCORE, KEY,
+        "0", "10",
+        "WITHSCORES",
+        "LIMIT", "0", "5",
+        "LIMIT", "0", "2",
+        "WITHSCORES"));
+    assertThat(result).containsExactlyElementsOf(expectedWithScores);
+  }
+
+  private void createZSetRangeTestMap() {
+    Map<String, Double> map = new HashMap<>();
+
+    map.put("a", Double.NEGATIVE_INFINITY);
+    map.put("b", 1d);
+    map.put("c", 2d);
+    map.put("d", 3d);
+    map.put("e", 4d);
+    map.put("f", 5d);
+    map.put("g", Double.POSITIVE_INFINITY);
+
+    jedis.zadd(KEY, map);
+  }
+}
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/SortedSetRangeOptions.java b/geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreIntegrationTest.java
similarity index 56%
copy from geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/SortedSetRangeOptions.java
copy to geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreIntegrationTest.java
index ee748e1..a48ccec 100644
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/SortedSetRangeOptions.java
+++ b/geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreIntegrationTest.java
@@ -14,32 +14,17 @@
  */
 package org.apache.geode.redis.internal.executor.sortedset;
 
-public class SortedSetRangeOptions {
-  private final double minDouble;
-  private final boolean minExclusive;
-  private final double maxDouble;
-  private final boolean maxExclusive;
+import org.junit.ClassRule;
 
-  public SortedSetRangeOptions(double min, boolean minExclusive, double max, boolean maxExclusive) {
-    this.minDouble = min;
-    this.minExclusive = minExclusive;
-    this.maxDouble = max;
-    this.maxExclusive = maxExclusive;
-  }
+import org.apache.geode.redis.GeodeRedisServerRule;
 
-  public double getMinDouble() {
-    return minDouble;
-  }
+public class ZRangeByScoreIntegrationTest extends AbstractZRangeByScoreIntegrationTest {
 
-  public boolean isMinExclusive() {
-    return minExclusive;
-  }
-
-  public double getMaxDouble() {
-    return maxDouble;
-  }
+  @ClassRule
+  public static GeodeRedisServerRule server = new GeodeRedisServerRule();
 
-  public boolean isMaxExclusive() {
-    return maxExclusive;
+  @Override
+  public int getPort() {
+    return server.getPort();
   }
 }
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
index 38def11..1067b12 100755
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
+++ b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
@@ -100,6 +100,7 @@ import org.apache.geode.redis.internal.executor.sortedset.ZAddExecutor;
 import org.apache.geode.redis.internal.executor.sortedset.ZCardExecutor;
 import org.apache.geode.redis.internal.executor.sortedset.ZCountExecutor;
 import org.apache.geode.redis.internal.executor.sortedset.ZIncrByExecutor;
+import org.apache.geode.redis.internal.executor.sortedset.ZRangeByScoreExecutor;
 import org.apache.geode.redis.internal.executor.sortedset.ZRangeExecutor;
 import org.apache.geode.redis.internal.executor.sortedset.ZRankExecutor;
 import org.apache.geode.redis.internal.executor.sortedset.ZRemExecutor;
@@ -213,6 +214,7 @@ public enum RedisCommandType {
   ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
   ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
       .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
   ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
   ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
   ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/NullRedisSortedSet.java b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/NullRedisSortedSet.java
index cdfc6f7..aa14b07 100644
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/NullRedisSortedSet.java
+++ b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/NullRedisSortedSet.java
@@ -63,6 +63,11 @@ class NullRedisSortedSet extends RedisSortedSet {
   }
 
   @Override
+  List<byte[]> zrangebyscore(SortedSetRangeOptions rangeOptions, boolean withScores) {
+    return Collections.emptyList();
+  }
+
+  @Override
   byte[] zincrby(Region<RedisKey, RedisData> region, RedisKey key, byte[] increment,
       byte[] member) {
     List<byte[]> valuesToAdd = new ArrayList<>();
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
index ce55b12..506c517 100644
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
+++ b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSet.java
@@ -20,6 +20,7 @@ import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_A_VALID_F
 import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SORTED_SET;
 import static org.apache.geode.redis.internal.netty.Coder.bytesToDouble;
 import static org.apache.geode.redis.internal.netty.Coder.doubleToBytes;
+import static org.apache.geode.redis.internal.netty.Coder.stripTrailingZeroFromDouble;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bGREATEST_MEMBER_NAME;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bLEAST_MEMBER_NAME;
 
@@ -28,6 +29,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -243,7 +245,8 @@ public class RedisSortedSet extends AbstractRedisData {
         continue;
       }
       byte[] oldScore = memberAdd(member, score);
-      if (options.isCH() && oldScore != null && !Arrays.equals(oldScore, score)) {
+      if (options.isCH() && oldScore != null
+          && !Arrays.equals(oldScore, stripTrailingZeroFromDouble(score))) {
         changesCount++;
       }
 
@@ -307,6 +310,46 @@ public class RedisSortedSet extends AbstractRedisData {
     return getRange(min, max, withScores, false);
   }
 
+
+  List<byte[]> zrangebyscore(SortedSetRangeOptions rangeOptions, boolean withScores) {
+    List<byte[]> result = new ArrayList<>();
+    AbstractOrderedSetEntry minEntry =
+        new DummyOrderedSetEntry(rangeOptions.getMinDouble(), rangeOptions.isMinExclusive(), true);
+    int minIndex = scoreSet.indexOf(minEntry);
+    if (minIndex >= scoreSet.size()) {
+      return Collections.emptyList();
+    }
+
+    AbstractOrderedSetEntry maxEntry =
+        new DummyOrderedSetEntry(rangeOptions.getMaxDouble(), rangeOptions.isMaxExclusive(), false);
+    int maxIndex = scoreSet.indexOf(maxEntry);
+    if (minIndex == maxIndex) {
+      return Collections.emptyList();
+    }
+
+    // Okay, if we make it this far there's a potential range of things to return.
+    int count = Integer.MAX_VALUE;
+    if (rangeOptions.hasLimit()) {
+      count = rangeOptions.getCount();
+      minIndex += rangeOptions.getOffset();
+      if (minIndex > getSortedSetSize()) {
+        return Collections.emptyList();
+      }
+    }
+    Iterator<AbstractOrderedSetEntry> entryIterator =
+        scoreSet.getIndexRange(minIndex, Math.min(count, maxIndex - minIndex), false);
+
+    while (entryIterator.hasNext()) {
+      AbstractOrderedSetEntry entry = entryIterator.next();
+
+      result.add(entry.member);
+      if (withScores) {
+        result.add(entry.scoreBytes);
+      }
+    }
+    return result;
+  }
+
   long zrank(byte[] member) {
     OrderedSetEntry orderedSetEntry = members.get(member);
     if (orderedSetEntry == null) {
@@ -500,7 +543,7 @@ public class RedisSortedSet extends AbstractRedisData {
 
     AbstractOrderedSetEntry(byte[] member, byte[] score) {
       this.member = member;
-      this.scoreBytes = score;
+      this.scoreBytes = stripTrailingZeroFromDouble(score);
       this.score = processByteArrayAsDouble(score);
     }
 
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSetCommandsFunctionExecutor.java b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSetCommandsFunctionExecutor.java
index 3bb3edc..fa2f210 100644
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSetCommandsFunctionExecutor.java
+++ b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSortedSetCommandsFunctionExecutor.java
@@ -63,6 +63,13 @@ public class RedisSortedSetCommandsFunctionExecutor extends RedisDataCommandsFun
   }
 
   @Override
+  public List<byte[]> zrangebyscore(RedisKey key, SortedSetRangeOptions rangeOptions,
+      boolean withScores) {
+    return stripedExecute(key,
+        () -> getRedisSortedSet(key, true).zrangebyscore(rangeOptions, withScores));
+  }
+
+  @Override
   public long zrank(RedisKey key, byte[] member) {
     return stripedExecute(key, () -> getRedisSortedSet(key, true).zrank(member));
   }
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/RedisSortedSetCommands.java b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/RedisSortedSetCommands.java
index 7165f38..bb336d5 100644
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/RedisSortedSetCommands.java
+++ b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/RedisSortedSetCommands.java
@@ -31,6 +31,8 @@ public interface RedisSortedSetCommands {
 
   List<byte[]> zrange(RedisKey key, int min, int max, boolean withScores);
 
+  List<byte[]> zrangebyscore(RedisKey key, SortedSetRangeOptions rangeOptions, boolean withScores);
+
   long zrank(RedisKey key, byte[] member);
 
   long zrem(RedisKey key, List<byte[]> membersToRemove);
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/SortedSetRangeOptions.java b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/SortedSetRangeOptions.java
index ee748e1..290567f 100644
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/SortedSetRangeOptions.java
+++ b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/SortedSetRangeOptions.java
@@ -14,17 +14,55 @@
  */
 package org.apache.geode.redis.internal.executor.sortedset;
 
+import static org.apache.geode.redis.internal.netty.Coder.bytesToDouble;
+import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bLEFT_PAREN;
+
+import java.util.Arrays;
+
 public class SortedSetRangeOptions {
   private final double minDouble;
   private final boolean minExclusive;
   private final double maxDouble;
   private final boolean maxExclusive;
 
-  public SortedSetRangeOptions(double min, boolean minExclusive, double max, boolean maxExclusive) {
-    this.minDouble = min;
-    this.minExclusive = minExclusive;
-    this.maxDouble = max;
-    this.maxExclusive = maxExclusive;
+  private boolean hasLimit = false;
+  private int offset = 0;
+  private int count = 0;
+
+  public SortedSetRangeOptions(byte[] minBytes, byte[] maxBytes) {
+    if (minBytes[0] == bLEFT_PAREN) {
+      // A value of "(" is equivalent to "(0"
+      if (minBytes.length == 1) {
+        minDouble = 0;
+      } else {
+        minDouble =
+            bytesToDouble(Arrays.copyOfRange(minBytes, 1, minBytes.length));
+      }
+      minExclusive = true;
+    } else {
+      minExclusive = false;
+      minDouble = bytesToDouble(minBytes);
+    }
+
+    if (maxBytes[0] == bLEFT_PAREN) {
+      // A value of "(" is equivalent to "(0"
+      if (maxBytes.length == 1) {
+        maxDouble = 0;
+      } else {
+        maxDouble =
+            bytesToDouble(Arrays.copyOfRange(maxBytes, 1, maxBytes.length));
+      }
+      maxExclusive = true;
+    } else {
+      maxExclusive = false;
+      maxDouble = bytesToDouble(maxBytes);
+    }
+  }
+
+  public void setLimitValues(int offset, int count) {
+    hasLimit = true;
+    this.offset = offset;
+    this.count = count;
   }
 
   public double getMinDouble() {
@@ -42,4 +80,16 @@ public class SortedSetRangeOptions {
   public boolean isMaxExclusive() {
     return maxExclusive;
   }
+
+  public boolean hasLimit() {
+    return hasLimit;
+  }
+
+  public int getOffset() {
+    return offset;
+  }
+
+  public int getCount() {
+    return count;
+  }
 }
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZCountExecutor.java b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZCountExecutor.java
index ca5c10b..0947749 100644
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZCountExecutor.java
+++ b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZCountExecutor.java
@@ -15,10 +15,7 @@
 package org.apache.geode.redis.internal.executor.sortedset;
 
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_MIN_MAX_NOT_A_FLOAT;
-import static org.apache.geode.redis.internal.netty.Coder.bytesToDouble;
-import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bLEFT_PAREN;
 
-import java.util.Arrays;
 import java.util.List;
 
 import org.apache.geode.redis.internal.executor.AbstractExecutor;
@@ -38,7 +35,7 @@ public class ZCountExecutor extends AbstractExecutor {
     try {
       byte[] minBytes = commandElements.get(2);
       byte[] maxBytes = commandElements.get(3);
-      rangeOptions = parseRangeArguments(minBytes, maxBytes);
+      rangeOptions = new SortedSetRangeOptions(minBytes, maxBytes);
     } catch (NumberFormatException ex) {
       return RedisResponse.error(ERROR_MIN_MAX_NOT_A_FLOAT);
     }
@@ -54,37 +51,4 @@ public class ZCountExecutor extends AbstractExecutor {
 
     return RedisResponse.integer(count);
   }
-
-  SortedSetRangeOptions parseRangeArguments(byte[] minBytes, byte[] maxBytes) {
-    boolean minExclusive = false;
-    double minDouble;
-    if (minBytes[0] == bLEFT_PAREN) {
-      // A value of "(" is equivalent to "(0"
-      if (minBytes.length == 1) {
-        minDouble = 0;
-      } else {
-        minDouble =
-            bytesToDouble(Arrays.copyOfRange(minBytes, 1, minBytes.length));
-      }
-      minExclusive = true;
-    } else {
-      minDouble = bytesToDouble(minBytes);
-    }
-
-    boolean maxExclusive = false;
-    double maxDouble;
-    if (maxBytes[0] == bLEFT_PAREN) {
-      // A value of "(" is equivalent to "(0"
-      if (maxBytes.length == 1) {
-        maxDouble = 0;
-      } else {
-        maxDouble =
-            bytesToDouble(Arrays.copyOfRange(maxBytes, 1, maxBytes.length));
-      }
-      maxExclusive = true;
-    } else {
-      maxDouble = bytesToDouble(maxBytes);
-    }
-    return new SortedSetRangeOptions(minDouble, minExclusive, maxDouble, maxExclusive);
-  }
 }
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreExecutor.java b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreExecutor.java
new file mode 100644
index 0000000..5a10d2f
--- /dev/null
+++ b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/sortedset/ZRangeByScoreExecutor.java
@@ -0,0 +1,106 @@
+/*
+ * 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.executor.sortedset;
+
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_MIN_MAX_NOT_A_FLOAT;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_SYNTAX;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToLong;
+import static org.apache.geode.redis.internal.netty.Coder.equalsIgnoreCaseBytes;
+import static org.apache.geode.redis.internal.netty.Coder.narrowLongToInt;
+import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bLIMIT;
+import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bWITHSCORES;
+
+import java.util.List;
+
+import org.apache.geode.redis.internal.executor.AbstractExecutor;
+import org.apache.geode.redis.internal.executor.RedisResponse;
+import org.apache.geode.redis.internal.netty.Command;
+import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+
+public class ZRangeByScoreExecutor extends AbstractExecutor {
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
+    RedisSortedSetCommands redisSortedSetCommands = context.getSortedSetCommands();
+
+    List<byte[]> commandElements = command.getProcessedCommand();
+
+    SortedSetRangeOptions rangeOptions;
+    boolean withScores = false;
+
+    try {
+      byte[] minBytes = commandElements.get(2);
+      byte[] maxBytes = commandElements.get(3);
+      rangeOptions = new SortedSetRangeOptions(minBytes, maxBytes);
+    } catch (NumberFormatException ex) {
+      return RedisResponse.error(ERROR_MIN_MAX_NOT_A_FLOAT);
+    }
+
+    // Native redis allows multiple "withscores" and "limit ? ?" clauses; the last "limit"
+    // clause overrides any previous ones
+    if (commandElements.size() >= 5) {
+      int currentCommandElement = 4;
+
+      while (currentCommandElement < commandElements.size()) {
+        try {
+          if (equalsIgnoreCaseBytes(commandElements.get(currentCommandElement),
+              bWITHSCORES)) {
+            withScores = true;
+            currentCommandElement++;
+          } else {
+            parseLimitArguments(rangeOptions, commandElements, currentCommandElement);
+            currentCommandElement += 3;
+          }
+        } catch (NumberFormatException nfex) {
+          return RedisResponse.error(ERROR_NOT_INTEGER);
+        } catch (IllegalArgumentException iex) {
+          return RedisResponse.error(ERROR_SYNTAX);
+        }
+      }
+    }
+
+    // If the range is empty (min > max or min == max and both are exclusive), or
+    // limit specified but count is zero, return early
+    if ((rangeOptions.hasLimit() && (rangeOptions.getCount() == 0 || rangeOptions.getOffset() < 0))
+        ||
+        rangeOptions.getMinDouble() > rangeOptions.getMaxDouble() ||
+        (rangeOptions.getMinDouble() == rangeOptions.getMaxDouble())
+            && rangeOptions.isMinExclusive() && rangeOptions.isMaxExclusive()) {
+      return RedisResponse.emptyArray();
+    }
+
+    List<byte[]> result =
+        redisSortedSetCommands.zrangebyscore(command.getKey(), rangeOptions, withScores);
+
+    return RedisResponse.array(result);
+  }
+
+  void parseLimitArguments(SortedSetRangeOptions rangeOptions, List<byte[]> commandElements,
+      int commandIndex) {
+    int offset;
+    int count;
+    if (equalsIgnoreCaseBytes(commandElements.get(commandIndex), bLIMIT)
+        && commandElements.size() > commandIndex + 2) {
+      offset = narrowLongToInt(bytesToLong(commandElements.get(commandIndex + 1)));
+      count = narrowLongToInt(bytesToLong(commandElements.get(commandIndex + 2)));
+      if (count < 0) {
+        count = Integer.MAX_VALUE;
+      }
+    } else {
+      throw new IllegalArgumentException();
+    }
+    rangeOptions.setLimitValues(offset, count);
+  }
+}
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/Coder.java b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/Coder.java
index 711e9bc..7997025 100644
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/Coder.java
+++ b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/Coder.java
@@ -22,6 +22,7 @@ import static org.apache.geode.redis.internal.netty.StringBytesGlossary.ARRAY_ID
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.BULK_STRING_ID;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.ERROR_ID;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.INTEGER_ID;
+import static org.apache.geode.redis.internal.netty.StringBytesGlossary.NUMBER_0_BYTE;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.N_INF;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.P_INF;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.SIMPLE_STRING_ID;
@@ -41,6 +42,7 @@ import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bN_INFIN
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bNaN;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bOK;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bOOM;
+import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bPERIOD;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bP_INF;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bP_INFINITY;
 import static org.apache.geode.redis.internal.netty.StringBytesGlossary.bWRONGTYPE;
@@ -49,6 +51,7 @@ import java.io.UnsupportedEncodingException;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.text.DecimalFormat;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 
@@ -471,4 +474,12 @@ public class Coder {
       return (int) toBeNarrowed;
     }
   }
+
+  public static byte[] stripTrailingZeroFromDouble(byte[] doubleBytes) {
+    if (doubleBytes.length > 1 && doubleBytes[doubleBytes.length - 2] == bPERIOD
+        && doubleBytes[doubleBytes.length - 1] == NUMBER_0_BYTE) {
+      return Arrays.copyOfRange(doubleBytes, 0, doubleBytes.length - 2);
+    }
+    return doubleBytes;
+  }
 }
diff --git a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/StringBytesGlossary.java b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/StringBytesGlossary.java
index 5489dbd..2dd772d 100644
--- a/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/StringBytesGlossary.java
+++ b/geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/netty/StringBytesGlossary.java
@@ -179,6 +179,9 @@ public class StringBytesGlossary {
   @MakeImmutable
   public static final byte[] bWITHSCORES = stringToBytes("WITHSCORES");
 
+  @MakeImmutable
+  public static final byte[] bLIMIT = stringToBytes("LIMIT");
+
   // ********** Constants for Double Infinity comparisons **********
   public static final String P_INF = "+inf";
   public static final String INF = "inf";
@@ -226,6 +229,8 @@ public class StringBytesGlossary {
 
   public static final byte bLEFT_PAREN = 40; // (
 
+  public static final byte bPERIOD = 46; // .
+
 
   public static final String PING_RESPONSE = "PONG";
 
diff --git a/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/SupportedCommandsJUnitTest.java b/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/SupportedCommandsJUnitTest.java
index 13721f9..64369e1 100644
--- a/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/SupportedCommandsJUnitTest.java
+++ b/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/SupportedCommandsJUnitTest.java
@@ -92,6 +92,7 @@ public class SupportedCommandsJUnitTest {
       "ZCARD",
       "ZCOUNT",
       "ZRANGE",
+      "ZRANGEBYSCORE",
       "ZRANK",
       "ZREM",
       "ZREVRANGE",
diff --git a/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/data/RedisSortedSetTest.java b/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/data/RedisSortedSetTest.java
index cb3f5db..934dad1 100644
--- a/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/data/RedisSortedSetTest.java
+++ b/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/data/RedisSortedSetTest.java
@@ -373,7 +373,7 @@ public class RedisSortedSetTest {
   public void zrange_shouldAlsoReturnScores_whenWithScoresSpecified() {
     Collection<byte[]> rangeList = rangeSortedSet.zrange(0, 5, true);
     assertThat(rangeList).hasSize(12);
-    assertThat(rangeList).containsExactly("member1".getBytes(), "1.0".getBytes(),
+    assertThat(rangeList).containsExactly("member1".getBytes(), "1".getBytes(),
         "member2".getBytes(), "1.1".getBytes(), "member3".getBytes(), "1.2".getBytes(),
         "member4".getBytes(), "1.3".getBytes(), "member5".getBytes(), "1.4".getBytes(),
         "member6".getBytes(), "1.5".getBytes());
diff --git a/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/netty/CoderTest.java b/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/netty/CoderTest.java
index 2e3f0b2..4211727 100644
--- a/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/netty/CoderTest.java
+++ b/geode-apis-compatible-with-redis/src/test/java/org/apache/geode/redis/internal/netty/CoderTest.java
@@ -24,6 +24,7 @@ import static org.apache.geode.redis.internal.netty.Coder.isNegativeInfinity;
 import static org.apache.geode.redis.internal.netty.Coder.isPositiveInfinity;
 import static org.apache.geode.redis.internal.netty.Coder.narrowLongToInt;
 import static org.apache.geode.redis.internal.netty.Coder.stringToBytes;
+import static org.apache.geode.redis.internal.netty.Coder.stripTrailingZeroFromDouble;
 import static org.apache.geode.redis.internal.netty.Coder.toUpperCaseBytes;
 import static org.assertj.core.api.Assertions.assertThat;
 
@@ -96,6 +97,16 @@ public class CoderTest {
     }
   }
 
+  @Test
+  @Parameters(method = "doubleBytes")
+  public void stripTrailingZeroFromDouble_correctlyStripsTrailingZero(byte[] input,
+      byte[] expected) {
+    byte[] output = stripTrailingZeroFromDouble(input);
+    assertThat(output).containsExactly(expected);
+  }
+
+
+
   @SuppressWarnings("unused")
   private Object[] stringPairs() {
     // string1, string2
@@ -137,6 +148,22 @@ public class CoderTest {
   }
 
   @SuppressWarnings("unused")
+  private Object[] doubleBytes() {
+    // input double bytes, double bytes with stripped trailing zero
+    return new Object[] {
+        new Object[] {stringToBytes("0.0"), stringToBytes("0")},
+        new Object[] {stringToBytes("0.01"), stringToBytes("0.01")},
+        new Object[] {stringToBytes("0"), stringToBytes("0")},
+        new Object[] {stringToBytes("1.0"), stringToBytes("1")},
+        new Object[] {stringToBytes("-1.0"), stringToBytes("-1")},
+        new Object[] {stringToBytes("6.0221409E23"), stringToBytes("6.0221409E23")},
+        new Object[] {stringToBytes("6.62607E-34"), stringToBytes("6.62607E-34")},
+        new Object[] {stringToBytes("Infinity"), stringToBytes("Infinity")},
+        new Object[] {stringToBytes("NaN"), stringToBytes("NaN")},
+    };
+  }
+
+  @SuppressWarnings("unused")
   private Object[] infinityReturnStrings() {
     // string, expectedString
     return new Object[] {