You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by kl...@apache.org on 2016/07/08 18:21:13 UTC

[05/21] incubator-geode git commit: GEODE-1566: rename GeodeRedisServer and repackage redis code into org.apache.geode

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MGetExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MGetExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MGetExecutor.java
new file mode 100755
index 0000000..e8e5bf8
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MGetExecutor.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import com.gemstone.gemfire.cache.Region;
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants.ArityDef;
+
+public class MGetExecutor extends StringExecutor {
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
+
+    if (commandElems.size() < 2) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.MGET));
+      return;
+    }
+
+    Collection<ByteArrayWrapper> keys = new ArrayList<ByteArrayWrapper>();
+    for (int i = 1; i < commandElems.size(); i++) {
+      byte[] keyArray = commandElems.get(i);
+      ByteArrayWrapper key = new ByteArrayWrapper(keyArray);
+      /*
+      try {
+        checkDataType(key, RedisDataType.REDIS_STRING, context);
+      } catch (RedisDataTypeMismatchException e) {
+        keys.ad
+        continue;
+      }
+      */
+      keys.add(key);
+    }
+
+    Map<ByteArrayWrapper, ByteArrayWrapper> results = r.getAll(keys);
+
+    Collection<ByteArrayWrapper> values = new ArrayList<ByteArrayWrapper>();
+
+    /*
+     * This is done to preserve order in the output
+     */
+    for (ByteArrayWrapper key : keys)
+      values.add(results.get(key));
+
+    command.setResponse(Coder.getBulkStringArrayResponse(context.getByteBufAllocator(), values));
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MSetExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MSetExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MSetExecutor.java
new file mode 100755
index 0000000..0cc1182
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MSetExecutor.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.gemstone.gemfire.cache.Region;
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants.ArityDef;
+import org.apache.geode.redis.internal.RedisDataTypeMismatchException;
+import org.apache.geode.redis.internal.Coder;
+
+public class MSetExecutor extends StringExecutor {
+
+  private final String SUCCESS = "OK";
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
+
+    if (commandElems.size() < 3 || commandElems.size() % 2 == 0) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.MSET));
+      return;
+    }
+
+    Map<ByteArrayWrapper, ByteArrayWrapper> map = new HashMap<ByteArrayWrapper, ByteArrayWrapper>();
+    for (int i = 1; i < commandElems.size(); i += 2) {
+      byte[] keyArray = commandElems.get(i);
+      ByteArrayWrapper key = new ByteArrayWrapper(keyArray);
+      try {
+        checkAndSetDataType(key, context);
+      } catch (RedisDataTypeMismatchException e) {
+        continue;
+      }
+      byte[] value = commandElems.get(i + 1);
+      map.put(key, new ByteArrayWrapper(value));
+    }
+    r.putAll(map);
+
+    command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), SUCCESS));
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MSetNXExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MSetNXExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MSetNXExecutor.java
new file mode 100755
index 0000000..9e45b5e
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/MSetNXExecutor.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.gemstone.gemfire.cache.Region;
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants.ArityDef;
+import org.apache.geode.redis.internal.RedisDataTypeMismatchException;
+import org.apache.geode.redis.internal.Coder;
+
+public class MSetNXExecutor extends StringExecutor {
+
+  private final int SET = 1;
+
+  private final int NOT_SET = 0;
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
+
+    if (commandElems.size() < 3 || commandElems.size() % 2 == 0) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.MSETNX));
+      return;
+    }
+
+    boolean hasEntry = false;
+
+    Map<ByteArrayWrapper, ByteArrayWrapper> map = new HashMap<ByteArrayWrapper, ByteArrayWrapper>();
+    for (int i = 1; i < commandElems.size(); i += 2) {
+      byte[] keyArray = commandElems.get(i);
+      ByteArrayWrapper key = new ByteArrayWrapper(keyArray);
+      try {
+        checkDataType(key, context);
+      } catch (RedisDataTypeMismatchException e) {
+        hasEntry = true;
+        break;
+      }
+      byte[] value = commandElems.get(i + 1);
+      map.put(key, new ByteArrayWrapper(value));
+      if (r.containsKey(key)) {
+        hasEntry = true;
+        break;
+      }
+    }
+    boolean successful = false;
+    if (!hasEntry) {
+      successful = true;
+      for (ByteArrayWrapper k : map.keySet()) {
+        try {
+          checkAndSetDataType(k, context);
+        } catch (RedisDataTypeMismatchException e) {
+          successful = false;
+          break;
+        }
+      }
+      r.putAll(map);
+    }
+    if (successful) {
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), SET));
+    } else {
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), NOT_SET));
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/PSetEXExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/PSetEXExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/PSetEXExecutor.java
new file mode 100755
index 0000000..c04de28
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/PSetEXExecutor.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import org.apache.geode.redis.internal.RedisConstants.ArityDef;
+
+
+public class PSetEXExecutor extends SetEXExecutor {
+
+  @Override
+  public boolean timeUnitMillis() {
+    return true;
+  }
+  
+  @Override
+  public String getArgsError() {
+    return ArityDef.PSETEX;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetBitExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetBitExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetBitExecutor.java
new file mode 100755
index 0000000..cbe7b82
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetBitExecutor.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import java.util.List;
+
+import com.gemstone.gemfire.cache.Region;
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants.ArityDef;
+
+public class SetBitExecutor extends StringExecutor {
+
+  private final String ERROR_NOT_INT = "The number provided must be numeric";
+
+  private final String ERROR_VALUE = "The value is out of range, must be 0 or 1";
+
+  private final String ERROR_ILLEGAL_OFFSET = "The offset is out of range, must be greater than or equal to 0  and at most 4294967295 (512MB)";
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
+
+    if (commandElems.size() < 4) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.SETBIT));
+      return;
+    }
+
+    ByteArrayWrapper key = command.getKey();
+    checkAndSetDataType(key, context);
+    ByteArrayWrapper wrapper = r.get(key);
+
+    long offset;
+    int value;
+    int returnBit = 0;
+    try {
+      byte[] offAr = commandElems.get(2);
+      byte[] valAr = commandElems.get(3);
+      offset = Coder.bytesToLong(offAr);
+      value = Coder.bytesToInt(valAr);
+    } catch (NumberFormatException e) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ERROR_NOT_INT));
+      return;
+    }
+
+    if (value != 0 && value != 1) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ERROR_VALUE));
+      return;
+    }
+
+    if (offset < 0 || offset > 4294967295L) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ERROR_ILLEGAL_OFFSET));
+      return;
+    }
+
+    int byteIndex = (int) (offset / 8);
+    offset %= 8;
+
+    if (wrapper == null) {
+      byte[] bytes = new byte[byteIndex + 1];
+      if (value == 1)
+        bytes[byteIndex] = (byte) (0x80 >> offset);
+      r.put(key, new ByteArrayWrapper(bytes));
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), 0));
+    } else {
+
+      byte[] bytes = wrapper.toBytes();
+      if (byteIndex < bytes.length)
+        returnBit = (bytes[byteIndex] & (0x80 >> offset)) >> (7 - offset);
+        else 
+          returnBit = 0;
+
+      if (byteIndex < bytes.length) {
+        bytes[byteIndex] = value == 1 ? (byte) (bytes[byteIndex] | (0x80 >> offset)) : (byte) (bytes[byteIndex] & ~(0x80 >> offset));
+        r.put(key, new ByteArrayWrapper(bytes));
+      } else {
+        byte[] newBytes = new byte[byteIndex + 1];
+        System.arraycopy(bytes, 0, newBytes, 0, bytes.length);
+        newBytes[byteIndex] = value == 1 ? (byte) (newBytes[byteIndex] | (0x80 >> offset)) : (byte) (newBytes[byteIndex] & ~(0x80 >> offset));
+        r.put(key, new ByteArrayWrapper(newBytes));
+      }
+
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), returnBit));
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetEXExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetEXExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetEXExecutor.java
new file mode 100755
index 0000000..cabb320
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetEXExecutor.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import java.util.List;
+
+import com.gemstone.gemfire.cache.Region;
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.Extendable;
+import org.apache.geode.redis.internal.RedisConstants.ArityDef;
+import org.apache.geode.redis.internal.executor.AbstractExecutor;
+
+public class SetEXExecutor extends StringExecutor implements Extendable {
+
+  private final String ERROR_SECONDS_NOT_A_NUMBER = "The expiration argument provided was not a number";
+
+  private final String ERROR_SECONDS_NOT_LEGAL = "The expiration argument must be greater than 0";
+
+  private final String SUCCESS = "OK";
+
+  private final int VALUE_INDEX = 3;
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
+
+    if (commandElems.size() < 4) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), getArgsError()));
+      return;
+    }
+
+    ByteArrayWrapper key = command.getKey();
+    byte[] value = commandElems.get(VALUE_INDEX);
+
+    byte[] expirationArray = commandElems.get(2);
+    long expiration;
+    try {
+      expiration = Coder.bytesToLong(expirationArray);
+    } catch (NumberFormatException e) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ERROR_SECONDS_NOT_A_NUMBER));
+      return;
+    }
+
+    if (expiration <= 0) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ERROR_SECONDS_NOT_LEGAL));
+      return;
+    }
+
+    if (!timeUnitMillis())
+      expiration *= AbstractExecutor.millisInSecond;
+
+    checkAndSetDataType(key, context);
+    r.put(key, new ByteArrayWrapper(value));
+
+    context.getRegionProvider().setExpiration(key, expiration);
+
+    command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), SUCCESS));
+
+  }
+
+  protected boolean timeUnitMillis() {
+    return false;
+  }
+
+  @Override
+  public String getArgsError() {
+    return ArityDef.SETEX;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetExecutor.java
new file mode 100755
index 0000000..ff60947
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetExecutor.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import java.util.List;
+
+import com.gemstone.gemfire.cache.Region;
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants.ArityDef;
+import org.apache.geode.redis.internal.executor.AbstractExecutor;
+
+public class SetExecutor extends StringExecutor {
+
+  private final String SUCCESS = "OK";
+
+  private final int VALUE_INDEX = 2;
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
+
+    if (commandElems.size() < 3) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.SET));
+      return;
+    }
+
+    ByteArrayWrapper key = command.getKey();
+    checkDataType(key, context);
+    byte[] value = commandElems.get(VALUE_INDEX);
+    ByteArrayWrapper valueWrapper = new ByteArrayWrapper(value);
+
+    boolean NX = false; // Set only if not exists
+    boolean XX = false; // Set only if exists
+    long expiration = 0L;
+
+    if (commandElems.size() >= 6) {
+      String elem4;
+      String elem5;
+      String elem6;
+
+      elem4 = Coder.bytesToString(commandElems.get(3));
+      elem5 = Coder.bytesToString(commandElems.get(4));
+      elem6 = Coder.bytesToString(commandElems.get(5));
+
+      if (elem4.equalsIgnoreCase("XX") || elem6.equalsIgnoreCase("XX"))
+        XX = true;
+      else if (elem4.equalsIgnoreCase("NX") || elem6.equalsIgnoreCase("NX"))
+        NX = true;
+
+      if (elem4.equalsIgnoreCase("PX"))
+        expiration = getExpirationMillis(elem4, elem5);
+      else if (elem5.equalsIgnoreCase("PX"))
+        expiration = getExpirationMillis(elem5, elem6);
+      else if (elem4.equalsIgnoreCase("EX"))
+        expiration = getExpirationMillis(elem4, elem5);
+      else if (elem5.equalsIgnoreCase("EX"))
+        expiration = getExpirationMillis(elem5, elem6);
+
+    } else if (commandElems.size() >= 5) {
+      String elem4;
+      String expiry;
+
+      elem4 = Coder.bytesToString(commandElems.get(3));
+      expiry = Coder.bytesToString(commandElems.get(4));
+
+      expiration = getExpirationMillis(elem4, expiry);
+    } else if (commandElems.size() >= 4) {
+      byte[] elem4 = commandElems.get(3);
+      if (elem4.length == 2 && Character.toUpperCase(elem4[1]) == 'X') {
+        if (Character.toUpperCase(elem4[0]) == 'N')
+          NX = true;
+        else if (Character.toUpperCase(elem4[0]) == 'X')
+          XX = true;
+      }
+    }
+
+    boolean keyWasSet = false;
+
+    if (NX)
+      keyWasSet = setNX(r, command, key, valueWrapper, context);
+    else if (XX)
+      keyWasSet = setXX(r, command, key, valueWrapper, context);
+    else {
+      checkAndSetDataType(key, context);
+      r.put(key, valueWrapper);
+      command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), SUCCESS));
+      keyWasSet = true;
+    }
+
+    if (keyWasSet && expiration > 0L) {
+      context.getRegionProvider().setExpiration(key, expiration);
+    }
+
+  }
+
+  private boolean setNX(Region<ByteArrayWrapper, ByteArrayWrapper> r, Command command, ByteArrayWrapper key, ByteArrayWrapper valueWrapper, ExecutionHandlerContext context) {
+    checkAndSetDataType(key, context);
+    Object oldValue = r.putIfAbsent(key, valueWrapper);
+    if (oldValue != null) {
+      command.setResponse(Coder.getNilResponse(context.getByteBufAllocator()));
+      return false;
+    } else {
+      command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), SUCCESS));
+      return true;
+    }
+  }
+
+  private boolean setXX(Region<ByteArrayWrapper, ByteArrayWrapper> r, Command command, ByteArrayWrapper key, ByteArrayWrapper valueWrapper, ExecutionHandlerContext context) {
+    if(r.containsKey(key)) {
+      checkAndSetDataType(key, context);
+      r.put(key, valueWrapper);
+      command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), SUCCESS));
+      return true;
+    } else {
+      command.setResponse(Coder.getNilResponse(context.getByteBufAllocator()));
+      return false;
+    }
+  }
+
+  private long getExpirationMillis(String expx, String expirationString) {
+    long expiration = 0L;
+    try {
+      expiration = Long.parseLong(expirationString);
+    } catch (NumberFormatException e) {
+      return 0L;
+    }
+
+    if (expx.equalsIgnoreCase("EX"))
+      return expiration * AbstractExecutor.millisInSecond;
+    else if (expx.equalsIgnoreCase("PX"))
+      return expiration;
+    else
+      return 0L;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetNXExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetNXExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetNXExecutor.java
new file mode 100755
index 0000000..a920696
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetNXExecutor.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import java.util.List;
+
+import com.gemstone.gemfire.cache.Region;
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants.ArityDef;
+
+public class SetNXExecutor extends StringExecutor {
+
+  private final int SET = 1;
+
+  private final int NOT_SET = 0;
+
+  private final int VALUE_INDEX = 2;
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
+
+    if (commandElems.size() < 3) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.SETNX));
+      return;
+    }
+
+    ByteArrayWrapper key = command.getKey();
+    checkAndSetDataType(key, context);
+    byte[] value = commandElems.get(VALUE_INDEX);
+
+    Object oldValue = r.putIfAbsent(key, new ByteArrayWrapper(value));
+
+    if (oldValue != null)
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), NOT_SET));
+    else
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), SET));
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetRangeExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetRangeExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetRangeExecutor.java
new file mode 100755
index 0000000..5a27749
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/SetRangeExecutor.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import java.util.List;
+
+import com.gemstone.gemfire.cache.Region;
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants.ArityDef;
+
+public class SetRangeExecutor extends StringExecutor {
+
+  private final String ERROR_NOT_INT = "The number provided must be numeric";
+
+  private final String ERROR_ILLEGAL_OFFSET = "The offset is out of range, must be greater than or equal to 0 and the offset added to the length of the value must be less than 536870911 (512MB), the maximum allowed size";
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
+
+    if (commandElems.size() < 4) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.SETRANGE));
+      return;
+    }
+
+    ByteArrayWrapper key = command.getKey();
+    checkAndSetDataType(key, context);
+    ByteArrayWrapper wrapper = r.get(key);
+
+    int offset;
+    byte[] value = commandElems.get(3);
+    try {
+      byte[] offAr = commandElems.get(2);
+      offset = Coder.bytesToInt(offAr);
+    } catch (NumberFormatException e) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ERROR_NOT_INT));
+      return;
+    }
+
+    int totalLength = offset + value.length;
+    if (offset < 0 || totalLength > 536870911) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ERROR_ILLEGAL_OFFSET));
+      return;
+    } else if (value.length == 0) {
+      int length = wrapper == null ? 0 : wrapper.toBytes().length;
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), length));
+      if (wrapper == null)
+        context.getRegionProvider().removeKey(key);
+      return;
+    }
+
+    if (wrapper == null) {
+      byte[] bytes = new byte[totalLength];
+      System.arraycopy(value, 0, bytes, offset, value.length);
+      r.put(key, new ByteArrayWrapper(bytes));
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), bytes.length));
+    } else {
+
+      byte[] bytes = wrapper.toBytes();
+      int returnLength;
+      if (totalLength < bytes.length) {
+        System.arraycopy(value, 0, bytes, offset, value.length);
+        r.put(key, new ByteArrayWrapper(bytes));
+        returnLength = bytes.length;
+      } else {
+        byte[] newBytes = new byte[totalLength];
+        System.arraycopy(bytes, 0, newBytes, 0, bytes.length);
+        System.arraycopy(value, 0, newBytes, offset, value.length);
+        returnLength = newBytes.length;
+        r.put(key, new ByteArrayWrapper(newBytes));
+      }
+
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), returnLength));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/StringExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/StringExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/StringExecutor.java
new file mode 100755
index 0000000..f566f43
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/StringExecutor.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisDataType;
+import org.apache.geode.redis.internal.RedisDataTypeMismatchException;
+import org.apache.geode.redis.internal.executor.AbstractExecutor;
+
+public abstract class StringExecutor extends AbstractExecutor {
+  
+  protected final void checkAndSetDataType(ByteArrayWrapper key, ExecutionHandlerContext context) {
+    Object oldVal = context.getRegionProvider().metaPutIfAbsent(key, RedisDataType.REDIS_STRING);
+    if (oldVal == RedisDataType.REDIS_PROTECTED)
+      throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is protected");
+    if (oldVal != null && oldVal != RedisDataType.REDIS_STRING)
+      throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is already used by a " + oldVal.toString());
+  }
+  
+  protected void checkDataType(ByteArrayWrapper key, ExecutionHandlerContext context) {
+    RedisDataType currentType = context.getRegionProvider().getRedisDataType(key);
+    if (currentType == null)
+      return;
+    if (currentType == RedisDataType.REDIS_PROTECTED)
+      throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is protected");
+    if (currentType != RedisDataType.REDIS_STRING)
+      throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is already used by a " + currentType.toString());
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/StrlenExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/StrlenExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/StrlenExecutor.java
new file mode 100755
index 0000000..3d05863
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/string/StrlenExecutor.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.redis.internal.executor.string;
+
+import java.util.List;
+
+import com.gemstone.gemfire.cache.Region;
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisDataType;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.RedisConstants.ArityDef;
+
+public class StrlenExecutor extends StringExecutor {
+
+  private final int KEY_DOES_NOT_EXIST = 0;
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    List<byte[]> commandElems = command.getProcessedCommand();
+
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
+
+    if (commandElems.size() < 2) {
+      command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.STRLEN));
+      return;
+    }
+
+    ByteArrayWrapper key = command.getKey();
+    checkDataType(key, RedisDataType.REDIS_STRING, context);
+    ByteArrayWrapper valueWrapper = r.get(key);
+
+
+    if (valueWrapper == null)
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), KEY_DOES_NOT_EXIST));
+    else
+      command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), valueWrapper.toBytes().length));
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/DiscardExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/DiscardExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/DiscardExecutor.java
new file mode 100755
index 0000000..c79cca6
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/DiscardExecutor.java
@@ -0,0 +1,42 @@
+/*
+ * 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.transactions;
+
+import com.gemstone.gemfire.cache.CacheTransactionManager;
+import com.gemstone.gemfire.cache.TransactionId;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+
+public class DiscardExecutor extends TransactionExecutor {
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+
+    CacheTransactionManager txm = context.getCacheTransactionManager();
+
+    if (context.hasTransaction()) {
+      TransactionId transactionId = context.getTransactionID();
+      txm.resume(transactionId);
+      txm.rollback();
+      context.clearTransaction();
+    }
+
+    command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), "OK"));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/ExecExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/ExecExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/ExecExecutor.java
new file mode 100755
index 0000000..6bf1c6d
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/ExecExecutor.java
@@ -0,0 +1,88 @@
+/*
+ * 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.transactions;
+
+import io.netty.buffer.ByteBuf;
+
+import java.util.Queue;
+
+import com.gemstone.gemfire.cache.CacheTransactionManager;
+import com.gemstone.gemfire.cache.CommitConflictException;
+import com.gemstone.gemfire.cache.TransactionId;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants;
+
+public class ExecExecutor extends TransactionExecutor {
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+
+    CacheTransactionManager txm = context.getCacheTransactionManager();    
+
+    if (!context.hasTransaction()) {
+      command.setResponse(Coder.getNilResponse(context.getByteBufAllocator()));
+      return;
+    }
+
+    TransactionId transactionId = context.getTransactionID();
+
+    txm.resume(transactionId);
+
+    boolean hasError = hasError(context.getTransactionQueue());
+
+    if (hasError)
+      txm.rollback();
+    else {
+      try {
+        txm.commit();
+      } catch (CommitConflictException e) {
+        command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), RedisConstants.ERROR_COMMIT_CONFLICT));
+        context.clearTransaction();
+        return;
+      }
+    }
+
+    ByteBuf response = constructResponseExec(context);
+    command.setResponse(response);
+
+    context.clearTransaction();
+  }
+
+  private ByteBuf constructResponseExec(ExecutionHandlerContext context) {
+    Queue<Command> cQ = context.getTransactionQueue();
+    ByteBuf response = context.getByteBufAllocator().buffer();
+    response.writeByte(Coder.ARRAY_ID);
+    response.writeBytes(Coder.intToBytes(cQ.size()));
+    response.writeBytes(Coder.CRLFar);
+
+    for (Command c: cQ) {
+      ByteBuf r = c.getResponse();
+      response.writeBytes(r);
+    }
+    return response;
+  }
+
+  private boolean hasError(Queue<Command> queue) {
+    for (Command c: queue) {
+      if (c.hasError())
+        return true;
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/MultiExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/MultiExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/MultiExecutor.java
new file mode 100755
index 0000000..17bf686
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/MultiExecutor.java
@@ -0,0 +1,47 @@
+/*
+ * 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.transactions;
+
+import com.gemstone.gemfire.cache.CacheTransactionManager;
+import com.gemstone.gemfire.cache.TransactionId;
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants;
+
+public class MultiExecutor extends TransactionExecutor {
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+
+    CacheTransactionManager txm = context.getCacheTransactionManager();
+
+    command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), "OK"));
+
+    if (context.hasTransaction()) {
+      throw new IllegalStateException(RedisConstants.ERROR_NESTED_MULTI);
+    }
+
+    txm.begin();
+
+    TransactionId id = txm.suspend();
+
+    context.setTransactionID(id);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/TransactionExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/TransactionExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/TransactionExecutor.java
new file mode 100755
index 0000000..ca2a3b0
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/TransactionExecutor.java
@@ -0,0 +1,23 @@
+/*
+ * 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.transactions;
+
+import org.apache.geode.redis.internal.executor.AbstractExecutor;
+
+public abstract class TransactionExecutor extends AbstractExecutor {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/UnwatchExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/UnwatchExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/UnwatchExecutor.java
new file mode 100755
index 0000000..ad0e560
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/UnwatchExecutor.java
@@ -0,0 +1,31 @@
+/*
+ * 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.transactions;
+
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants;
+
+public class UnwatchExecutor extends TransactionExecutor {
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), RedisConstants.ERROR_UNWATCH));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/WatchExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/WatchExecutor.java b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/WatchExecutor.java
new file mode 100755
index 0000000..c74862c
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/executor/transactions/WatchExecutor.java
@@ -0,0 +1,31 @@
+/*
+ * 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.transactions;
+
+import org.apache.geode.redis.internal.Command;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisConstants;
+
+public class WatchExecutor extends TransactionExecutor {
+
+  @Override
+  public void executeCommand(Command command, ExecutionHandlerContext context) {
+    command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), RedisConstants.ERROR_WATCH));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/main/java/org/apache/geode/redis/internal/org/apache/hadoop/fs/GlobPattern.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/internal/org/apache/hadoop/fs/GlobPattern.java b/geode-core/src/main/java/org/apache/geode/redis/internal/org/apache/hadoop/fs/GlobPattern.java
new file mode 100644
index 0000000..e7f92bf
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/redis/internal/org/apache/hadoop/fs/GlobPattern.java
@@ -0,0 +1,164 @@
+/**
+ * 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.org.apache.hadoop.fs;
+
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+
+/**
+ * A class for POSIX glob pattern with brace expansions.
+ */
+public class GlobPattern {
+  private static final char BACKSLASH = '\\';
+  private Pattern compiled;
+  private boolean hasWildcard = false;
+
+  /**
+   * Construct the glob pattern object with a glob pattern string
+   * @param globPattern the glob pattern string
+   */
+  public GlobPattern(String globPattern) {
+    set(globPattern);
+  }
+
+  /**
+   * @return the compiled pattern
+   */
+  public Pattern compiled() {
+    return compiled;
+  }
+
+  /**
+   * Compile glob pattern string
+   * @param globPattern the glob pattern
+   * @return the pattern object
+   */
+  public static Pattern compile(String globPattern) {
+    return new GlobPattern(globPattern).compiled();
+  }
+
+  /**
+   * Match input against the compiled glob pattern
+   * @param s input chars
+   * @return true for successful matches
+   */
+  public boolean matches(CharSequence s) {
+    return compiled.matcher(s).matches();
+  }
+
+  /**
+   * Set and compile a glob pattern
+   * @param glob  the glob pattern string
+   */
+  public void set(String glob) {
+    StringBuilder regex = new StringBuilder();
+    int setOpen = 0;
+    int curlyOpen = 0;
+    int len = glob.length();
+    hasWildcard = false;
+
+    for (int i = 0; i < len; i++) {
+      char c = glob.charAt(i);
+
+      switch (c) {
+      case BACKSLASH:
+        if (++i >= len) {
+          error("Missing escaped character", glob, i);
+        }
+        regex.append(c).append(glob.charAt(i));
+        continue;
+      case '.':
+      case '$':
+      case '(':
+      case ')':
+      case '|':
+      case '+':
+        // escape regex special chars that are not glob special chars
+        regex.append(BACKSLASH);
+        break;
+      case '*':
+        regex.append('.');
+        hasWildcard = true;
+        break;
+      case '?':
+        regex.append('.');
+        hasWildcard = true;
+        continue;
+      case '{': // start of a group
+        regex.append("(?:"); // non-capturing
+        curlyOpen++;
+        hasWildcard = true;
+        continue;
+      case ',':
+        regex.append(curlyOpen > 0 ? '|' : c);
+        continue;
+      case '}':
+        if (curlyOpen > 0) {
+          // end of a group
+          curlyOpen--;
+          regex.append(")");
+          continue;
+        }
+        break;
+      case '[':
+        if (setOpen > 0) {
+          error("Unclosed character class", glob, i);
+        }
+        setOpen++;
+        hasWildcard = true;
+        break;
+      case '^': // ^ inside [...] can be unescaped
+        if (setOpen == 0) {
+          regex.append(BACKSLASH);
+        }
+        break;
+      case '!': // [! needs to be translated to [^
+        regex.append(setOpen > 0 && '[' == glob.charAt(i - 1) ? '^' : '!');
+        continue;
+      case ']':
+        // Many set errors like [][] could not be easily detected here,
+        // as []], []-] and [-] are all valid POSIX glob and java regex.
+        // We'll just let the regex compiler do the real work.
+        setOpen = 0;
+        break;
+      default:
+      }
+      regex.append(c);
+    }
+
+    if (setOpen > 0) {
+      error("Unclosed character class", glob, len);
+    }
+    if (curlyOpen > 0) {
+      error("Unclosed group", glob, len);
+    }
+    compiled = Pattern.compile(regex.toString());
+  }
+
+  /**
+   * @return true if this is a wildcard pattern (with special chars)
+   */
+  public boolean hasWildcard() {
+    return hasWildcard;
+  }
+
+  private static void error(String message, String pattern, int pos) {
+    throw new PatternSyntaxException(message, pattern, pos);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/test/java/com/gemstone/gemfire/redis/AuthJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/AuthJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/AuthJUnitTest.java
deleted file mode 100644
index 0241a04..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/AuthJUnitTest.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * 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 com.gemstone.gemfire.redis;
-
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.GemFireCache;
-import com.gemstone.gemfire.distributed.ConfigurationProperties;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import redis.clients.jedis.Jedis;
-import redis.clients.jedis.exceptions.JedisDataException;
-
-import java.io.IOException;
-import java.util.Random;
-
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-@Category(IntegrationTest.class)
-public class AuthJUnitTest {
-
-  private static final String PASSWORD = "pwd";
-  Jedis jedis;
-  GemFireRedisServer server;
-  GemFireCache cache;
-  Random rand;
-  int port;
-
-  int runs = 150;
-
-  @Before
-  public void setUp() throws IOException {
-    rand = new Random();
-    port = AvailablePortHelper.getRandomAvailableTCPPort();
-    this.jedis = new Jedis("localhost", port, 100000);
-  }
-
-  @After
-  public void tearDown() throws InterruptedException {
-    server.shutdown();
-    cache.close();
-  }
-  private void setupCacheWithPassword() {
-    CacheFactory cf = new CacheFactory();
-    cf.set(LOG_LEVEL, "error");
-    cf.set(MCAST_PORT, "0");
-    cf.set(LOCATORS, "");
-    cf.set(ConfigurationProperties.REDIS_PASSWORD, PASSWORD);
-    cache = cf.create();
-    server = new GemFireRedisServer("localhost", port);
-    server.start();
-  }
-
-  @Test
-  public void testAuthConfig() {
-    setupCacheWithPassword();
-    InternalDistributedSystem iD = (InternalDistributedSystem) cache.getDistributedSystem();
-    assert(iD.getConfig().getRedisPassword().equals(PASSWORD));
-  }
-
-  @Test
-  public void testAuthRejectAccept() {
-    setupCacheWithPassword();
-    Exception ex = null;
-    try {                        
-      jedis.auth("wrongpwd");
-    } catch (JedisDataException e) {
-      ex = e;
-    }
-    assertNotNull(ex);
-
-    String res = jedis.auth(PASSWORD);
-    assertEquals(res, "OK");
-  }
-
-  @Test
-  public void testAuthNoPwd() {
-    CacheFactory cf = new CacheFactory();
-    cf.set(LOG_LEVEL, "error");
-    cf.set(MCAST_PORT, "0");
-    cf.set(LOCATORS, "");
-    cache = cf.create();
-    server = new GemFireRedisServer("localhost", port);
-    server.start();
-
-    Exception ex = null;
-    try {                        
-      jedis.auth(PASSWORD);
-    } catch (JedisDataException e) {
-      ex = e;
-    }
-    assertNotNull(ex);
-  }
-
-  @Test
-  public void testAuthAcceptRequests() {
-    setupCacheWithPassword();
-    Exception ex = null;
-    try {                        
-      jedis.set("foo", "bar");
-    } catch (JedisDataException e) {
-      ex = e;
-    }
-    assertNotNull(ex);
-
-    String res = jedis.auth(PASSWORD);
-    assertEquals(res, "OK");
-
-    jedis.set("foo", "bar"); // No exception
-  }
-
-  @Test
-  public void testSeparateClientRequests() {
-    setupCacheWithPassword();
-    Jedis authorizedJedis = null;
-    Jedis nonAuthorizedJedis = null;
-    try {
-      authorizedJedis =  new Jedis("localhost", port, 100000);
-      nonAuthorizedJedis = new Jedis("localhost", port, 100000);
-      String res = authorizedJedis.auth(PASSWORD);
-      assertEquals(res, "OK");
-      authorizedJedis.set("foo", "bar"); // No exception for authorized client
-
-      authorizedJedis.auth(PASSWORD);
-      Exception ex = null;
-      try {                        
-        nonAuthorizedJedis.set("foo", "bar");
-      } catch (JedisDataException e) {
-        ex = e;
-      }
-      assertNotNull(ex);
-    } finally {
-      if (authorizedJedis != null)
-        authorizedJedis.close();
-      if (nonAuthorizedJedis != null)
-        nonAuthorizedJedis.close();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java
deleted file mode 100644
index 4538ca6..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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 com.gemstone.gemfire.redis;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.contrib.java.lang.system.RestoreSystemProperties;
-import org.junit.experimental.categories.Category;
-
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.LOCATORS;
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.MCAST_PORT;
-import static org.junit.Assert.assertFalse;
-
-@Category(IntegrationTest.class)
-public class ConcurrentStartTest {
-
-  private Cache cache;
-  private int numServers = 10;
-  
-  @Rule
-  public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
-  
-  @Before
-  public void setUp() {
-    System.setProperty(DistributedSystem.PROPERTIES_FILE_PROPERTY, getClass().getSimpleName() + ".properties");
-  }
-
-  @After
-  public void tearDown() {
-    if (this.cache != null) {
-      this.cache.close();
-      this.cache = null;
-    }
-  }
-  
-  @Test
-  public void testCachelessStart() throws InterruptedException {
-    runNServers(numServers);
-    GemFireCacheImpl.getInstance().close();
-  }
-  
-  @Test
-  public void testCachefulStart() throws InterruptedException {
-    CacheFactory cf = new CacheFactory();
-    cf.set(MCAST_PORT, "0");
-    cf.set(LOCATORS, "");
-    this.cache = cf.create();
-    
-    runNServers(numServers);
-  }
-  
-  private void runNServers(int n) throws InterruptedException {
-    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(numServers);
-    final Thread[] threads = new Thread[n];
-    for (int i = 0; i < n; i++) {
-      final int j = i;
-      Runnable r = new Runnable() {
-
-        @Override
-        public void run() {
-          GemFireRedisServer s = new GemFireRedisServer(ports[j]);
-          s.start();
-          s.shutdown();
-        }
-      };
-      
-      Thread t = new Thread(r);
-      t.setDaemon(true);
-      t.start();
-      threads[i] = t;
-    }
-    for (Thread t : threads)
-      t.join();
-    this.cache = GemFireCacheImpl.getInstance();
-    assertFalse(this.cache.isClosed());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/test/java/com/gemstone/gemfire/redis/HashesJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/HashesJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/HashesJUnitTest.java
deleted file mode 100755
index 351de74..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/HashesJUnitTest.java
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * 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 com.gemstone.gemfire.redis;
-
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.GemFireCache;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import redis.clients.jedis.Jedis;
-
-import java.io.IOException;
-import java.util.*;
-
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-@Category(IntegrationTest.class)
-public class HashesJUnitTest {
-  private static Jedis jedis;
-  private static GemFireRedisServer server;
-  private static GemFireCache cache;
-  private static Random rand;
-  private static int port = 6379;
-
-  @BeforeClass
-  public static void setUp() throws IOException {
-    rand = new Random();
-    CacheFactory cf = new CacheFactory();
-    //cf.set("log-file", "redis.log");
-    cf.set(LOG_LEVEL, "error");
-    cf.set(MCAST_PORT, "0");
-    cf.set(LOCATORS, "");
-    cache = cf.create();
-    port = AvailablePortHelper.getRandomAvailableTCPPort();
-    server = new GemFireRedisServer("localhost", port);
-
-    server.start();
-    jedis = new Jedis("localhost", port, 10000000);
-  }
-
-  @Test
-  public void testHMSetHSetHLen() {
-    int num = 10;
-    String key = randString();
-    Map<String, String> hash = new HashMap<String, String>();
-    for (int i = 0; i < num; i++) {
-      hash.put(randString(), randString());
-    }
-    String response = jedis.hmset(key, hash);
-    assertTrue(response.equals("OK"));
-    assertEquals(new Long(hash.size()), jedis.hlen(key));
-
-    key = randString();
-    hash = new HashMap<String, String>();
-    for (int i = 0; i < num; i++) {
-      hash.put(randString(), randString());
-    }
-    Set<String> keys = hash.keySet();
-    Long count = 1L;
-    for (String field: keys) {
-      Long res = jedis.hset(key, field, hash.get(field));
-      assertTrue(res == 1L);
-      assertEquals(count++, jedis.hlen(key));
-    }
-  }
-
-  @Test
-  public void testHMGetHDelHGetAllHVals() {
-    String key = randString();
-    Map<String, String> hash = new HashMap<String, String>();
-    for (int i = 0; i < 10; i++) {
-      String m = randString();
-      String f = randString();
-      hash.put(m, f);
-    }
-    jedis.hmset(key, hash);
-    Set<String> keys = hash.keySet();
-    String[] keyArray = keys.toArray(new String[keys.size()]);
-    List<String> retList = jedis.hmget(key, keyArray);
-
-    for (int i = 0; i < keys.size(); i++) {
-      assertEquals(retList.get(i), hash.get(keyArray[i]));
-    }
-
-    Map<String, String> retMap = jedis.hgetAll(key);
-
-    assertEquals(retMap, hash);
-
-    List<String> retVals = jedis.hvals(key);
-    Set<String> retSet = new HashSet<String>(retVals);
-
-    assertTrue(retSet.containsAll(hash.values()));
-
-    jedis.hdel(key, keyArray);
-    assertTrue(jedis.hlen(key) == 0);
-  }
-
-  @Test
-  public void testHkeys() {
-    String key = randString();
-    Map<String, String> hash = new HashMap<String, String>();
-    for (int i = 0; i < 10; i++) {
-      hash.put(randString(), randString());
-    }
-    String response = jedis.hmset(key, hash);
-
-    Set<String> keys = hash.keySet();
-    Set<String> retSet = jedis.hkeys(key);
-
-    assertTrue(retSet.containsAll(keys));
-  }
-
-  @Test
-  public void testHIncrBy() {
-    String key = randString();
-    String field = randString();
-
-    Long incr = (long) rand.nextInt(50);
-    if (incr == 0)
-      incr++;
-
-    long response1 = jedis.hincrBy(key, field, incr);
-    assertTrue(response1 == incr);
-
-    long response2 = jedis.hincrBy(randString(), randString(), incr);
-    assertTrue(response2 == incr);
-
-    long response3 = jedis.hincrBy(key, field, incr);
-    assertTrue(response3 == 2*incr);
-
-
-    String field1 = randString();
-    Exception ex = null;
-    try {
-      jedis.hincrBy(key, field1, Long.MAX_VALUE);
-      jedis.hincrBy(key, field1, incr);
-    } catch (Exception e) {
-      ex = e;
-    }
-
-    assertNotNull(ex);
-  }
-
-  private String randString() {
-    int length = rand.nextInt(8) + 5;
-    StringBuilder rString = new StringBuilder();
-    for (int i = 0; i < length; i++)
-      rString.append((char) (rand.nextInt(57) + 65));
-    return rString.toString();
-    //return Long.toHexString(Double.doubleToLongBits(Math.random()));
-  }
-
-  @After
-  public void flushAll() {
-    jedis.flushAll();
-  }
-
-  @AfterClass
-  public static void tearDown() {
-    jedis.close();
-    cache.close();
-    server.shutdown();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dfd481e0/geode-core/src/test/java/com/gemstone/gemfire/redis/ListsJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/redis/ListsJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/redis/ListsJUnitTest.java
deleted file mode 100755
index fe4134c..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/redis/ListsJUnitTest.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/*
- * 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 com.gemstone.gemfire.redis;
-
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.GemFireCache;
-import com.gemstone.gemfire.internal.AvailablePortHelper;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import redis.clients.jedis.Jedis;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
-import static org.junit.Assert.*;
-
-@Category(IntegrationTest.class)
-public class ListsJUnitTest {
-
-  private static Jedis jedis;
-  private static GemFireRedisServer server;
-  private static GemFireCache cache;
-  private static Random rand;
-  private static int port = 6379;
-
-  @BeforeClass
-  public static void setUp() throws IOException {
-    rand = new Random();
-    CacheFactory cf = new CacheFactory();
-    //cf.set("log-file", "redis.log");
-    cf.set(LOG_LEVEL, "error");
-    cf.set(MCAST_PORT, "0");
-    cf.set(LOCATORS, "");
-    cache = cf.create();
-    port = AvailablePortHelper.getRandomAvailableTCPPort();
-    server = new GemFireRedisServer("localhost", port);
-
-    server.start();
-    jedis = new Jedis("localhost", port, 10000000);
-  }
-
-  @Test
-  public void testLindex() {
-    int elements = 50;
-    ArrayList<String> strings = new ArrayList<String>();
-    String key = randString();
-    for (int i = 0; i < elements; i++) {
-      String elem = randString();
-      strings.add(elem);
-    }
-    String[] stringArray = strings.toArray(new String[strings.size()]);
-    jedis.rpush(key, stringArray);
-    
-
-    for (int i = 0; i < elements; i++) {
-      String gemString = jedis.lindex(key, i);
-      String s = strings.get(i);
-      assertEquals(gemString, s);
-    }
-  }
-
-  @Test
-  public void testLPopRPush() {
-    int elements = 50;
-    ArrayList<String> strings = new ArrayList<String>();
-    String key = randString();
-    for (int i = 0; i < elements; i++) {
-      String elem = randString();
-      strings.add(elem);
-    }
-    String[] stringArray = strings.toArray(new String[strings.size()]);
-    jedis.rpush(key, stringArray);
-
-    for (int i = 0; i < elements; i++) {
-      String gemString = jedis.lpop(key);
-      String s = strings.get(i);
-      assertEquals(s, gemString);
-    }
-  }
-
-  @Test
-  public void testRPopLPush() {
-    int elements = 500;
-    ArrayList<String> strings = new ArrayList<String>();
-    String key = randString();
-    for (int i = 0; i < elements; i++) {
-      String elem = randString();
-      strings.add(elem);
-    }
-    String[] stringArray = strings.toArray(new String[strings.size()]);
-    jedis.lpush(key, stringArray);
-
-    for (int i = 0; i < elements; i++) {
-      String gemString = jedis.rpop(key);
-      String s = strings.get(i);
-      assertEquals(gemString, s);
-    }
-
-  }
-
-  @Test
-  public void testLRange() {
-    int elements = 10;
-    ArrayList<String> strings = new ArrayList<String>();
-    String key = randString();
-    for (int i = 0; i < elements; i++) {
-      String elem = randString();
-      strings.add(elem);
-    }
-    String[] stringArray = strings.toArray(new String[strings.size()]);
-    jedis.rpush(key, stringArray);
-
-    for (int i = 0; i < elements; i++) {
-      List<String> range = jedis.lrange(key, 0, i);
-      assertEquals(range, strings.subList(0, i+1));
-    }
-
-    for (int i = 0; i < elements; i++) {
-      List<String> range = jedis.lrange(key, i, -1);
-      assertEquals(range, strings.subList(i, strings.size()));
-    }
-  }
-
-  @Test
-  public void testLTrim() {
-    int elements = 5;
-    ArrayList<String> strings = new ArrayList<String>();
-    String key = randString();
-    for (int i = 0; i < elements; i++) {
-      String elem = randString();
-      strings.add(elem);
-    }
-    String[] stringArray = strings.toArray(new String[strings.size()]);
-    jedis.rpush(key, stringArray);
-    // Take off last element one at a time
-    for (int i = elements - 1; i >= 0; i--) {
-      jedis.ltrim(key, 0, i);
-      List<String> range = jedis.lrange(key, 0, -1);
-      assertEquals(range, strings.subList(0, i+1));
-    }
-    jedis.rpop(key);
-    jedis.rpush(key, stringArray);
-    // Take off first element one at a time
-    for (int i = 1; i < elements; i++) {
-      jedis.ltrim(key, 1, -1);
-      List<String> range = jedis.lrange(key, 0, -1);
-      List<String> expected = strings.subList(i, strings.size());
-      assertEquals(range, expected);
-    }
-  }
-
-  @Test
-  public void testLRPushX() {
-    String key = randString();
-    String otherKey = "Other key";
-    jedis.lpush(key, randString());
-    assertTrue(jedis.lpushx(key, randString()) > 0);
-    assertTrue(jedis.rpushx(key, randString()) > 0);
-
-    assertTrue(jedis.lpushx(otherKey, randString()) == 0);
-    assertTrue(jedis.rpushx(otherKey, randString()) == 0);
-
-    jedis.del(key);
-
-    assertTrue(jedis.lpushx(key, randString()) == 0);
-    assertTrue(jedis.rpushx(key, randString()) == 0);
-  }
-
-  @Test
-  public void testLRem() {
-    int elements = 5;
-    ArrayList<String> strings = new ArrayList<String>();
-    String key = randString();
-    for (int i = 0; i < elements; i++) {
-      String elem = randString();
-      strings.add(elem);
-    }
-    String[] stringArray = strings.toArray(new String[strings.size()]);
-    jedis.rpush(key, stringArray);
-
-    for (int i = 0; i < elements; i++) {
-      String remove = strings.remove(0);
-      jedis.lrem(key, 0, remove);
-      List<String> range = jedis.lrange(key, 0, -1);
-      assertEquals(strings, range);
-    }
-  }
-
-  @Test
-  public void testLSet() {
-    int elements = 10;
-    ArrayList<String> strings = new ArrayList<String>();
-    String key = randString();
-    for (int i = 0; i < elements; i++) {
-      String elem = randString();
-      strings.add(elem);
-    }
-    String[] stringArray = strings.toArray(new String[strings.size()]);
-    jedis.rpush(key, stringArray);
-
-    for (int i = 0; i < elements; i++) {
-      String s = randString();
-      strings.set(i, s);
-      jedis.lset(key, i, s);
-      List<String> range = jedis.lrange(key, 0, -1);
-      assertEquals(range, strings);
-    }
-  }
-
-  private String randString() {
-    int length = rand.nextInt(8) + 5;
-    StringBuilder rString = new StringBuilder();
-    for (int i = 0; i < length; i++)
-      rString.append((char) (rand.nextInt(57) + 65));
-    //return rString.toString();
-    return Long.toHexString(Double.doubleToLongBits(Math.random()));
-  }
-
-  @After
-  public void flushAll() {
-    jedis.flushAll();
-  }
-
-  @AfterClass
-  public static void tearDown() {
-    jedis.close();
-    cache.close();
-    server.shutdown();
-  }
-}