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/09/27 13:45:30 UTC

[GitHub] [geode] jdeppe-pivotal opened a new pull request #6907: [WIP] GEODE-9623: Add Radish COMMAND command

jdeppe-pivotal opened a new pull request #6907:
URL: https://github.com/apache/geode/pull/6907


   - Refactor of all parameter requirements and convert to structure that
     can hold meta info about a command.
   - Does not yet contain info about the 'type' or 'flag' associated with
     commands - for example 'write' or 'readonly'.
   
   <!-- Thank you for submitting a contribution to Apache Geode. -->
   
   <!-- In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken: 
   -->
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced in the commit message?
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `develop`)?
   
   - [ ] Is your initial contribution a single, squashed commit?
   
   - [ ] Does `gradlew build` run cleanly?
   
   - [ ] Have you written or updated unit tests to verify your changes?
   
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   
   <!-- Note:
   Please ensure that once the PR is submitted, check Concourse for build issues and
   submit an update to your PR as soon as possible. If you need help, please send an
   email to dev@geode.apache.org.
   -->
   


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



[GitHub] [geode] DonalEvans commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r718039774



##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/server/CommandIntegrationTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.server;
+
+import static org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.api.sync.RedisCommands;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.geode.NativeRedisTestRule;
+import org.apache.geode.redis.GeodeRedisServerRule;
+
+public class CommandIntegrationTest {
+
+  @ClassRule
+  public static NativeRedisTestRule redisServer = new NativeRedisTestRule("redis:6.2.4");
+
+  @ClassRule
+  public static GeodeRedisServerRule radishServer = new GeodeRedisServerRule();
+
+  private RedisCommands<String, String> redisClient;
+  private RedisCommands<String, String> radishClient;
+
+  @Before
+  public void setup() {
+    redisClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, redisServer.getPort()))
+            .connect().sync();
+
+    radishClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, radishServer.getPort()))
+            .connect().sync();
+  }
+
+  @After
+  public void teardown() {}
+
+  @Test
+  public void commandReturnsResultsMatchingNativeRedis() {
+    Map<String, CommandStructure> goldenResults = processRawCommands(redisClient.command());
+    Map<String, CommandStructure> results = processRawCommands(radishClient.command());
+
+    List<String> commands = new ArrayList<>(results.keySet());
+    Collections.sort(commands);
+
+    SoftAssertions softly = new SoftAssertions();
+    for (String command : commands) {
+      softly.assertThatCode(() -> compareCommands(results.get(command), goldenResults.get(command)))
+          .as("command: " + command)
+          .doesNotThrowAnyException();
+    }
+    softly.assertAll();
+  }
+
+  private void compareCommands(CommandStructure actual, CommandStructure expected) {
+    assertThat(actual).as("no metadata for " + expected.name).isNotNull();
+    SoftAssertions softly = new SoftAssertions();
+    softly.assertThat(actual.arity).as(expected.name + ".arity").isEqualTo(expected.arity);
+    softly.assertThat(actual.flags).as(expected.name + ".flags")
+        .containsExactlyInAnyOrderElementsOf(expected.flags);
+    softly.assertThat(actual.firstKey).as(expected.name + ".firstKey").isEqualTo(expected.firstKey);
+    softly.assertThat(actual.lastKey).as(expected.name + ".lastKey").isEqualTo(expected.lastKey);
+    softly.assertThat(actual.stepCount).as(expected.name + ".stepCount")
+        .isEqualTo(expected.stepCount);
+    softly.assertAll();
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<String, CommandStructure> processRawCommands(List<Object> rawCommands) {
+    Map<String, CommandStructure> commands = new HashMap<>();
+
+    for (Object rawEntry : rawCommands) {
+      List<Object> entry = (List<Object>) rawEntry;
+      String key = (String) entry.get(0);
+      List<String> flags = new ArrayList<>();
+      flags.addAll((List<String>) entry.get(2));
+
+      CommandStructure cmd = new CommandStructure(
+          key,
+          (long) entry.get(1),
+          flags,
+          (long) entry.get(3),
+          (long) entry.get(4),
+          (long) entry.get(5));

Review comment:
       This can be simplified slightly to inline the `flags` variable:
   ```
         CommandStructure cmd = new CommandStructure(
             key,
             (long) entry.get(1),
             (List<String>) entry.get(2),
             (long) entry.get(3),
             (long) entry.get(4),
             (long) entry.get(5));
   ```

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/parameters/SpopParameterRequirements.java
##########
@@ -18,18 +18,22 @@
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
 import static org.apache.geode.redis.internal.netty.Coder.bytesToLong;
 
+import java.util.function.BiConsumer;
+
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
 
-public class SpopParameterRequirements implements ParameterRequirements {
-  @Override
-  public void checkParameters(Command command, ExecutionHandlerContext context) {
-    if (command.getProcessedCommand().size() == 3) {
-      try {
-        bytesToLong(command.getProcessedCommand().get(2));
-      } catch (NumberFormatException nex) {
-        throw new RedisParametersMismatchException(ERROR_NOT_INTEGER);
+public class SpopParameterRequirements {

Review comment:
       Possibly out of scope for this PR, but this class probably shouldn't exist, and the check that it's doing to validate arguments being integers should be moved to the SPopExecutor.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/server/CommandExecutor.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.server;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.geode.redis.internal.RedisCommandType;
+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 CommandExecutor extends AbstractExecutor {
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context)
+      throws Exception {

Review comment:
       `Exception` is never thrown from this method.

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/server/CommandIntegrationTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.server;
+
+import static org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.api.sync.RedisCommands;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.geode.NativeRedisTestRule;
+import org.apache.geode.redis.GeodeRedisServerRule;
+
+public class CommandIntegrationTest {
+
+  @ClassRule
+  public static NativeRedisTestRule redisServer = new NativeRedisTestRule("redis:6.2.4");
+
+  @ClassRule
+  public static GeodeRedisServerRule radishServer = new GeodeRedisServerRule();
+
+  private RedisCommands<String, String> redisClient;
+  private RedisCommands<String, String> radishClient;
+
+  @Before
+  public void setup() {
+    redisClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, redisServer.getPort()))
+            .connect().sync();
+
+    radishClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, radishServer.getPort()))
+            .connect().sync();
+  }
+
+  @After
+  public void teardown() {}

Review comment:
       This can probably be removed.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/parameters/Parameter.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.parameters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.BiConsumer;
+
+import org.apache.geode.redis.internal.RedisCommandType;
+import org.apache.geode.redis.internal.netty.Command;
+import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+
+public class Parameter {
+
+  private int arity;
+  private List<RedisCommandType.Flag> flags = new ArrayList<>();
+  private int firstKey = 1;
+  private int lastKey = 1;
+  private int step = 1;
+  private List<BiConsumer<Command, ExecutionHandlerContext>> predicates = new ArrayList<>();

Review comment:
       None of the predicates use an `ExecutionHandlerContext`, so this could be changed to be just a `Consumer<Command>`.




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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r718697265



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),

Review comment:
       That's a good idea, but it also seems that some of the deferred params use `odd()` and `custom()` which complicate things, so I'd prefer to leave it as it is unless you have some other suggestions.




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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r718697556



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/parameters/SpopParameterRequirements.java
##########
@@ -18,18 +18,22 @@
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
 import static org.apache.geode.redis.internal.netty.Coder.bytesToLong;
 
+import java.util.function.BiConsumer;
+
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
 
-public class SpopParameterRequirements implements ParameterRequirements {
-  @Override
-  public void checkParameters(Command command, ExecutionHandlerContext context) {
-    if (command.getProcessedCommand().size() == 3) {
-      try {
-        bytesToLong(command.getProcessedCommand().get(2));
-      } catch (NumberFormatException nex) {
-        throw new RedisParametersMismatchException(ERROR_NOT_INTEGER);
+public class SpopParameterRequirements {

Review comment:
       Good point - I've made that change.




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



[GitHub] [geode] jdeppe-pivotal merged pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal merged pull request #6907:
URL: https://github.com/apache/geode/pull/6907


   


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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r718693433



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  SETBIT(new SetBitExecutor(), UNSUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
 
   /**************** Sets *****************/
 
-  SCARD(new SCardExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
-  SDIFF(new SDiffExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SINTER(new SInterExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
-  SMOVE(new SMoveExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
-  SPOP(new SPopExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX)).and(new SpopParameterRequirements())),
-  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SSCAN(new SScanExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  SUNION(new SUnionExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
+  SCARD(new SCardExecutor(), UNSUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  SDIFF(new SDiffExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SINTER(new SInterExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  SMOVE(new SMoveExecutor(), UNSUPPORTED, new Parameter().exact(4).lastKey(2).flags(write, fast)),
+  SPOP(new SPopExecutor(), UNSUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX)
+      .custom(SpopParameterRequirements.checkParameters()).flags(write, random, fast)),
+  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED,
+      new Parameter().min(2).flags(readonly, random)),
+  SSCAN(new SScanExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX).firstKey(0)),
+  SUNION(new SUnionExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
 
   /*************** Server ****************/
 
-  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
-  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  TIME(new TimeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
+  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new Parameter().exact(1).firstKey(0).flags(readonly, fast)),
+  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(admin, noscript, loading,
+          stale)),
+  TIME(new TimeExecutor(), UNSUPPORTED,
+      new Parameter().exact(1).firstKey(0).flags(random, loading, stale, fast)),
 
   /***************************************
    *********** Unknown Commands **********
    ***************************************/
   UNKNOWN(new UnknownExecutor(), RedisCommandSupportLevel.UNKNOWN);
 
+  public enum Flag {
+    admin,

Review comment:
       I've made them all uppercase now. The reason some use underscores is to match the flags in redis - https://redis.io/commands/command and https://redis.io/topics/modules-api-ref

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  SETBIT(new SetBitExecutor(), UNSUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
 
   /**************** Sets *****************/
 
-  SCARD(new SCardExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
-  SDIFF(new SDiffExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SINTER(new SInterExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
-  SMOVE(new SMoveExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
-  SPOP(new SPopExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX)).and(new SpopParameterRequirements())),
-  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SSCAN(new SScanExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  SUNION(new SUnionExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
+  SCARD(new SCardExecutor(), UNSUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  SDIFF(new SDiffExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SINTER(new SInterExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  SMOVE(new SMoveExecutor(), UNSUPPORTED, new Parameter().exact(4).lastKey(2).flags(write, fast)),
+  SPOP(new SPopExecutor(), UNSUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX)
+      .custom(SpopParameterRequirements.checkParameters()).flags(write, random, fast)),
+  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED,
+      new Parameter().min(2).flags(readonly, random)),
+  SSCAN(new SScanExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX).firstKey(0)),
+  SUNION(new SUnionExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
 
   /*************** Server ****************/
 
-  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
-  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  TIME(new TimeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
+  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new Parameter().exact(1).firstKey(0).flags(readonly, fast)),
+  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(admin, noscript, loading,
+          stale)),
+  TIME(new TimeExecutor(), UNSUPPORTED,
+      new Parameter().exact(1).firstKey(0).flags(random, loading, stale, fast)),
 
   /***************************************
    *********** Unknown Commands **********
    ***************************************/
   UNKNOWN(new UnknownExecutor(), RedisCommandSupportLevel.UNKNOWN);
 
+  public enum Flag {
+    admin,
+    denyoom,
+    fast,
+    loading,
+    may_replicate,
+    movablekeys,
+    no_auth,
+    noscript,
+    pubsub,
+    random,
+    readonly,
+    sort_for_script,
+    stale,
+    write;
+  }
+
   private final Executor executor;
-  private final ParameterRequirements parameterRequirements;
-  private final ParameterRequirements deferredParameterRequirements;
+  private final Parameter parameterRequirements;
+  private final Parameter deferredParameterRequirements;
   private final RedisCommandSupportLevel supportLevel;
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel) {
-    this(executor, supportLevel, new UnspecifiedParameterRequirements());
+    this(executor, supportLevel, new Parameter().custom((c, e) -> {
+    }));
   }
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel,
-      ParameterRequirements parameterRequirements) {
-    this(executor, supportLevel, parameterRequirements, new UnspecifiedParameterRequirements());
+      Parameter parameterRequirements) {
+    this(executor, supportLevel, parameterRequirements, new Parameter().custom((c, e) -> {
+    }));
   }
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel,
-      ParameterRequirements parameterRequirements,
-      ParameterRequirements deferredParameterRequirements) {
+      Parameter parameterRequirements,
+      Parameter deferredParameterRequirements) {
     this.executor = executor;
     this.supportLevel = supportLevel;
     this.parameterRequirements = parameterRequirements;
     this.deferredParameterRequirements = deferredParameterRequirements;
   }
 
+  public int arity() {
+    return parameterRequirements.getArity();
+  }
+
+  public List<Flag> flags() {

Review comment:
       Cool - haven't used that before.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),

Review comment:
       That's a good idea, but it also seems that some of the deferred params use `odd()` and `custom()` which complicate things, so I'd prefer to leave it as it is unless you have some other suggestions.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/parameters/SpopParameterRequirements.java
##########
@@ -18,18 +18,22 @@
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
 import static org.apache.geode.redis.internal.netty.Coder.bytesToLong;
 
+import java.util.function.BiConsumer;
+
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
 
-public class SpopParameterRequirements implements ParameterRequirements {
-  @Override
-  public void checkParameters(Command command, ExecutionHandlerContext context) {
-    if (command.getProcessedCommand().size() == 3) {
-      try {
-        bytesToLong(command.getProcessedCommand().get(2));
-      } catch (NumberFormatException nex) {
-        throw new RedisParametersMismatchException(ERROR_NOT_INTEGER);
+public class SpopParameterRequirements {

Review comment:
       Good point - I've made that change.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/server/CommandExecutor.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.server;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.geode.redis.internal.RedisCommandType;
+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 CommandExecutor extends AbstractExecutor {
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context)
+      throws Exception {

Review comment:
       Removed

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/server/CommandIntegrationTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.server;
+
+import static org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.api.sync.RedisCommands;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.geode.NativeRedisTestRule;
+import org.apache.geode.redis.GeodeRedisServerRule;
+
+public class CommandIntegrationTest {
+
+  @ClassRule
+  public static NativeRedisTestRule redisServer = new NativeRedisTestRule("redis:6.2.4");
+
+  @ClassRule
+  public static GeodeRedisServerRule radishServer = new GeodeRedisServerRule();
+
+  private RedisCommands<String, String> redisClient;
+  private RedisCommands<String, String> radishClient;
+
+  @Before
+  public void setup() {
+    redisClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, redisServer.getPort()))
+            .connect().sync();
+
+    radishClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, radishServer.getPort()))
+            .connect().sync();
+  }
+
+  @After
+  public void teardown() {}

Review comment:
       Done

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/server/CommandIntegrationTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.server;
+
+import static org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.api.sync.RedisCommands;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.geode.NativeRedisTestRule;
+import org.apache.geode.redis.GeodeRedisServerRule;
+
+public class CommandIntegrationTest {
+
+  @ClassRule
+  public static NativeRedisTestRule redisServer = new NativeRedisTestRule("redis:6.2.4");
+
+  @ClassRule
+  public static GeodeRedisServerRule radishServer = new GeodeRedisServerRule();
+
+  private RedisCommands<String, String> redisClient;
+  private RedisCommands<String, String> radishClient;
+
+  @Before
+  public void setup() {
+    redisClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, redisServer.getPort()))
+            .connect().sync();
+
+    radishClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, radishServer.getPort()))
+            .connect().sync();
+  }
+
+  @After
+  public void teardown() {}
+
+  @Test
+  public void commandReturnsResultsMatchingNativeRedis() {
+    Map<String, CommandStructure> goldenResults = processRawCommands(redisClient.command());
+    Map<String, CommandStructure> results = processRawCommands(radishClient.command());
+
+    List<String> commands = new ArrayList<>(results.keySet());
+    Collections.sort(commands);
+
+    SoftAssertions softly = new SoftAssertions();
+    for (String command : commands) {
+      softly.assertThatCode(() -> compareCommands(results.get(command), goldenResults.get(command)))
+          .as("command: " + command)
+          .doesNotThrowAnyException();
+    }
+    softly.assertAll();
+  }
+
+  private void compareCommands(CommandStructure actual, CommandStructure expected) {
+    assertThat(actual).as("no metadata for " + expected.name).isNotNull();
+    SoftAssertions softly = new SoftAssertions();
+    softly.assertThat(actual.arity).as(expected.name + ".arity").isEqualTo(expected.arity);
+    softly.assertThat(actual.flags).as(expected.name + ".flags")
+        .containsExactlyInAnyOrderElementsOf(expected.flags);
+    softly.assertThat(actual.firstKey).as(expected.name + ".firstKey").isEqualTo(expected.firstKey);
+    softly.assertThat(actual.lastKey).as(expected.name + ".lastKey").isEqualTo(expected.lastKey);
+    softly.assertThat(actual.stepCount).as(expected.name + ".stepCount")
+        .isEqualTo(expected.stepCount);
+    softly.assertAll();
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<String, CommandStructure> processRawCommands(List<Object> rawCommands) {
+    Map<String, CommandStructure> commands = new HashMap<>();
+
+    for (Object rawEntry : rawCommands) {
+      List<Object> entry = (List<Object>) rawEntry;
+      String key = (String) entry.get(0);
+      List<String> flags = new ArrayList<>();
+      flags.addAll((List<String>) entry.get(2));
+
+      CommandStructure cmd = new CommandStructure(
+          key,
+          (long) entry.get(1),
+          flags,
+          (long) entry.get(3),
+          (long) entry.get(4),
+          (long) entry.get(5));

Review comment:
       Done

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/parameters/Parameter.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.parameters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.BiConsumer;
+
+import org.apache.geode.redis.internal.RedisCommandType;
+import org.apache.geode.redis.internal.netty.Command;
+import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+
+public class Parameter {
+
+  private int arity;
+  private List<RedisCommandType.Flag> flags = new ArrayList<>();
+  private int firstKey = 1;
+  private int lastKey = 1;
+  private int step = 1;
+  private List<BiConsumer<Command, ExecutionHandlerContext>> predicates = new ArrayList<>();

Review comment:
       Yes, good observation! That cleans things up nicely.




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



[GitHub] [geode] jdeppe-pivotal merged pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal merged pull request #6907:
URL: https://github.com/apache/geode/pull/6907


   


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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r718697657



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/server/CommandExecutor.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.server;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.geode.redis.internal.RedisCommandType;
+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 CommandExecutor extends AbstractExecutor {
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context)
+      throws Exception {

Review comment:
       Removed

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/server/CommandIntegrationTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.server;
+
+import static org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.api.sync.RedisCommands;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.geode.NativeRedisTestRule;
+import org.apache.geode.redis.GeodeRedisServerRule;
+
+public class CommandIntegrationTest {
+
+  @ClassRule
+  public static NativeRedisTestRule redisServer = new NativeRedisTestRule("redis:6.2.4");
+
+  @ClassRule
+  public static GeodeRedisServerRule radishServer = new GeodeRedisServerRule();
+
+  private RedisCommands<String, String> redisClient;
+  private RedisCommands<String, String> radishClient;
+
+  @Before
+  public void setup() {
+    redisClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, redisServer.getPort()))
+            .connect().sync();
+
+    radishClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, radishServer.getPort()))
+            .connect().sync();
+  }
+
+  @After
+  public void teardown() {}

Review comment:
       Done

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/server/CommandIntegrationTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.server;
+
+import static org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.api.sync.RedisCommands;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.geode.NativeRedisTestRule;
+import org.apache.geode.redis.GeodeRedisServerRule;
+
+public class CommandIntegrationTest {
+
+  @ClassRule
+  public static NativeRedisTestRule redisServer = new NativeRedisTestRule("redis:6.2.4");
+
+  @ClassRule
+  public static GeodeRedisServerRule radishServer = new GeodeRedisServerRule();
+
+  private RedisCommands<String, String> redisClient;
+  private RedisCommands<String, String> radishClient;
+
+  @Before
+  public void setup() {
+    redisClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, redisServer.getPort()))
+            .connect().sync();
+
+    radishClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, radishServer.getPort()))
+            .connect().sync();
+  }
+
+  @After
+  public void teardown() {}
+
+  @Test
+  public void commandReturnsResultsMatchingNativeRedis() {
+    Map<String, CommandStructure> goldenResults = processRawCommands(redisClient.command());
+    Map<String, CommandStructure> results = processRawCommands(radishClient.command());
+
+    List<String> commands = new ArrayList<>(results.keySet());
+    Collections.sort(commands);
+
+    SoftAssertions softly = new SoftAssertions();
+    for (String command : commands) {
+      softly.assertThatCode(() -> compareCommands(results.get(command), goldenResults.get(command)))
+          .as("command: " + command)
+          .doesNotThrowAnyException();
+    }
+    softly.assertAll();
+  }
+
+  private void compareCommands(CommandStructure actual, CommandStructure expected) {
+    assertThat(actual).as("no metadata for " + expected.name).isNotNull();
+    SoftAssertions softly = new SoftAssertions();
+    softly.assertThat(actual.arity).as(expected.name + ".arity").isEqualTo(expected.arity);
+    softly.assertThat(actual.flags).as(expected.name + ".flags")
+        .containsExactlyInAnyOrderElementsOf(expected.flags);
+    softly.assertThat(actual.firstKey).as(expected.name + ".firstKey").isEqualTo(expected.firstKey);
+    softly.assertThat(actual.lastKey).as(expected.name + ".lastKey").isEqualTo(expected.lastKey);
+    softly.assertThat(actual.stepCount).as(expected.name + ".stepCount")
+        .isEqualTo(expected.stepCount);
+    softly.assertAll();
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<String, CommandStructure> processRawCommands(List<Object> rawCommands) {
+    Map<String, CommandStructure> commands = new HashMap<>();
+
+    for (Object rawEntry : rawCommands) {
+      List<Object> entry = (List<Object>) rawEntry;
+      String key = (String) entry.get(0);
+      List<String> flags = new ArrayList<>();
+      flags.addAll((List<String>) entry.get(2));
+
+      CommandStructure cmd = new CommandStructure(
+          key,
+          (long) entry.get(1),
+          flags,
+          (long) entry.get(3),
+          (long) entry.get(4),
+          (long) entry.get(5));

Review comment:
       Done




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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r717964417



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  SETBIT(new SetBitExecutor(), UNSUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
 
   /**************** Sets *****************/
 
-  SCARD(new SCardExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
-  SDIFF(new SDiffExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SINTER(new SInterExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
-  SMOVE(new SMoveExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
-  SPOP(new SPopExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX)).and(new SpopParameterRequirements())),
-  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SSCAN(new SScanExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  SUNION(new SUnionExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
+  SCARD(new SCardExecutor(), UNSUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  SDIFF(new SDiffExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SINTER(new SInterExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  SMOVE(new SMoveExecutor(), UNSUPPORTED, new Parameter().exact(4).lastKey(2).flags(write, fast)),
+  SPOP(new SPopExecutor(), UNSUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX)
+      .custom(SpopParameterRequirements.checkParameters()).flags(write, random, fast)),
+  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED,
+      new Parameter().min(2).flags(readonly, random)),
+  SSCAN(new SScanExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX).firstKey(0)),
+  SUNION(new SUnionExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
 
   /*************** Server ****************/
 
-  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
-  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  TIME(new TimeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
+  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new Parameter().exact(1).firstKey(0).flags(readonly, fast)),
+  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(admin, noscript, loading,
+          stale)),
+  TIME(new TimeExecutor(), UNSUPPORTED,
+      new Parameter().exact(1).firstKey(0).flags(random, loading, stale, fast)),
 
   /***************************************
    *********** Unknown Commands **********
    ***************************************/
   UNKNOWN(new UnknownExecutor(), RedisCommandSupportLevel.UNKNOWN);
 
+  public enum Flag {
+    admin,

Review comment:
       I think our style guide for enum members is that they are uppercase since they are constants. These are all lowercase. Also you have some that use an underscore but others that are multiple words (denyoom, movablekeys, noscript, readonly) but do not use an underscore.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),

Review comment:
       What do you think of the idea of getting rid of "new Parameter()." by adding static factory methods to Parameter. You could name one of them min(int) and the other exact(int). It seems like you always start with min or exact. This is not a big deal but might make this class a bit more readable.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  SETBIT(new SetBitExecutor(), UNSUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
 
   /**************** Sets *****************/
 
-  SCARD(new SCardExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
-  SDIFF(new SDiffExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SINTER(new SInterExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
-  SMOVE(new SMoveExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
-  SPOP(new SPopExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX)).and(new SpopParameterRequirements())),
-  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SSCAN(new SScanExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  SUNION(new SUnionExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
+  SCARD(new SCardExecutor(), UNSUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  SDIFF(new SDiffExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SINTER(new SInterExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  SMOVE(new SMoveExecutor(), UNSUPPORTED, new Parameter().exact(4).lastKey(2).flags(write, fast)),
+  SPOP(new SPopExecutor(), UNSUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX)
+      .custom(SpopParameterRequirements.checkParameters()).flags(write, random, fast)),
+  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED,
+      new Parameter().min(2).flags(readonly, random)),
+  SSCAN(new SScanExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX).firstKey(0)),
+  SUNION(new SUnionExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
 
   /*************** Server ****************/
 
-  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
-  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  TIME(new TimeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
+  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new Parameter().exact(1).firstKey(0).flags(readonly, fast)),
+  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(admin, noscript, loading,
+          stale)),
+  TIME(new TimeExecutor(), UNSUPPORTED,
+      new Parameter().exact(1).firstKey(0).flags(random, loading, stale, fast)),
 
   /***************************************
    *********** Unknown Commands **********
    ***************************************/
   UNKNOWN(new UnknownExecutor(), RedisCommandSupportLevel.UNKNOWN);
 
+  public enum Flag {
+    admin,
+    denyoom,
+    fast,
+    loading,
+    may_replicate,
+    movablekeys,
+    no_auth,
+    noscript,
+    pubsub,
+    random,
+    readonly,
+    sort_for_script,
+    stale,
+    write;
+  }
+
   private final Executor executor;
-  private final ParameterRequirements parameterRequirements;
-  private final ParameterRequirements deferredParameterRequirements;
+  private final Parameter parameterRequirements;
+  private final Parameter deferredParameterRequirements;
   private final RedisCommandSupportLevel supportLevel;
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel) {
-    this(executor, supportLevel, new UnspecifiedParameterRequirements());
+    this(executor, supportLevel, new Parameter().custom((c, e) -> {
+    }));
   }
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel,
-      ParameterRequirements parameterRequirements) {
-    this(executor, supportLevel, parameterRequirements, new UnspecifiedParameterRequirements());
+      Parameter parameterRequirements) {
+    this(executor, supportLevel, parameterRequirements, new Parameter().custom((c, e) -> {
+    }));
   }
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel,
-      ParameterRequirements parameterRequirements,
-      ParameterRequirements deferredParameterRequirements) {
+      Parameter parameterRequirements,
+      Parameter deferredParameterRequirements) {
     this.executor = executor;
     this.supportLevel = supportLevel;
     this.parameterRequirements = parameterRequirements;
     this.deferredParameterRequirements = deferredParameterRequirements;
   }
 
+  public int arity() {
+    return parameterRequirements.getArity();
+  }
+
+  public List<Flag> flags() {

Review comment:
       instead of a List<Flag> should this be EnumSet<Flag>?

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),

Review comment:
       sounds good




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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r718693433



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  SETBIT(new SetBitExecutor(), UNSUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
 
   /**************** Sets *****************/
 
-  SCARD(new SCardExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
-  SDIFF(new SDiffExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SINTER(new SInterExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
-  SMOVE(new SMoveExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
-  SPOP(new SPopExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX)).and(new SpopParameterRequirements())),
-  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SSCAN(new SScanExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  SUNION(new SUnionExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
+  SCARD(new SCardExecutor(), UNSUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  SDIFF(new SDiffExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SINTER(new SInterExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  SMOVE(new SMoveExecutor(), UNSUPPORTED, new Parameter().exact(4).lastKey(2).flags(write, fast)),
+  SPOP(new SPopExecutor(), UNSUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX)
+      .custom(SpopParameterRequirements.checkParameters()).flags(write, random, fast)),
+  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED,
+      new Parameter().min(2).flags(readonly, random)),
+  SSCAN(new SScanExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX).firstKey(0)),
+  SUNION(new SUnionExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
 
   /*************** Server ****************/
 
-  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
-  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  TIME(new TimeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
+  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new Parameter().exact(1).firstKey(0).flags(readonly, fast)),
+  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(admin, noscript, loading,
+          stale)),
+  TIME(new TimeExecutor(), UNSUPPORTED,
+      new Parameter().exact(1).firstKey(0).flags(random, loading, stale, fast)),
 
   /***************************************
    *********** Unknown Commands **********
    ***************************************/
   UNKNOWN(new UnknownExecutor(), RedisCommandSupportLevel.UNKNOWN);
 
+  public enum Flag {
+    admin,

Review comment:
       I've made them all uppercase now. The reason some use underscores is to match the flags in redis - https://redis.io/commands/command and https://redis.io/topics/modules-api-ref




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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r717964417



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  SETBIT(new SetBitExecutor(), UNSUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
 
   /**************** Sets *****************/
 
-  SCARD(new SCardExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
-  SDIFF(new SDiffExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SINTER(new SInterExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
-  SMOVE(new SMoveExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
-  SPOP(new SPopExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX)).and(new SpopParameterRequirements())),
-  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SSCAN(new SScanExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  SUNION(new SUnionExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
+  SCARD(new SCardExecutor(), UNSUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  SDIFF(new SDiffExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SINTER(new SInterExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  SMOVE(new SMoveExecutor(), UNSUPPORTED, new Parameter().exact(4).lastKey(2).flags(write, fast)),
+  SPOP(new SPopExecutor(), UNSUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX)
+      .custom(SpopParameterRequirements.checkParameters()).flags(write, random, fast)),
+  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED,
+      new Parameter().min(2).flags(readonly, random)),
+  SSCAN(new SScanExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX).firstKey(0)),
+  SUNION(new SUnionExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
 
   /*************** Server ****************/
 
-  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
-  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  TIME(new TimeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
+  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new Parameter().exact(1).firstKey(0).flags(readonly, fast)),
+  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(admin, noscript, loading,
+          stale)),
+  TIME(new TimeExecutor(), UNSUPPORTED,
+      new Parameter().exact(1).firstKey(0).flags(random, loading, stale, fast)),
 
   /***************************************
    *********** Unknown Commands **********
    ***************************************/
   UNKNOWN(new UnknownExecutor(), RedisCommandSupportLevel.UNKNOWN);
 
+  public enum Flag {
+    admin,

Review comment:
       I think our style guide for enum members is that they are uppercase since they are constants. These are all lowercase. Also you have some that use an underscore but others that are multiple words (denyoom, movablekeys, noscript, readonly) but do not use an underscore.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),

Review comment:
       What do you think of the idea of getting rid of "new Parameter()." by adding static factory methods to Parameter. You could name one of them min(int) and the other exact(int). It seems like you always start with min or exact. This is not a big deal but might make this class a bit more readable.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  SETBIT(new SetBitExecutor(), UNSUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
 
   /**************** Sets *****************/
 
-  SCARD(new SCardExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
-  SDIFF(new SDiffExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SINTER(new SInterExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
-  SMOVE(new SMoveExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
-  SPOP(new SPopExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX)).and(new SpopParameterRequirements())),
-  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SSCAN(new SScanExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  SUNION(new SUnionExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
+  SCARD(new SCardExecutor(), UNSUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  SDIFF(new SDiffExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SINTER(new SInterExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  SMOVE(new SMoveExecutor(), UNSUPPORTED, new Parameter().exact(4).lastKey(2).flags(write, fast)),
+  SPOP(new SPopExecutor(), UNSUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX)
+      .custom(SpopParameterRequirements.checkParameters()).flags(write, random, fast)),
+  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED,
+      new Parameter().min(2).flags(readonly, random)),
+  SSCAN(new SScanExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX).firstKey(0)),
+  SUNION(new SUnionExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
 
   /*************** Server ****************/
 
-  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
-  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  TIME(new TimeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
+  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new Parameter().exact(1).firstKey(0).flags(readonly, fast)),
+  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(admin, noscript, loading,
+          stale)),
+  TIME(new TimeExecutor(), UNSUPPORTED,
+      new Parameter().exact(1).firstKey(0).flags(random, loading, stale, fast)),
 
   /***************************************
    *********** Unknown Commands **********
    ***************************************/
   UNKNOWN(new UnknownExecutor(), RedisCommandSupportLevel.UNKNOWN);
 
+  public enum Flag {
+    admin,
+    denyoom,
+    fast,
+    loading,
+    may_replicate,
+    movablekeys,
+    no_auth,
+    noscript,
+    pubsub,
+    random,
+    readonly,
+    sort_for_script,
+    stale,
+    write;
+  }
+
   private final Executor executor;
-  private final ParameterRequirements parameterRequirements;
-  private final ParameterRequirements deferredParameterRequirements;
+  private final Parameter parameterRequirements;
+  private final Parameter deferredParameterRequirements;
   private final RedisCommandSupportLevel supportLevel;
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel) {
-    this(executor, supportLevel, new UnspecifiedParameterRequirements());
+    this(executor, supportLevel, new Parameter().custom((c, e) -> {
+    }));
   }
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel,
-      ParameterRequirements parameterRequirements) {
-    this(executor, supportLevel, parameterRequirements, new UnspecifiedParameterRequirements());
+      Parameter parameterRequirements) {
+    this(executor, supportLevel, parameterRequirements, new Parameter().custom((c, e) -> {
+    }));
   }
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel,
-      ParameterRequirements parameterRequirements,
-      ParameterRequirements deferredParameterRequirements) {
+      Parameter parameterRequirements,
+      Parameter deferredParameterRequirements) {
     this.executor = executor;
     this.supportLevel = supportLevel;
     this.parameterRequirements = parameterRequirements;
     this.deferredParameterRequirements = deferredParameterRequirements;
   }
 
+  public int arity() {
+    return parameterRequirements.getArity();
+  }
+
+  public List<Flag> flags() {

Review comment:
       instead of a List<Flag> should this be EnumSet<Flag>?




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



[GitHub] [geode] dschneider-pivotal commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
dschneider-pivotal commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r718730282



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),

Review comment:
       sounds good




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



[GitHub] [geode] DonalEvans commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
DonalEvans commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r718039774



##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/server/CommandIntegrationTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.server;
+
+import static org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.api.sync.RedisCommands;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.geode.NativeRedisTestRule;
+import org.apache.geode.redis.GeodeRedisServerRule;
+
+public class CommandIntegrationTest {
+
+  @ClassRule
+  public static NativeRedisTestRule redisServer = new NativeRedisTestRule("redis:6.2.4");
+
+  @ClassRule
+  public static GeodeRedisServerRule radishServer = new GeodeRedisServerRule();
+
+  private RedisCommands<String, String> redisClient;
+  private RedisCommands<String, String> radishClient;
+
+  @Before
+  public void setup() {
+    redisClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, redisServer.getPort()))
+            .connect().sync();
+
+    radishClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, radishServer.getPort()))
+            .connect().sync();
+  }
+
+  @After
+  public void teardown() {}
+
+  @Test
+  public void commandReturnsResultsMatchingNativeRedis() {
+    Map<String, CommandStructure> goldenResults = processRawCommands(redisClient.command());
+    Map<String, CommandStructure> results = processRawCommands(radishClient.command());
+
+    List<String> commands = new ArrayList<>(results.keySet());
+    Collections.sort(commands);
+
+    SoftAssertions softly = new SoftAssertions();
+    for (String command : commands) {
+      softly.assertThatCode(() -> compareCommands(results.get(command), goldenResults.get(command)))
+          .as("command: " + command)
+          .doesNotThrowAnyException();
+    }
+    softly.assertAll();
+  }
+
+  private void compareCommands(CommandStructure actual, CommandStructure expected) {
+    assertThat(actual).as("no metadata for " + expected.name).isNotNull();
+    SoftAssertions softly = new SoftAssertions();
+    softly.assertThat(actual.arity).as(expected.name + ".arity").isEqualTo(expected.arity);
+    softly.assertThat(actual.flags).as(expected.name + ".flags")
+        .containsExactlyInAnyOrderElementsOf(expected.flags);
+    softly.assertThat(actual.firstKey).as(expected.name + ".firstKey").isEqualTo(expected.firstKey);
+    softly.assertThat(actual.lastKey).as(expected.name + ".lastKey").isEqualTo(expected.lastKey);
+    softly.assertThat(actual.stepCount).as(expected.name + ".stepCount")
+        .isEqualTo(expected.stepCount);
+    softly.assertAll();
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<String, CommandStructure> processRawCommands(List<Object> rawCommands) {
+    Map<String, CommandStructure> commands = new HashMap<>();
+
+    for (Object rawEntry : rawCommands) {
+      List<Object> entry = (List<Object>) rawEntry;
+      String key = (String) entry.get(0);
+      List<String> flags = new ArrayList<>();
+      flags.addAll((List<String>) entry.get(2));
+
+      CommandStructure cmd = new CommandStructure(
+          key,
+          (long) entry.get(1),
+          flags,
+          (long) entry.get(3),
+          (long) entry.get(4),
+          (long) entry.get(5));

Review comment:
       This can be simplified slightly to inline the `flags` variable:
   ```
         CommandStructure cmd = new CommandStructure(
             key,
             (long) entry.get(1),
             (List<String>) entry.get(2),
             (long) entry.get(3),
             (long) entry.get(4),
             (long) entry.get(5));
   ```

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/parameters/SpopParameterRequirements.java
##########
@@ -18,18 +18,22 @@
 import static org.apache.geode.redis.internal.RedisConstants.ERROR_NOT_INTEGER;
 import static org.apache.geode.redis.internal.netty.Coder.bytesToLong;
 
+import java.util.function.BiConsumer;
+
 import org.apache.geode.redis.internal.netty.Command;
 import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
 
-public class SpopParameterRequirements implements ParameterRequirements {
-  @Override
-  public void checkParameters(Command command, ExecutionHandlerContext context) {
-    if (command.getProcessedCommand().size() == 3) {
-      try {
-        bytesToLong(command.getProcessedCommand().get(2));
-      } catch (NumberFormatException nex) {
-        throw new RedisParametersMismatchException(ERROR_NOT_INTEGER);
+public class SpopParameterRequirements {

Review comment:
       Possibly out of scope for this PR, but this class probably shouldn't exist, and the check that it's doing to validate arguments being integers should be moved to the SPopExecutor.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/executor/server/CommandExecutor.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.server;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.geode.redis.internal.RedisCommandType;
+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 CommandExecutor extends AbstractExecutor {
+  @Override
+  public RedisResponse executeCommand(Command command, ExecutionHandlerContext context)
+      throws Exception {

Review comment:
       `Exception` is never thrown from this method.

##########
File path: geode-apis-compatible-with-redis/src/integrationTest/java/org/apache/geode/redis/internal/executor/server/CommandIntegrationTest.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.server;
+
+import static org.apache.geode.test.dunit.rules.RedisClusterStartupRule.BIND_ADDRESS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import io.lettuce.core.RedisClient;
+import io.lettuce.core.api.sync.RedisCommands;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.geode.NativeRedisTestRule;
+import org.apache.geode.redis.GeodeRedisServerRule;
+
+public class CommandIntegrationTest {
+
+  @ClassRule
+  public static NativeRedisTestRule redisServer = new NativeRedisTestRule("redis:6.2.4");
+
+  @ClassRule
+  public static GeodeRedisServerRule radishServer = new GeodeRedisServerRule();
+
+  private RedisCommands<String, String> redisClient;
+  private RedisCommands<String, String> radishClient;
+
+  @Before
+  public void setup() {
+    redisClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, redisServer.getPort()))
+            .connect().sync();
+
+    radishClient =
+        RedisClient.create(String.format("redis://%s:%d", BIND_ADDRESS, radishServer.getPort()))
+            .connect().sync();
+  }
+
+  @After
+  public void teardown() {}

Review comment:
       This can probably be removed.

##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/parameters/Parameter.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.parameters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.BiConsumer;
+
+import org.apache.geode.redis.internal.RedisCommandType;
+import org.apache.geode.redis.internal.netty.Command;
+import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+
+public class Parameter {
+
+  private int arity;
+  private List<RedisCommandType.Flag> flags = new ArrayList<>();
+  private int firstKey = 1;
+  private int lastKey = 1;
+  private int step = 1;
+  private List<BiConsumer<Command, ExecutionHandlerContext>> predicates = new ArrayList<>();

Review comment:
       None of the predicates use an `ExecutionHandlerContext`, so this could be changed to be just a `Consumer<Command>`.




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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r718698282



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/parameters/Parameter.java
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.parameters;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.function.BiConsumer;
+
+import org.apache.geode.redis.internal.RedisCommandType;
+import org.apache.geode.redis.internal.netty.Command;
+import org.apache.geode.redis.internal.netty.ExecutionHandlerContext;
+
+public class Parameter {
+
+  private int arity;
+  private List<RedisCommandType.Flag> flags = new ArrayList<>();
+  private int firstKey = 1;
+  private int lastKey = 1;
+  private int step = 1;
+  private List<BiConsumer<Command, ExecutionHandlerContext>> predicates = new ArrayList<>();

Review comment:
       Yes, good observation! That cleans things up nicely.




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



[GitHub] [geode] jdeppe-pivotal commented on a change in pull request #6907: GEODE-9623: Add Radish COMMAND command

Posted by GitBox <gi...@apache.org>.
jdeppe-pivotal commented on a change in pull request #6907:
URL: https://github.com/apache/geode/pull/6907#discussion_r718695114



##########
File path: geode-apis-compatible-with-redis/src/main/java/org/apache/geode/redis/internal/RedisCommandType.java
##########
@@ -154,207 +165,274 @@
    ***************************************/
 
   /*************** Connection ****************/
-  AUTH(new AuthExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ECHO(new EchoExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PING(new PingExecutor(), SUPPORTED, new MaximumParameterRequirements(2)),
-  QUIT(new QuitExecutor(), SUPPORTED),
+  AUTH(new AuthExecutor(), SUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX).firstKey(0)
+      .flags(noscript, loading, stale, fast, no_auth)),
+  ECHO(new EchoExecutor(), SUPPORTED, new Parameter().exact(2).firstKey(0).flags(fast)),
+  PING(new PingExecutor(), SUPPORTED, new Parameter().min(1).max(2).firstKey(0).flags(stale, fast)),
+  QUIT(new QuitExecutor(), SUPPORTED, new Parameter().firstKey(0)),
 
   /*************** Keys ******************/
 
-  DEL(new DelExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  DUMP(new DumpExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  EXISTS(new ExistsExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  EXPIRE(new ExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  KEYS(new KeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PERSIST(new PersistExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  PEXPIRE(new PExpireExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PTTL(new PTTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  RENAME(new RenameExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  RESTORE(new RestoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  TTL(new TTLExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  TYPE(new TypeExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  DEL(new DelExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(write)),
+  DUMP(new DumpExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  EXISTS(new ExistsExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  EXPIRE(new ExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  EXPIREAT(new ExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  KEYS(new KeysExecutor(), SUPPORTED,
+      new Parameter().exact(2).firstKey(0).flags(readonly, sort_for_script)),
+  PERSIST(new PersistExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, fast)),
+  PEXPIRE(new PExpireExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PEXPIREAT(new PExpireAtExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, fast)),
+  PTTL(new PTTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  RENAME(new RenameExecutor(), SUPPORTED, new Parameter().exact(3).lastKey(2).flags(write)),
+  RESTORE(new RestoreExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom)),
+  TTL(new TTLExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random, fast)),
+  TYPE(new TypeExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Strings *****************/
 
-  APPEND(new AppendExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  DECR(new DecrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  DECRBY(new DecrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  GET(new GetExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETSET(new GetSetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCRBY(new IncrByExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  INCR(new IncrExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  GETRANGE(new GetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  MGET(new MGetExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  MSET(new MSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  PSETEX(new PSetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SET(new SetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SETEX(new SetEXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  SETNX(new SetNXExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  SETRANGE(new SetRangeExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  STRLEN(new StrlenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  APPEND(new AppendExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  DECR(new DecrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  DECRBY(new DecrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  GET(new GetExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  GETSET(new GetSetExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCRBY(new IncrByExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  INCR(new IncrExecutor(), SUPPORTED, new Parameter().exact(2).flags(write, denyoom, fast)),
+  GETRANGE(new GetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly)),
+  INCRBYFLOAT(new IncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(3).flags(write, denyoom, fast)),
+  MGET(new MGetExecutor(), SUPPORTED, new Parameter().min(2).lastKey(-1).flags(readonly, fast)),
+  MSET(new MSetExecutor(), UNSUPPORTED,
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  PSETEX(new PSetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SET(new SetExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom)),
+  SETEX(new SetEXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  SETNX(new SetNXExecutor(), SUPPORTED, new Parameter().exact(3).flags(write, denyoom, fast)),
+  SETRANGE(new SetRangeExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
+  STRLEN(new StrlenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
 
   /************* Hashes *****************/
 
-  HDEL(new HDelExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HGET(new HGetExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HGETALL(new HGetAllExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HLEN(new HLenExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HMGET(new HMGetExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  HMSET(new HMSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSET(new HSetExecutor(), SUPPORTED,
-      new MinimumParameterRequirements(4).and(new EvenParameterRequirements())),
-  HSETNX(new HSetNXExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HINCRBY(new HIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  HVALS(new HValsExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  HSCAN(new HScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  HEXISTS(new HExistsExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  HKEYS(new HKeysExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
+  HDEL(new HDelExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  HGET(new HGetExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HGETALL(new HGetAllExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, random)),
+  HINCRBYFLOAT(new HIncrByFloatExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write, denyoom, fast)),
+  HLEN(new HLenExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  HMGET(new HMGetExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, fast)),
+  HMSET(new HMSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSET(new HSetExecutor(), SUPPORTED, new Parameter().min(4).even().flags(write, denyoom, fast)),
+  HSETNX(new HSetNXExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HSTRLEN(new HStrLenExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HINCRBY(new HIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  HVALS(new HValsExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
+  HSCAN(new HScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  HEXISTS(new HExistsExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  HKEYS(new HKeysExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, sort_for_script)),
 
   /************* Sets *****************/
 
-  SADD(new SAddExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  SMEMBERS(new SMembersExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  SREM(new SRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
+  SADD(new SAddExecutor(), SUPPORTED, new Parameter().min(3).flags(write, denyoom, fast)),
+  SMEMBERS(new SMembersExecutor(), SUPPORTED,
+      new Parameter().exact(2).flags(readonly, sort_for_script)),
+  SREM(new SRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
 
   /************ Sorted Sets **************/
 
-  ZADD(new ZAddExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZCARD(new ZCardExecutor(), SUPPORTED, new ExactParameterRequirements(2)),
-  ZCOUNT(new ZCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX))),
-  ZRANGE(new ZRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZRANK(new ZRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZREM(new ZRemExecutor(), SUPPORTED, new MinimumParameterRequirements(3)),
-  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED, new ExactParameterRequirements(4)),
-  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED, new MinimumParameterRequirements(4)
-      .and(new MaximumParameterRequirements(5, ERROR_SYNTAX))),
-  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
-  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZSCAN(new ZScanExecutor(), SUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  ZSCORE(new ZScoreExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED, new MinimumParameterRequirements(4)),
+  ZADD(new ZAddExecutor(), SUPPORTED, new Parameter().min(4).flags(write, denyoom, fast)),
+  ZCARD(new ZCardExecutor(), SUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  ZCOUNT(new ZCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZINCRBY(new ZIncrByExecutor(), SUPPORTED, new Parameter().exact(4).flags(write, denyoom, fast)),
+  ZLEXCOUNT(new ZLexCountExecutor(), SUPPORTED, new Parameter().exact(4).flags(readonly, fast)),
+  ZPOPMAX(new ZPopMaxExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZPOPMIN(new ZPopMinExecutor(), SUPPORTED,
+      new Parameter().min(2).max(3, ERROR_SYNTAX).flags(write, fast)),
+  ZRANGE(new ZRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZRANGEBYLEX(new ZRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANGEBYSCORE(new ZRangeByScoreExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZRANK(new ZRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZREM(new ZRemExecutor(), SUPPORTED, new Parameter().min(3).flags(write, fast)),
+  ZREMRANGEBYLEX(new ZRemRangeByLexExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYRANK(new ZRemRangeByRankExecutor(), SUPPORTED, new Parameter().exact(4).flags(write)),
+  ZREMRANGEBYSCORE(new ZRemRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().exact(4).flags(write)),
+  ZREVRANGE(new ZRevRangeExecutor(), SUPPORTED,
+      new Parameter().min(4).max(5, ERROR_SYNTAX).flags(readonly)),
+  ZREVRANGEBYLEX(new ZRevRangeByLexExecutor(), SUPPORTED, new Parameter().min(4).flags(readonly)),
+  ZREVRANGEBYSCORE(new ZRevRangeByScoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(readonly)),
+  ZREVRANK(new ZRevRankExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZSCAN(new ZScanExecutor(), SUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX)),
+  ZSCORE(new ZScoreExecutor(), SUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  ZUNIONSTORE(new ZUnionStoreExecutor(), SUPPORTED,
+      new Parameter().min(4).flags(write, denyoom, movablekeys)),
 
   /************* Server *****************/
-  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new SlowlogParameterRequirements()),
-  INFO(new InfoExecutor(), SUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
+  COMMAND(new CommandExecutor(), SUPPORTED, new Parameter().min(1).firstKey(0).flags(random, loading, stale)),
+  SLOWLOG(new SlowlogExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(SlowlogParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, loading, stale)),
+  INFO(new InfoExecutor(), SUPPORTED, new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0)
+      .flags(random, loading, stale)),
 
   /********** Publish Subscribe **********/
-  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUBLISH(new PublishExecutor(), SUPPORTED, new ExactParameterRequirements(3)),
-  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
-  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED, new MinimumParameterRequirements(1)),
-  PUBSUB(new PubSubExecutor(), SUPPORTED, new MinimumParameterRequirements(2)),
+  SUBSCRIBE(new SubscribeExecutor(), SUPPORTED, new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBLISH(new PublishExecutor(), SUPPORTED,
+      new Parameter().exact(3).firstKey(0).flags(pubsub, loading, stale, fast, may_replicate)),
+  PSUBSCRIBE(new PsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUNSUBSCRIBE(new PunsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  UNSUBSCRIBE(new UnsubscribeExecutor(), SUPPORTED,
+      new Parameter().min(1).firstKey(0).flags(pubsub, noscript, loading, stale)),
+  PUBSUB(new PubSubExecutor(), SUPPORTED,
+      new Parameter().min(2).firstKey(0).flags(pubsub, random, loading, stale)),
 
   /************* Cluster *****************/
-  CLUSTER(new ClusterExecutor(), SUPPORTED, new ClusterParameterRequirements()),
+  CLUSTER(new ClusterExecutor(), SUPPORTED, new Parameter().min(2)
+      .custom(ClusterParameterRequirements.checkParameters()).firstKey(0)
+      .flags(admin, random, stale)),
 
   /***************************************
    ********* Internal Commands ***********
    ***************************************/
   // do not call these directly, only to be used in other commands
-  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(2)),
-  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new ExactParameterRequirements(3)),
+  INTERNALPTTL(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALTYPE(new UnknownExecutor(), INTERNAL, new Parameter().exact(2)),
+  INTERNALSMEMBERS(new UnknownExecutor(), INTERNAL, new Parameter().exact(3)),
 
   /***************************************
    ******** Unsupported Commands *********
    ***************************************/
 
   /*************** Connection *************/
 
-  SELECT(new SelectExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
+  SELECT(new SelectExecutor(), UNSUPPORTED, new Parameter().exact(2).firstKey(0).flags(loading, stale, fast)),
 
   /*************** Keys ******************/
 
-  SCAN(new ScanExecutor(), UNSUPPORTED,
-      new EvenParameterRequirements(ERROR_SYNTAX).and(new MinimumParameterRequirements(2))),
-  UNLINK(new DelExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
+  SCAN(new ScanExecutor(), UNSUPPORTED, new Parameter().min(2).even(ERROR_SYNTAX).firstKey(0).flags(readonly, random)),
+  UNLINK(new DelExecutor(), UNSUPPORTED, new Parameter().min(2).lastKey(-1).flags(write, fast)),
 
   /************** Strings ****************/
 
-  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  BITOP(new BitOpExecutor(), UNSUPPORTED, new MinimumParameterRequirements(4)),
-  BITPOS(new BitPosExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  GETBIT(new GetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
+  BITCOUNT(new BitCountExecutor(), UNSUPPORTED, new Parameter().min(2).flags(readonly)),
+  BITOP(new BitOpExecutor(), UNSUPPORTED,
+      new Parameter().min(4).firstKey(2).lastKey(-1).flags(write, denyoom)),
+  BITPOS(new BitPosExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly)),
+  GETBIT(new GetBitExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
   MSETNX(new MSetNXExecutor(), UNSUPPORTED,
-      new MinimumParameterRequirements(3).and(new OddParameterRequirements())),
-  SETBIT(new SetBitExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
+      new Parameter().min(3).odd().lastKey(-1).step(2).flags(write, denyoom)),
+  SETBIT(new SetBitExecutor(), UNSUPPORTED, new Parameter().exact(4).flags(write, denyoom)),
 
   /**************** Sets *****************/
 
-  SCARD(new SCardExecutor(), UNSUPPORTED, new ExactParameterRequirements(2)),
-  SDIFF(new SDiffExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SINTER(new SInterExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
-  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new ExactParameterRequirements(3)),
-  SMOVE(new SMoveExecutor(), UNSUPPORTED, new ExactParameterRequirements(4)),
-  SPOP(new SPopExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)
-      .and(new MaximumParameterRequirements(3, ERROR_SYNTAX)).and(new SpopParameterRequirements())),
-  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SSCAN(new SScanExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3),
-      new OddParameterRequirements(ERROR_SYNTAX)),
-  SUNION(new SUnionExecutor(), UNSUPPORTED, new MinimumParameterRequirements(2)),
-  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED, new MinimumParameterRequirements(3)),
+  SCARD(new SCardExecutor(), UNSUPPORTED, new Parameter().exact(2).flags(readonly, fast)),
+  SDIFF(new SDiffExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SDIFFSTORE(new SDiffStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SINTER(new SInterExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SINTERSTORE(new SInterStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
+  SISMEMBER(new SIsMemberExecutor(), UNSUPPORTED, new Parameter().exact(3).flags(readonly, fast)),
+  SMOVE(new SMoveExecutor(), UNSUPPORTED, new Parameter().exact(4).lastKey(2).flags(write, fast)),
+  SPOP(new SPopExecutor(), UNSUPPORTED, new Parameter().min(2).max(3, ERROR_SYNTAX)
+      .custom(SpopParameterRequirements.checkParameters()).flags(write, random, fast)),
+  SRANDMEMBER(new SRandMemberExecutor(), UNSUPPORTED,
+      new Parameter().min(2).flags(readonly, random)),
+  SSCAN(new SScanExecutor(), UNSUPPORTED, new Parameter().min(3).flags(readonly, random),
+      new Parameter().odd(ERROR_SYNTAX).firstKey(0)),
+  SUNION(new SUnionExecutor(), UNSUPPORTED,
+      new Parameter().min(2).lastKey(-1).flags(readonly, sort_for_script)),
+  SUNIONSTORE(new SUnionStoreExecutor(), UNSUPPORTED,
+      new Parameter().min(3).lastKey(-1).flags(write, denyoom)),
 
   /*************** Server ****************/
 
-  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
-  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED, new MaximumParameterRequirements(2, ERROR_SYNTAX)),
-  TIME(new TimeExecutor(), UNSUPPORTED, new ExactParameterRequirements(1)),
+  DBSIZE(new DBSizeExecutor(), UNSUPPORTED, new Parameter().exact(1).firstKey(0).flags(readonly, fast)),
+  FLUSHALL(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  FLUSHDB(new FlushAllExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(write)),
+  SHUTDOWN(new ShutDownExecutor(), UNSUPPORTED,
+      new Parameter().min(1).max(2, ERROR_SYNTAX).firstKey(0).flags(admin, noscript, loading,
+          stale)),
+  TIME(new TimeExecutor(), UNSUPPORTED,
+      new Parameter().exact(1).firstKey(0).flags(random, loading, stale, fast)),
 
   /***************************************
    *********** Unknown Commands **********
    ***************************************/
   UNKNOWN(new UnknownExecutor(), RedisCommandSupportLevel.UNKNOWN);
 
+  public enum Flag {
+    admin,
+    denyoom,
+    fast,
+    loading,
+    may_replicate,
+    movablekeys,
+    no_auth,
+    noscript,
+    pubsub,
+    random,
+    readonly,
+    sort_for_script,
+    stale,
+    write;
+  }
+
   private final Executor executor;
-  private final ParameterRequirements parameterRequirements;
-  private final ParameterRequirements deferredParameterRequirements;
+  private final Parameter parameterRequirements;
+  private final Parameter deferredParameterRequirements;
   private final RedisCommandSupportLevel supportLevel;
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel) {
-    this(executor, supportLevel, new UnspecifiedParameterRequirements());
+    this(executor, supportLevel, new Parameter().custom((c, e) -> {
+    }));
   }
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel,
-      ParameterRequirements parameterRequirements) {
-    this(executor, supportLevel, parameterRequirements, new UnspecifiedParameterRequirements());
+      Parameter parameterRequirements) {
+    this(executor, supportLevel, parameterRequirements, new Parameter().custom((c, e) -> {
+    }));
   }
 
   RedisCommandType(Executor executor, RedisCommandSupportLevel supportLevel,
-      ParameterRequirements parameterRequirements,
-      ParameterRequirements deferredParameterRequirements) {
+      Parameter parameterRequirements,
+      Parameter deferredParameterRequirements) {
     this.executor = executor;
     this.supportLevel = supportLevel;
     this.parameterRequirements = parameterRequirements;
     this.deferredParameterRequirements = deferredParameterRequirements;
   }
 
+  public int arity() {
+    return parameterRequirements.getArity();
+  }
+
+  public List<Flag> flags() {

Review comment:
       Cool - haven't used that before.




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