You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2021/10/01 23:21:34 UTC

[GitHub] [geode] DonalEvans commented on a change in pull request #6925: GEODE-9587: Support MSETNX command

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



##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/string/AbstractMSetNXIntegrationTest.java
##########
@@ -62,29 +59,105 @@ public void givenValueNotProvided_returnsWrongNumberOfArgumentsError() {
 
   @Test
   public void givenEvenNumberOfArgumentsProvided_returnsWrongNumberOfArgumentsError() {
-    // Redis returns this message in this scenario: "ERR wrong number of arguments for MSET"
-    assertThatThrownBy(() -> jedis.sendCommand(hashTag, Protocol.Command.MSETNX, "key1" + hashTag,
-        "value1", "key2" + hashTag, "value2", "key3" + hashTag))
-            .hasMessageContaining("ERR wrong number of arguments");
+    // Redis returns this message in this scenario: "ERR wrong number of arguments for MSETNX"
+    assertThatThrownBy(
+        () -> jedis.sendCommand(HASHTAG, Protocol.Command.MSETNX, "key1" + HASHTAG, "value1",
+            "key2" + HASHTAG, "value2", "key3" + HASHTAG))
+                .hasMessageContaining("ERR wrong number of arguments");
+  }
+
+  @Test
+  public void givenDifferentSlots_returnsError() {
+    assertThatThrownBy(
+        () -> jedis.sendCommand("key1", Protocol.Command.MSETNX, "key1", "value1", "key2",
+            "value2"))
+                .hasMessageContaining("CROSSSLOT Keys in request don't hash to the same slot");
+  }
+
+  @Test
+  public void testMSet_clearsExpiration() {
+    jedis.setex("foo", 20L, "bar");
+    jedis.mset("foo", "baz");
+
+    assertThat(jedis.ttl("foo")).isEqualTo(-1);
   }
 
   @Test
   public void testMSetNX() {
-    Set<String> keysAndVals = new HashSet<String>();
-    for (int i = 0; i < 2 * 5; i++) {
-      keysAndVals.add(randString() + hashTag);
+    int KEY_COUNT = 5;

Review comment:
       This can probably be inlined, since it's not used anywhere else in the test.

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/string/AbstractMSetNXIntegrationTest.java
##########
@@ -62,29 +59,105 @@ public void givenValueNotProvided_returnsWrongNumberOfArgumentsError() {
 
   @Test
   public void givenEvenNumberOfArgumentsProvided_returnsWrongNumberOfArgumentsError() {
-    // Redis returns this message in this scenario: "ERR wrong number of arguments for MSET"
-    assertThatThrownBy(() -> jedis.sendCommand(hashTag, Protocol.Command.MSETNX, "key1" + hashTag,
-        "value1", "key2" + hashTag, "value2", "key3" + hashTag))
-            .hasMessageContaining("ERR wrong number of arguments");
+    // Redis returns this message in this scenario: "ERR wrong number of arguments for MSETNX"
+    assertThatThrownBy(
+        () -> jedis.sendCommand(HASHTAG, Protocol.Command.MSETNX, "key1" + HASHTAG, "value1",
+            "key2" + HASHTAG, "value2", "key3" + HASHTAG))
+                .hasMessageContaining("ERR wrong number of arguments");
+  }
+
+  @Test
+  public void givenDifferentSlots_returnsError() {
+    assertThatThrownBy(
+        () -> jedis.sendCommand("key1", Protocol.Command.MSETNX, "key1", "value1", "key2",
+            "value2"))
+                .hasMessageContaining("CROSSSLOT Keys in request don't hash to the same slot");
+  }
+
+  @Test
+  public void testMSet_clearsExpiration() {
+    jedis.setex("foo", 20L, "bar");
+    jedis.mset("foo", "baz");
+
+    assertThat(jedis.ttl("foo")).isEqualTo(-1);
   }
 
   @Test
   public void testMSetNX() {
-    Set<String> keysAndVals = new HashSet<String>();
-    for (int i = 0; i < 2 * 5; i++) {
-      keysAndVals.add(randString() + hashTag);
+    int KEY_COUNT = 5;
+    String[] keys = new String[KEY_COUNT];
+
+    for (int i = 0; i < keys.length; i++) {
+      keys[i] = HASHTAG + "key" + i;
     }
-    String[] keysAndValsArray = keysAndVals.toArray(new String[0]);
-    long response = jedis.msetnx(keysAndValsArray);
+    String[] keysAndValues = makeKeysAndValues(keys, "valueOne");
+
+    long response = jedis.msetnx(keysAndValues);
 
     assertThat(response).isEqualTo(1);

Review comment:
       This could be `assertThat(response).isOne();`

##########
File path: geode-apis-compatible-with-redis/src/distributedTest/java/org/apache/geode/redis/internal/executor/string/MSetnxDUnitTest.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.string;
+
+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.assertj.core.api.Assertions.assertThat;
+
+import java.util.List;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Ignore;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisCluster;
+
+import org.apache.geode.cache.control.RebalanceFactory;
+import org.apache.geode.cache.control.ResourceManager;
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+import org.apache.geode.test.junit.rules.ExecutorServiceRule;
+
+public class MSetnxDUnitTest {
+
+  private static final Logger logger = LogService.getLogger();

Review comment:
       This is never used in this class.

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/string/AbstractMSetNXIntegrationTest.java
##########
@@ -62,29 +59,105 @@ public void givenValueNotProvided_returnsWrongNumberOfArgumentsError() {
 
   @Test
   public void givenEvenNumberOfArgumentsProvided_returnsWrongNumberOfArgumentsError() {
-    // Redis returns this message in this scenario: "ERR wrong number of arguments for MSET"
-    assertThatThrownBy(() -> jedis.sendCommand(hashTag, Protocol.Command.MSETNX, "key1" + hashTag,
-        "value1", "key2" + hashTag, "value2", "key3" + hashTag))
-            .hasMessageContaining("ERR wrong number of arguments");
+    // Redis returns this message in this scenario: "ERR wrong number of arguments for MSETNX"
+    assertThatThrownBy(
+        () -> jedis.sendCommand(HASHTAG, Protocol.Command.MSETNX, "key1" + HASHTAG, "value1",
+            "key2" + HASHTAG, "value2", "key3" + HASHTAG))
+                .hasMessageContaining("ERR wrong number of arguments");
+  }
+
+  @Test
+  public void givenDifferentSlots_returnsError() {
+    assertThatThrownBy(
+        () -> jedis.sendCommand("key1", Protocol.Command.MSETNX, "key1", "value1", "key2",
+            "value2"))
+                .hasMessageContaining("CROSSSLOT Keys in request don't hash to the same slot");
+  }
+
+  @Test
+  public void testMSet_clearsExpiration() {
+    jedis.setex("foo", 20L, "bar");
+    jedis.mset("foo", "baz");
+
+    assertThat(jedis.ttl("foo")).isEqualTo(-1);
   }
 
   @Test
   public void testMSetNX() {

Review comment:
       Could this test name be made a bit more descriptive? It seems like the test is checking that a successful MSETNX returns 1 and an unsuccessful one (due to one or more of the keys already existing) returns 0. It might also be good to add a test case where we try to call MSETNX on a list of keys, only one of which already exists, then confirm that none of the keys were set.

##########
File path: geode-apis-compatible-with-redis/src/distributedTest/java/org/apache/geode/redis/internal/executor/string/MSetnxDUnitTest.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.string;
+
+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.assertj.core.api.Assertions.assertThat;
+
+import java.util.List;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Ignore;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisCluster;
+
+import org.apache.geode.cache.control.RebalanceFactory;
+import org.apache.geode.cache.control.ResourceManager;
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+import org.apache.geode.test.junit.rules.ExecutorServiceRule;
+
+public class MSetnxDUnitTest {
+
+  private static final Logger logger = LogService.getLogger();
+
+  @ClassRule
+  public static RedisClusterStartupRule clusterStartUp = new RedisClusterStartupRule(4);
+
+  @ClassRule
+  public static ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  private static final String HASHTAG = "{tag}";
+  private static JedisCluster jedis;
+  private static MemberVM server1;
+  private static int locatorPort;
+
+  private static MemberVM locator;
+
+  @BeforeClass
+  public static void classSetup() {
+    locator = clusterStartUp.startLocatorVM(0);
+    locatorPort = locator.getPort();
+    server1 = clusterStartUp.startRedisVM(1, locatorPort);
+    clusterStartUp.startRedisVM(2, locatorPort);
+    clusterStartUp.startRedisVM(3, locatorPort);
+
+    int redisServerPort1 = clusterStartUp.getRedisPort(1);
+    // Ensure that buckets are created using a connection with a fairly high timeout since
+    // clusterNodes does not get retried.
+    new Jedis(BIND_ADDRESS, redisServerPort1, REDIS_CLIENT_TIMEOUT).clusterNodes();

Review comment:
       I'm not sure I understand why this is needed. The test runs fine locally for me without this line.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/string/AbstractMSetExecutor.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.string;
+
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_WRONG_SLOT;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.geode.redis.internal.data.RedisKey;
+import org.apache.geode.redis.internal.data.RedisKeyExistsException;
+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 abstract class AbstractMSetExecutor extends AbstractExecutor {
+
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
+
+    List<byte[]> commandElems = command.getCommandArguments();
+    RedisStringCommands stringCommands = context.getStringCommands();
+
+    int numElements = commandElems.size() / 2;
+    List<RedisKey> keys = new ArrayList<>(numElements);
+    List<byte[]> values = new ArrayList<>(numElements);
+
+    RedisKey previousKey = null;
+    for (int i = 0; i < commandElems.size(); i += 2) {
+      RedisKey key = new RedisKey(commandElems.get(i));
+
+      if (previousKey != null && key.getBucketId() != previousKey.getBucketId()) {

Review comment:
       I believe this should be using `key.getSlot()` rather than `key.getBucketId()`.

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/string/AbstractMSetNXIntegrationTest.java
##########
@@ -62,29 +59,105 @@ public void givenValueNotProvided_returnsWrongNumberOfArgumentsError() {
 
   @Test
   public void givenEvenNumberOfArgumentsProvided_returnsWrongNumberOfArgumentsError() {
-    // Redis returns this message in this scenario: "ERR wrong number of arguments for MSET"
-    assertThatThrownBy(() -> jedis.sendCommand(hashTag, Protocol.Command.MSETNX, "key1" + hashTag,
-        "value1", "key2" + hashTag, "value2", "key3" + hashTag))
-            .hasMessageContaining("ERR wrong number of arguments");
+    // Redis returns this message in this scenario: "ERR wrong number of arguments for MSETNX"
+    assertThatThrownBy(
+        () -> jedis.sendCommand(HASHTAG, Protocol.Command.MSETNX, "key1" + HASHTAG, "value1",
+            "key2" + HASHTAG, "value2", "key3" + HASHTAG))
+                .hasMessageContaining("ERR wrong number of arguments");
+  }
+
+  @Test
+  public void givenDifferentSlots_returnsError() {
+    assertThatThrownBy(
+        () -> jedis.sendCommand("key1", Protocol.Command.MSETNX, "key1", "value1", "key2",
+            "value2"))
+                .hasMessageContaining("CROSSSLOT Keys in request don't hash to the same slot");
+  }
+
+  @Test
+  public void testMSet_clearsExpiration() {
+    jedis.setex("foo", 20L, "bar");
+    jedis.mset("foo", "baz");
+
+    assertThat(jedis.ttl("foo")).isEqualTo(-1);
   }
 
   @Test
   public void testMSetNX() {
-    Set<String> keysAndVals = new HashSet<String>();
-    for (int i = 0; i < 2 * 5; i++) {
-      keysAndVals.add(randString() + hashTag);
+    int KEY_COUNT = 5;
+    String[] keys = new String[KEY_COUNT];
+
+    for (int i = 0; i < keys.length; i++) {
+      keys[i] = HASHTAG + "key" + i;
     }
-    String[] keysAndValsArray = keysAndVals.toArray(new String[0]);
-    long response = jedis.msetnx(keysAndValsArray);
+    String[] keysAndValues = makeKeysAndValues(keys, "valueOne");
+
+    long response = jedis.msetnx(keysAndValues);
 
     assertThat(response).isEqualTo(1);
 
-    long response2 = jedis.msetnx(keysAndValsArray[0], randString());
+    long response2 = jedis.msetnx(keysAndValues[0], randString());
 
     assertThat(response2).isEqualTo(0);

Review comment:
       This could be `assertThat(response2).isZero();`

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/string/AbstractMSetNXIntegrationTest.java
##########
@@ -62,29 +59,105 @@ public void givenValueNotProvided_returnsWrongNumberOfArgumentsError() {
 
   @Test
   public void givenEvenNumberOfArgumentsProvided_returnsWrongNumberOfArgumentsError() {
-    // Redis returns this message in this scenario: "ERR wrong number of arguments for MSET"
-    assertThatThrownBy(() -> jedis.sendCommand(hashTag, Protocol.Command.MSETNX, "key1" + hashTag,
-        "value1", "key2" + hashTag, "value2", "key3" + hashTag))
-            .hasMessageContaining("ERR wrong number of arguments");
+    // Redis returns this message in this scenario: "ERR wrong number of arguments for MSETNX"
+    assertThatThrownBy(
+        () -> jedis.sendCommand(HASHTAG, Protocol.Command.MSETNX, "key1" + HASHTAG, "value1",
+            "key2" + HASHTAG, "value2", "key3" + HASHTAG))
+                .hasMessageContaining("ERR wrong number of arguments");
+  }
+
+  @Test
+  public void givenDifferentSlots_returnsError() {
+    assertThatThrownBy(
+        () -> jedis.sendCommand("key1", Protocol.Command.MSETNX, "key1", "value1", "key2",
+            "value2"))
+                .hasMessageContaining("CROSSSLOT Keys in request don't hash to the same slot");
+  }
+
+  @Test
+  public void testMSet_clearsExpiration() {
+    jedis.setex("foo", 20L, "bar");
+    jedis.mset("foo", "baz");
+
+    assertThat(jedis.ttl("foo")).isEqualTo(-1);
   }
 
   @Test
   public void testMSetNX() {
-    Set<String> keysAndVals = new HashSet<String>();
-    for (int i = 0; i < 2 * 5; i++) {
-      keysAndVals.add(randString() + hashTag);
+    int KEY_COUNT = 5;
+    String[] keys = new String[KEY_COUNT];
+
+    for (int i = 0; i < keys.length; i++) {
+      keys[i] = HASHTAG + "key" + i;
     }
-    String[] keysAndValsArray = keysAndVals.toArray(new String[0]);
-    long response = jedis.msetnx(keysAndValsArray);
+    String[] keysAndValues = makeKeysAndValues(keys, "valueOne");
+
+    long response = jedis.msetnx(keysAndValues);
 
     assertThat(response).isEqualTo(1);
 
-    long response2 = jedis.msetnx(keysAndValsArray[0], randString());
+    long response2 = jedis.msetnx(keysAndValues[0], randString());
 
     assertThat(response2).isEqualTo(0);
-    assertThat(keysAndValsArray[1]).isEqualTo(jedis.get(keysAndValsArray[0]));
+    assertThat(keysAndValues[1]).isEqualTo(jedis.get(keysAndValues[0]));
+  }
+
+  @Test
+  public void testMSet_setsKeysAndReturnsCorrectValues() {
+    int keyCount = 5;
+    String[] keyvals = new String[(keyCount * 2)];
+    String[] keys = new String[keyCount];
+    String[] vals = new String[keyCount];
+    for (int i = 0; i < keyCount; i++) {
+      String key = randString() + HASHTAG;
+      String val = randString();
+      keyvals[2 * i] = key;
+      keyvals[2 * i + 1] = val;
+      keys[i] = key;
+      vals[i] = val;
+    }
+
+    long resultString = jedis.msetnx(keyvals);

Review comment:
       This variable should probably be renamed, since it's not a String.

##########
File path: geode-apis-compatible-with-redis/src/distributedTest/java/org/apache/geode/redis/internal/executor/string/MSetnxDUnitTest.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.string;
+
+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.assertj.core.api.Assertions.assertThat;
+
+import java.util.List;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.logging.log4j.Logger;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Ignore;
+import org.junit.Test;
+import redis.clients.jedis.HostAndPort;
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.JedisCluster;
+
+import org.apache.geode.cache.control.RebalanceFactory;
+import org.apache.geode.cache.control.ResourceManager;
+import org.apache.geode.logging.internal.log4j.api.LogService;
+import org.apache.geode.redis.ConcurrentLoopingThreads;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.dunit.rules.RedisClusterStartupRule;
+import org.apache.geode.test.junit.rules.ExecutorServiceRule;
+
+public class MSetnxDUnitTest {
+
+  private static final Logger logger = LogService.getLogger();
+
+  @ClassRule
+  public static RedisClusterStartupRule clusterStartUp = new RedisClusterStartupRule(4);
+
+  @ClassRule
+  public static ExecutorServiceRule executor = new ExecutorServiceRule();
+
+  private static final String HASHTAG = "{tag}";
+  private static JedisCluster jedis;
+  private static MemberVM server1;
+  private static int locatorPort;
+
+  private static MemberVM locator;
+
+  @BeforeClass
+  public static void classSetup() {
+    locator = clusterStartUp.startLocatorVM(0);
+    locatorPort = locator.getPort();
+    server1 = clusterStartUp.startRedisVM(1, locatorPort);
+    clusterStartUp.startRedisVM(2, locatorPort);
+    clusterStartUp.startRedisVM(3, locatorPort);
+
+    int redisServerPort1 = clusterStartUp.getRedisPort(1);
+    // Ensure that buckets are created using a connection with a fairly high timeout since
+    // clusterNodes does not get retried.
+    new Jedis(BIND_ADDRESS, redisServerPort1, REDIS_CLIENT_TIMEOUT).clusterNodes();
+
+    jedis = new JedisCluster(new HostAndPort(BIND_ADDRESS, redisServerPort1), 5000, 20);

Review comment:
       This should be using `REDIS_CLIENT_TIMEOUT` rather than 5000, and unless there is a specific reason to use max attempts of 20, that should probably be removed.




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