You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@geode.apache.org by GitBox <gi...@apache.org> on 2022/01/15 00:14:48 UTC

[GitHub] [geode] DonalEvans commented on a change in pull request #7228: GEODE-9834: SRANDMEMBER Command Support

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



##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableObjectOpenCustomHashSet.java
##########
@@ -62,6 +62,14 @@ public boolean remove(Object k) {
     return removed;
   }
 
+  public K getKey(final int pos) {
+    return key[pos];
+  }
+
+  public int getMemberMapSize() {

Review comment:
       This method name would be a bit clearer as "getBackingArrayLength()" since it's explicitly not returning the size of the map.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/collections/SizeableObjectOpenCustomHashSet.java
##########
@@ -62,6 +62,14 @@ public boolean remove(Object k) {
     return removed;
   }
 
+  public K getKey(final int pos) {

Review comment:
       This method name would be more accurately "getElementForIndex()" or "getFromBackingArray()" or something similar, since a Set doesn't really have a concept of keys.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SetRandomExecutor.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.commands.executor.set;
+
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToLong;
+import static org.apache.geode.redis.internal.netty.Coder.narrowLongToInt;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.geode.redis.internal.commands.Command;
+import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
+import org.apache.geode.redis.internal.commands.executor.RedisResponse;
+import org.apache.geode.redis.internal.data.RedisKey;
+import org.apache.geode.redis.internal.data.RedisSet;
+import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
+
+public abstract class SetRandomExecutor implements CommandExecutor {
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+    RedisKey key = command.getKey();
+    int argsCount = commandElems.size();

Review comment:
       This method can be simplified/clarified a little bit by introducing a boolean here:
   ```
   boolean hasCount = commandElems.size() == 3;
   ```
   and using it in the if statements on lines 41 and 53.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SetRandomExecutor.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.commands.executor.set;
+
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToLong;
+import static org.apache.geode.redis.internal.netty.Coder.narrowLongToInt;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.geode.redis.internal.commands.Command;
+import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
+import org.apache.geode.redis.internal.commands.executor.RedisResponse;
+import org.apache.geode.redis.internal.data.RedisKey;
+import org.apache.geode.redis.internal.data.RedisSet;
+import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
+
+public abstract class SetRandomExecutor implements CommandExecutor {
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+    RedisKey key = command.getKey();
+    int argsCount = commandElems.size();
+    int count;
+
+    if (argsCount == 3) {
+      try {
+        count = narrowLongToInt(bytesToLong(commandElems.get(2)));
+      } catch (NumberFormatException e) {
+        return RedisResponse.error(ERROR_NOT_INTEGER);
+      }
+    } else {
+      count = 1;
+    }
+
+    List<byte[]> results =
+        context.lockedExecute(key, () -> getResult(count, context.getRegionProvider(), key));
+    if (argsCount == 2) {
+      byte[] byteResult = null;
+      if (!results.isEmpty()) {
+        byteResult = results.get(0);
+      }
+      return RedisResponse.bulkString(byteResult);
+    }
+    return RedisResponse.array(results, true);
+  }
+
+  private List<byte[]> getResult(int count, RegionProvider regionProvider, RedisKey key) {
+    RedisSet set =
+        regionProvider.getTypedRedisData(REDIS_SET, key, true);
+    if (count == 0 || set.scard() == 0) {

Review comment:
       For consistency with other code that's been added recently, this `set.scard() == 0` check might be better as `set == NULL_REDIS_SET`

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/commands/executor/set/SetRandomExecutor.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.commands.executor.set;
+
+import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
+import static org.apache.geode.redis.internal.data.RedisDataType.REDIS_SET;
+import static org.apache.geode.redis.internal.netty.Coder.bytesToLong;
+import static org.apache.geode.redis.internal.netty.Coder.narrowLongToInt;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.geode.redis.internal.commands.Command;
+import org.apache.geode.redis.internal.commands.executor.CommandExecutor;
+import org.apache.geode.redis.internal.commands.executor.RedisResponse;
+import org.apache.geode.redis.internal.data.RedisKey;
+import org.apache.geode.redis.internal.data.RedisSet;
+import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.services.RegionProvider;
+
+public abstract class SetRandomExecutor implements CommandExecutor {
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+    RedisKey key = command.getKey();
+    int argsCount = commandElems.size();
+    int count;
+
+    if (argsCount == 3) {
+      try {
+        count = narrowLongToInt(bytesToLong(commandElems.get(2)));
+      } catch (NumberFormatException e) {
+        return RedisResponse.error(ERROR_NOT_INTEGER);
+      }
+    } else {
+      count = 1;
+    }
+
+    List<byte[]> results =
+        context.lockedExecute(key, () -> getResult(count, context.getRegionProvider(), key));
+    if (argsCount == 2) {
+      byte[] byteResult = null;
+      if (!results.isEmpty()) {
+        byteResult = results.get(0);
+      }
+      return RedisResponse.bulkString(byteResult);
+    }
+    return RedisResponse.array(results, true);

Review comment:
       If the suggestion to use the `hasCount` boolean above is taken, this can be refactored to be a little more easily readable:
   ```
       if (hasCount) {
         return RedisResponse.array(results, true);
       } else {
         if (results.isEmpty()) {
           return RedisResponse.nil();
         } else {
           return RedisResponse.bulkString(results.get(0));
         }
       }
   ```

##########
File path: geode-for-redis/src/integrationTest/java/org/apache/geode/redis/internal/commands/executor/set/AbstractSRandMemberIntegrationTest.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.commands.executor.set;
+
+import static org.apache.geode.redis.RedisCommandArgumentsTestHelper.assertAtLeastNArgs;
+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.RedisConstants.ERROR_WRONG_TYPE;
+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 static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.util.List;
+
+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 org.apache.geode.redis.RedisIntegrationTest;
+
+public abstract class AbstractSRandMemberIntegrationTest implements RedisIntegrationTest {
+  private JedisCluster jedis;
+  private static final String NON_EXISTENT_SET_KEY = "{user1}nonExistentSet";
+  private static final String SET_KEY = "{user1}setKey";
+  private static final String[] SET_MEMBERS = {"one", "two", "three", "four", "five"};
+
+  @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 srandmemberTooFewArgs_returnsError() {
+    assertAtLeastNArgs(jedis, Protocol.Command.SRANDMEMBER, 1);
+  }
+
+  @Test
+  public void srandmemberTooManyArgs_returnsError() {
+    assertThatThrownBy(
+        () -> jedis.sendCommand(SET_KEY, Protocol.Command.SRANDMEMBER, SET_KEY, "5", "5"))
+            .hasMessageContaining(ERROR_SYNTAX);
+  }
+
+  @Test
+  public void srandmemberWithInvalidCount_returnsError() {
+    assertThatThrownBy(() -> jedis.sendCommand(SET_KEY, Protocol.Command.SRANDMEMBER, SET_KEY, "b"))
+        .hasMessageContaining(ERROR_NOT_INTEGER);
+  }
+
+  @Test
+  public void srandmemberWithoutCount_withNonExistentSet_returnsNull() {
+    assertThat(jedis.srandmember(NON_EXISTENT_SET_KEY)).isNull();
+    assertThat(jedis.exists(NON_EXISTENT_SET_KEY)).isFalse();
+  }
+
+  @Test
+  public void srandmemberWithCount_withNonExistentSet_returnsEmptySet() {
+    assertThat(jedis.srandmember(NON_EXISTENT_SET_KEY, 1)).isEmpty();
+    assertThat(jedis.exists(NON_EXISTENT_SET_KEY)).isFalse();
+  }
+
+  @Test
+  public void srandmemberWithoutCount_withExistentSet_returnsOneMember() {
+    jedis.sadd(SET_KEY, SET_MEMBERS);
+
+    String result = jedis.srandmember(SET_KEY);
+    assertThat(SET_MEMBERS).contains(result);

Review comment:
       Strictly speaking, this assertion should be:
   ```
   assertThat(result).isIn(Arrays.asList(SET_MEMBERS));
   ```
   since the thing we're asserting on should be the unknown/variable and the thing we're comparing to should be the expected/fixed thing. In this case it's functionally equivalent, but just for following best practices, changing the assertion would be good.

##########
File path: geode-for-redis/src/main/java/org/apache/geode/redis/internal/data/RedisSet.java
##########
@@ -255,48 +255,61 @@ static int setOpStoreResult(RegionProvider regionProvider, RedisKey destinationK
     return popped;
   }
 
-  public Collection<byte[]> srandmember(int count) {
-    int membersSize = members.size();
-    boolean duplicatesAllowed = count < 0;
-    if (duplicatesAllowed) {
-      count = -count;
+  public List<byte[]> srandmember(int count) {
+    if (count == 0) {
+      return Collections.emptyList();
     }
 
-    if (!duplicatesAllowed && membersSize <= count && count != 1) {
-      return new ArrayList<>(members);
+    List<byte[]> result = new ArrayList<>();
+    int randMethodRatio = 3;
+    if (count < 0) {
+      srandomDuplicateList(-count, result);
+    } else if (count * randMethodRatio < members.size()) {
+      // Count is small enough to add random elements to result
+      srandomUniqueListWithSmallCount(count, result);
+    } else {
+      // Count either equal or greater to member size or close to the member size
+      result.addAll(members);
+
+      // Removes elemnts if count is less than member size
+      if (count < members.size()) {
+        srandomUniqueListWithLargeCount(count, result);
+      }

Review comment:
       It would be better to move the `result.addAll(members);` line and the early return logic of the if statement inside the `srandomUniqueListWithLargeCount()` method, so that method doing the same as `srandomUniqueListWithSmallCount()` in terms of what gets passed in as arguments. Having one of them expect an empty list and one expect a full list is a little counterintuitive.
   
   With this change, srandomUniqueListWithLargeCount() becomes:
   ```
     private void srandomUniqueListWithLargeCount(int count, List<byte[]> result) {
       Random rand = new Random();
       result.addAll(members);
       if (count >= members.size()) {
         return;
       }
       int resultSize;
       while ((resultSize = result.size()) != count) {
         int randIndex = rand.nextInt(resultSize);
         result.remove(randIndex);
       }
     }
   ```
   
   However, it's quite expensive to do repeated array removals, as each one is O(n), so this approach should be tweaked to use a `MemberSet` as the intermediate collection rather than an `ArrayList`, since removing an element from a set is O(1). A more efficient implementation would then be something like:
   ```
     private void srandomUniqueListWithLargeCount(int count, List<byte[]> result) {
       if (count >= members.size()) {
         result.addAll(members);
         return;
       }
       MemberSet duplicateSet = new MemberSet(members);
       Random rand = new Random();
       
       while (duplicateSet.size() != count) {
         int randIndex = rand.nextInt(duplicateSet.getMemberMapSize());
         byte[] member = duplicateSet.getKey(randIndex);
         if (member != null) {
           duplicateSet.remove(member);
         }
       }
       result.addAll(duplicateSet);
     }
   ```
   
   A small side benefit is that the code in this method can then be refactored and used in SPOP, since that's effectively what's being done on `duplicateSet`.




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