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 2015/09/25 00:44:48 UTC

[11/21] incubator-geode git commit: Fix for GEODE-109

Fix for GEODE-109

1) To fix this floating meta data problem for the redis list implementation, this meta data has been moved into the list region itself
2) Sorted set query failures are fixed by using fully qualified names and also depends on GEODE-146
3) Concurrent region creation/destruction logic completely changed to use dedicated locks for synchronization, fixed distributed deadlock
4) Added integration tests
5) Add javadocs where necessary

This closes #13


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/7d2296bb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/7d2296bb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/7d2296bb

Branch: refs/heads/feature/GEODE-189
Commit: 7d2296bb0409a3618a90969ab58503a41694d8a3
Parents: 7d99f4a
Author: Vito Gavrilov <vg...@pivotal.io>
Authored: Tue Jul 28 09:40:42 2015 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Thu Aug 13 11:39:43 2015 -0700

----------------------------------------------------------------------
 gemfire-assembly/build.gradle                   |   3 +-
 .../internal/DistributionConfigImpl.java        |   1 -
 .../internal/redis/ByteArrayWrapper.java        |  11 +-
 .../internal/redis/ByteToCommandDecoder.java    |  57 +-
 .../gemstone/gemfire/internal/redis/Coder.java  |  24 +-
 .../gemfire/internal/redis/Command.java         |  25 +-
 .../gemfire/internal/redis/DoubleWrapper.java   |  14 +-
 .../internal/redis/ExecutionHandlerContext.java | 139 +++--
 .../gemfire/internal/redis/Executor.java        |   2 +-
 .../gemfire/internal/redis/Extendable.java      |   6 +-
 .../gemfire/internal/redis/RedisConstants.java  |   2 +-
 .../gemfire/internal/redis/RedisDataType.java   |   9 -
 .../gemfire/internal/redis/RegionCache.java     | 410 --------------
 .../internal/redis/RegionCreationException.java |   9 +-
 .../gemfire/internal/redis/RegionProvider.java  | 531 +++++++++++++++++++
 .../redis/executor/AbstractExecutor.java        |  12 +-
 .../redis/executor/AbstractScanExecutor.java    |   2 +-
 .../internal/redis/executor/DBSizeExecutor.java |   2 +-
 .../internal/redis/executor/DelExecutor.java    |   2 +-
 .../internal/redis/executor/EchoExecutor.java   |   2 +-
 .../internal/redis/executor/ExistsExecutor.java |   2 +-
 .../redis/executor/ExpirationExecutor.java      |   6 +-
 .../redis/executor/ExpireAtExecutor.java        |   4 +-
 .../internal/redis/executor/ExpireExecutor.java |   4 +-
 .../redis/executor/FlushAllExecutor.java        |  15 +-
 .../internal/redis/executor/KeysExecutor.java   |   7 +-
 .../internal/redis/executor/ListQuery.java      |  12 +-
 .../redis/executor/PersistExecutor.java         |   2 +-
 .../internal/redis/executor/PingExecutor.java   |   2 +-
 .../internal/redis/executor/QuitExecutor.java   |   2 +-
 .../internal/redis/executor/ScanExecutor.java   |   4 +-
 .../internal/redis/executor/SortedSetQuery.java |  36 +-
 .../internal/redis/executor/TTLExecutor.java    |   4 +-
 .../internal/redis/executor/TypeExecutor.java   |   2 +-
 .../internal/redis/executor/UnkownExecutor.java |   2 +-
 .../redis/executor/hash/HDelExecutor.java       |   2 +-
 .../redis/executor/hash/HGetAllExecutor.java    |   7 +-
 .../redis/executor/hash/HKeysExecutor.java      |   7 +-
 .../redis/executor/hash/HScanExecutor.java      |   5 +-
 .../redis/executor/hash/HValsExecutor.java      |   9 +-
 .../redis/executor/hash/HashExecutor.java       |   4 +-
 .../redis/executor/hll/HllExecutor.java         |   2 +-
 .../redis/executor/hll/PFAddExecutor.java       |   2 +-
 .../redis/executor/hll/PFCountExecutor.java     |   2 +-
 .../redis/executor/hll/PFMergeExecutor.java     |   4 +-
 .../redis/executor/list/LIndexExecutor.java     |   4 +-
 .../redis/executor/list/LLenExecutor.java       |   2 +-
 .../redis/executor/list/LRangeExecutor.java     |  10 +-
 .../redis/executor/list/LRemExecutor.java       |   4 +-
 .../redis/executor/list/LSetExecutor.java       |   4 +-
 .../redis/executor/list/LTrimExecutor.java      |  26 +-
 .../redis/executor/list/ListExecutor.java       |  26 +-
 .../redis/executor/list/PopExecutor.java        |  23 +-
 .../redis/executor/list/PushExecutor.java       |   2 +-
 .../redis/executor/list/PushXExecutor.java      |   2 +-
 .../org/apache/hadoop/fs/GlobPattern.java       | 164 ------
 .../redis/executor/set/SAddExecutor.java        |   2 +-
 .../redis/executor/set/SCardExecutor.java       |   2 +-
 .../redis/executor/set/SIsMemberExecutor.java   |   2 +-
 .../redis/executor/set/SMembersExecutor.java    |   7 +-
 .../redis/executor/set/SMoveExecutor.java       |   2 +-
 .../redis/executor/set/SPopExecutor.java        |   4 +-
 .../redis/executor/set/SRandMemberExecutor.java |   4 +-
 .../redis/executor/set/SRemExecutor.java        |   2 +-
 .../redis/executor/set/SScanExecutor.java       |   4 +-
 .../redis/executor/set/SetOpExecutor.java       |   9 +-
 .../executor/sortedset/SortedSetExecutor.java   |   4 +-
 .../executor/sortedset/ZRangeByLexExecutor.java |   2 -
 .../sortedset/ZRangeByScoreExecutor.java        |   3 +-
 .../redis/executor/sortedset/ZRemExecutor.java  |   2 +-
 .../sortedset/ZRemRangeByLexExecutor.java       |   3 +-
 .../sortedset/ZRemRangeByRankExecutor.java      |   4 +-
 .../sortedset/ZRemRangeByScoreExecutor.java     |   4 +-
 .../redis/executor/sortedset/ZScanExecutor.java |   5 +-
 .../redis/executor/string/AppendExecutor.java   |   2 +-
 .../redis/executor/string/BitCountExecutor.java |   2 +-
 .../redis/executor/string/BitOpExecutor.java    |   2 +-
 .../redis/executor/string/BitPosExecutor.java   |   2 +-
 .../redis/executor/string/DecrByExecutor.java   |   2 +-
 .../redis/executor/string/DecrExecutor.java     |   4 +-
 .../redis/executor/string/GetBitExecutor.java   |   2 +-
 .../redis/executor/string/GetExecutor.java      |   2 +-
 .../redis/executor/string/GetRangeExecutor.java |   2 +-
 .../redis/executor/string/GetSetExecutor.java   |   2 +-
 .../redis/executor/string/IncrByExecutor.java   |   2 +-
 .../executor/string/IncrByFloatExecutor.java    |   2 +-
 .../redis/executor/string/IncrExecutor.java     |   2 +-
 .../redis/executor/string/MGetExecutor.java     |   2 +-
 .../redis/executor/string/MSetExecutor.java     |   2 +-
 .../redis/executor/string/MSetNXExecutor.java   |   2 +-
 .../redis/executor/string/SetBitExecutor.java   |   2 +-
 .../redis/executor/string/SetEXExecutor.java    |   4 +-
 .../redis/executor/string/SetExecutor.java      |   4 +-
 .../redis/executor/string/SetNXExecutor.java    |   2 +-
 .../redis/executor/string/SetRangeExecutor.java |   4 +-
 .../redis/executor/string/StringExecutor.java   |   4 +-
 .../redis/executor/string/StrlenExecutor.java   |   2 +-
 .../redis/org/apache/hadoop/fs/GlobPattern.java | 164 ++++++
 .../gemfire/redis/GemFireRedisServer.java       | 130 ++---
 .../gemfire/redis/ConcurrentStartTest.java      |  58 ++
 .../gemstone/gemfire/redis/HashesJUnitTest.java | 175 ++++++
 .../gemstone/gemfire/redis/ListsJUnitTest.java  | 238 +++++++++
 .../gemfire/redis/RedisDistDUnitTest.java       | 231 ++++++++
 .../gemstone/gemfire/redis/SetsJUnitTest.java   | 242 +++++++++
 .../gemfire/redis/SortedSetsJUnitTest.java      | 414 +++++++++++++++
 .../gemfire/redis/StringsJunitTest.java         | 296 +++++++++++
 106 files changed, 2789 insertions(+), 958 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-assembly/build.gradle b/gemfire-assembly/build.gradle
index 0e51563..f65930d 100755
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@ -112,7 +112,8 @@ def cp = {
       it.contains('spring-core') ||
       it.contains('spring-shell') ||
       it.contains('snappy-java') ||
-      it.contains('hbase')
+      it.contains('hbase') ||
+      it.contains('netty')
     }.join(' ') 
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
index b8dfeb3..3707ff3 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
@@ -30,7 +30,6 @@ import com.gemstone.gemfire.internal.ConfigSource;
 import com.gemstone.gemfire.internal.SocketCreator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.process.ProcessLauncherContext;
-import com.gemstone.gemfire.internal.redis.Coder;
 import com.gemstone.gemfire.memcached.GemFireMemcachedServer;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ByteArrayWrapper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ByteArrayWrapper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ByteArrayWrapper.java
index 2d617da..fb2bc21 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ByteArrayWrapper.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ByteArrayWrapper.java
@@ -12,7 +12,7 @@ import com.gemstone.gemfire.DataSerializer;
  * This class is a wrapper for the any Regions that need to store a 
  * byte[]. The only data this an instance will store is a byte[]
  * for the data but it is also serializable and comparable so it is able to be used
- * in querying. The hash code and to string variant are created lazily
+ * in querying
  * 
  * @author Vitaliy Gavrilov
  *
@@ -97,6 +97,13 @@ public class ByteArrayWrapper implements DataSerializable, Comparable<ByteArrayW
     return this.hashCode;
   }
 
+
+  /**
+   * This equals is neither symmetric and therefore not transitive, 
+   * because a String with the same underlying bytes is considered
+   * equal. Clearly calling {@link String#equals(Object)) would not 
+   * yield the same result
+   */
   @Override
   public boolean equals(Object other) {
     if (other instanceof ByteArrayWrapper)
@@ -109,7 +116,7 @@ public class ByteArrayWrapper implements DataSerializable, Comparable<ByteArrayW
 
   /**
    * This is a byte to byte comparator, it is not lexicographical but purely compares
-   * byte values
+   * byte by byte values
    */
   @Override
   public int compareTo(ByteArrayWrapper other) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ByteToCommandDecoder.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ByteToCommandDecoder.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ByteToCommandDecoder.java
index 441ab06..7e307fe 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ByteToCommandDecoder.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ByteToCommandDecoder.java
@@ -4,14 +4,37 @@ import io.netty.buffer.ByteBuf;
 import io.netty.channel.ChannelHandlerContext;
 import io.netty.handler.codec.ByteToMessageDecoder;
 
-import java.nio.BufferUnderflowException;
 import java.util.ArrayList;
 import java.util.List;
 
-
+/**
+ * This is the first part of the channel pipeline for Netty. Here incoming
+ * bytes are read and a created {@link Command} is sent down the pipeline.
+ * It is unfortunate that this class is not {@link Sharable} because no state
+ * is kept in this class. State is kept by {@link ByteToMessageDecoder}, it may
+ * be worthwhile to look at a different decoder setup as to avoid allocating a decoder
+ * for every new connection.
+ * <p>
+ * The code flow of the protocol parsing may not be exactly Java like, but this is done 
+ * very intentionally. It was found that in cases where large Redis requests are sent
+ * that end up being fragmented, throwing exceptions when the command could not be fully
+ * parsed took up an enormous amount of cpu time. The simplicity of the Redis protocol
+ * allows us to just back out and wait for more data, while exceptions are left to 
+ * malformed requests which should never happen if using a proper Redis client.
+ * 
+ * @author Vitaliy Gavrilov
+ *
+ */
 public class ByteToCommandDecoder extends ByteToMessageDecoder {
 
-
+  /**
+   * Important note
+   * 
+   * Do not use '' <-- java primitive chars. Redis uses {@link Coder#CHARSET}
+   * encoding so we should not risk java handling char to byte conversions, rather 
+   * just hard code {@link Coder#CHARSET} chars as bytes
+   */
+  
   private static final byte rID = 13; // '\r';
   private static final byte nID = 10; // '\n';
   private static final byte bulkStringID = 36; // '$';
@@ -35,18 +58,7 @@ public class ByteToCommandDecoder extends ByteToMessageDecoder {
     } while (in.isReadable()); // Try to take advantage of pipelining if it is being used
   }
 
-  /**
-   * The only public method for CommandParser is parse. It will take a buffer
-   * and break up the individual pieces into a list is char[] for the caller
-   * based on the Redis protocol.
-   * 
-   * @param buffer The buffer to read the command from
-   * @return A new {@link Command} object
-   * @throws RedisCommandParserException Thrown when the command has illegal syntax
-   * @throws BufferUnderflowException Thrown when the parser runs out of chars
-   * to read when it still expects chars to remain in the command
-   */
-  public static Command parse(ByteBuf buffer) throws RedisCommandParserException {
+  private Command parse(ByteBuf buffer) throws RedisCommandParserException {
     if (buffer == null)
       throw new NullPointerException();
     if (!buffer.isReadable())
@@ -63,14 +75,7 @@ public class ByteToCommandDecoder extends ByteToMessageDecoder {
     return new Command(commandElems);
   }
 
-  /**
-   * Helper method to parse the array which contains the Redis command
-   * 
-   * @param commandElems The list to add the elements of the command to
-   * @param buffer The buffer to read from
-   * @throws RedisCommandParserException Thrown when command contains illegal syntax
-   */
-  private static boolean parseArray(ArrayList<byte[]> commandElems, ByteBuf buffer) throws RedisCommandParserException { 
+  private boolean parseArray(ArrayList<byte[]> commandElems, ByteBuf buffer) throws RedisCommandParserException { 
     byte currentChar;
     int arrayLength = parseCurrentNumber(buffer);
     if (arrayLength == Integer.MIN_VALUE || !parseRN(buffer))
@@ -100,7 +105,7 @@ public class ByteToCommandDecoder extends ByteToMessageDecoder {
    * @return byte[] representation of the Bulk String read
    * @throws RedisCommandParserException Thrown when there is illegal syntax
    */
-  private static byte[] parseBulkString(ByteBuf buffer) throws RedisCommandParserException {
+  private byte[] parseBulkString(ByteBuf buffer) throws RedisCommandParserException {
     int bulkStringLength = parseCurrentNumber(buffer);
     if (bulkStringLength == Integer.MIN_VALUE)
       return null;
@@ -126,7 +131,7 @@ public class ByteToCommandDecoder extends ByteToMessageDecoder {
    * @param buffer Buffer to read
    * @return The number found at the beginning of the buffer
    */
-  private static int parseCurrentNumber(ByteBuf buffer) {
+  private int parseCurrentNumber(ByteBuf buffer) {
     int number = 0;
     int readerIndex = buffer.readerIndex();
     byte b = 0;
@@ -153,7 +158,7 @@ public class ByteToCommandDecoder extends ByteToMessageDecoder {
    * @throws RedisCommandParserException Thrown when the next two characters
    * are not "\r\n"
    */
-  private static boolean parseRN(ByteBuf buffer) throws RedisCommandParserException {
+  private boolean parseRN(ByteBuf buffer) throws RedisCommandParserException {
     if (!buffer.isReadable(2))
       return false;
     byte b = buffer.readByte();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Coder.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Coder.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Coder.java
index 4792eec..9415cd3 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Coder.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Coder.java
@@ -10,10 +10,15 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
 
 import com.gemstone.gemfire.cache.query.Struct;
 
+/**
+ * This is a safe encoder and decoder for all redis matching needs
+ * 
+ * @author Vitaliy Gavrilov
+ *
+ */
 public class Coder {
 
   /*
@@ -121,23 +126,6 @@ public class Coder {
     return response;
   }
 
-  public static final ByteBuf getBulkStringArrayResponse(ByteBufAllocator alloc, Set<String> items) {
-    Iterator<String> it = items.iterator();
-    ByteBuf response = alloc.buffer();
-    response.writeByte(ARRAY_ID);
-    response.writeBytes(intToBytes(items.size()));
-    response.writeBytes(CRLFar);
-    while(it.hasNext()) {
-      String next = it.next();
-      response.writeByte(BULK_STRING_ID);
-      response.writeBytes(intToBytes(next.length()));
-      response.writeBytes(CRLFar);
-      response.writeBytes(stringToBytes(next));
-      response.writeBytes(CRLFar);
-    }
-    return response;
-  }
-
   public static final ByteBuf getBulkStringArrayResponse(ByteBufAllocator alloc, List<String> items) {
     Iterator<String> it = items.iterator();
     ByteBuf response = alloc.buffer();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Command.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Command.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Command.java
index 368a5d7..ae5f3f7 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Command.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Command.java
@@ -9,16 +9,6 @@ import java.util.List;
  * The command class is used in holding a received Redis command. Each sent 
  * command resides in an instance of this class. This class is designed to be
  * used strictly by getter and setter methods.
- * <p>
- * The contents include the following
- * <p>
- * <li>The channel from which the command was read and to where
- * the response will be written.</li>
- * <li>The list of elements in the command. Every command is sent as an array of BulkStrings.
- * For example if the command "SADD key member1 member2 member3" was sent, the list would contain
- * {"SADD", "key", "member1", "member2", "member3"}.
- * <li>The {@link RedisCommandType}</li>
- * <li>The response to be written back to the client</li>
  * 
  * @author Vitaliy Gavrilov
  *
@@ -113,10 +103,11 @@ public class Command {
    */
   public String getStringKey() {
     if (this.commandElems.size() > 1) {
-      if (this.key == null) {
+      if (this.bytes == null) {
         this.bytes = new ByteArrayWrapper(this.commandElems.get(1));
         this.key = this.bytes.toString();
-      }
+      } else if (this.key == null)
+        this.key = this.bytes.toString();
       return this.key;
     } else 
       return null;
@@ -130,4 +121,14 @@ public class Command {
     } else 
       return null;
   }
+
+  @Override
+  public String toString() {
+    StringBuilder b = new StringBuilder();
+    for (byte[] bs : this.commandElems) {
+      b.append(Coder.bytesToString(bs));
+      b.append(' ');
+    }
+    return b.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/DoubleWrapper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/DoubleWrapper.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/DoubleWrapper.java
index f2b4e50..ed5fb99 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/DoubleWrapper.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/DoubleWrapper.java
@@ -7,6 +7,12 @@ import java.io.IOException;
 import com.gemstone.gemfire.DataSerializable;
 import com.gemstone.gemfire.DataSerializer;
 
+/**
+ * This is a wrapper class for doubles, similar to {@link ByteArrayWrapper}
+ * 
+ * @author Vitaliy Gavrilov
+ *
+ */
 public class DoubleWrapper implements DataSerializable, Comparable<Object> {
 
   private static final long serialVersionUID = 6946858357297398633L;
@@ -35,14 +41,6 @@ public class DoubleWrapper implements DataSerializable, Comparable<Object> {
     Double other;
     if (arg0 instanceof DoubleWrapper) {
       other = ((DoubleWrapper) arg0).score;
-    } else if (arg0 instanceof String) {
-      String arg = (String) arg0;
-      if (arg.equalsIgnoreCase("INFINITY"))
-        other = Double.POSITIVE_INFINITY;
-      else if (arg.equalsIgnoreCase("-INFINITY"))
-        other = Double.NEGATIVE_INFINITY;
-      else
-        return 0;
     } else if (arg0 instanceof Double) {
       other = (Double) arg0;
     } else

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ExecutionHandlerContext.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ExecutionHandlerContext.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ExecutionHandlerContext.java
index 5f51c35..cf20ea8 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ExecutionHandlerContext.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/ExecutionHandlerContext.java
@@ -15,17 +15,33 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheClosedException;
 import com.gemstone.gemfire.cache.CacheTransactionManager;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.TransactionException;
 import com.gemstone.gemfire.cache.TransactionId;
 import com.gemstone.gemfire.cache.UnsupportedOperationInTransactionException;
+import com.gemstone.gemfire.cache.query.QueryInvocationTargetException;
 import com.gemstone.gemfire.internal.redis.executor.transactions.TransactionExecutor;
 import com.gemstone.gemfire.redis.GemFireRedisServer;
 
+/**
+ * This class extends {@link ChannelInboundHandlerAdapter} from Netty and it is
+ * the last part of the channel pipeline. The {@link ByteToCommandDecoder} forwards a
+ * {@link Command} to this class which executes it and sends the result back to the
+ * client. Additionally, all exception handling is done by this class. 
+ * <p>
+ * Besides being part of Netty's pipeline, this class also serves as a context to the
+ * execution of a command. It abstracts transactions, provides access to the {@link RegionProvider}
+ * and anything else an executing {@link Command} may need.
+ * 
+ * @author Vitaliy Gavrilov
+ *
+ */
 public class ExecutionHandlerContext extends ChannelInboundHandlerAdapter {
 
-  private static final int MAXIMUM_NUM_RETRIES = 5;
+  private static final int WAIT_REGION_DSTRYD_MILLIS = 100;
+  private static final int MAXIMUM_NUM_RETRIES = (1000*60)/WAIT_REGION_DSTRYD_MILLIS; // 60 seconds total
 
   private final Cache cache;
   private final GemFireRedisServer server;
@@ -44,12 +60,23 @@ public class ExecutionHandlerContext extends ChannelInboundHandlerAdapter {
    * Queue of commands for a given transaction
    */
   private Queue<Command> transactionQueue;
-  private final RegionCache regionCache;
+  private final RegionProvider regionProvider;
   private final byte[] authPwd;
 
   private boolean isAuthenticated;
 
-  public ExecutionHandlerContext(Channel ch, Cache cache, RegionCache regions, GemFireRedisServer server, byte[] pwd) {
+  /**
+   * Default constructor for execution contexts. 
+   * 
+   * @param ch Channel used by this context, should be one to one
+   * @param cache The Geode cache instance of this vm
+   * @param regionProvider The region provider of this context
+   * @param server Instance of the server it is attached to, only used so that any execution can initiate a shutdwon
+   * @param pwd Authentication password for each context, can be null
+   */
+  public ExecutionHandlerContext(Channel ch, Cache cache, RegionProvider regionProvider, GemFireRedisServer server, byte[] pwd) {
+    if (ch == null || cache == null || regionProvider == null || server == null)
+      throw new IllegalArgumentException("Only the authentication password may be null");
     this.cache = cache;
     this.server = server;
     this.logger = cache.getLogger();
@@ -67,7 +94,7 @@ public class ExecutionHandlerContext extends ChannelInboundHandlerAdapter {
     this.byteBufAllocator = channel.alloc();
     this.transactionID = null;
     this.transactionQueue = null; // Lazy
-    this.regionCache = regions;
+    this.regionProvider = regionProvider;
     this.authPwd = pwd;
     this.isAuthenticated = pwd != null ? false : true;
   }
@@ -78,19 +105,25 @@ public class ExecutionHandlerContext extends ChannelInboundHandlerAdapter {
     }
   }
 
-  private void writeToChannel(Object message) {
+  private void writeToChannel(ByteBuf message) {
     channel.write(message, channel.voidPromise());
     if (!needChannelFlush.getAndSet(true)) {
       this.lastExecutor.execute(flusher);
     }
   }
 
+  /**
+   * This will handle the execution of received commands
+   */
   @Override
   public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
     Command command = (Command) msg;
     executeCommand(ctx, command);
   }
 
+  /**
+   * Exception handler for the entire pipeline
+   */
   @Override
   public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) {
     if (cause instanceof IOException) {
@@ -107,15 +140,16 @@ public class ExecutionHandlerContext extends ChannelInboundHandlerAdapter {
       response = Coder.getWrongTypeResponse(this.byteBufAllocator, cause.getMessage());
     else if (cause instanceof DecoderException && cause.getCause() instanceof RedisCommandParserException)
       response = Coder.getErrorResponse(this.byteBufAllocator, RedisConstants.PARSING_EXCEPTION_MESSAGE);
-    else if (cause instanceof RegionCreationException)
+    else if (cause instanceof RegionCreationException) {
+      this.logger.error(cause);
       response = Coder.getErrorResponse(this.byteBufAllocator, RedisConstants.ERROR_REGION_CREATION);
-    else if (cause instanceof InterruptedException)
+    } else if (cause instanceof InterruptedException || cause instanceof CacheClosedException)
       response = Coder.getErrorResponse(this.byteBufAllocator, RedisConstants.SERVER_ERROR_SHUTDOWN);
     else if (cause instanceof IllegalStateException) {
-      response = Coder.getErrorResponse(this.byteBufAllocator,  cause.getMessage());
+      response = Coder.getErrorResponse(this.byteBufAllocator, cause.getMessage());
     } else {
       if (this.logger.errorEnabled())
-        this.logger.error("GemFireRedisServer-Unexpected error handler", cause);
+        this.logger.error("GemFireRedisServer-Unexpected error handler for " + ctx.channel(), cause);
       response = Coder.getErrorResponse(this.byteBufAllocator, RedisConstants.SERVER_ERROR_MESSAGE);
     }
     return response;
@@ -129,17 +163,7 @@ public class ExecutionHandlerContext extends ChannelInboundHandlerAdapter {
     ctx.close();
   }
 
-  /**
-   * This method is used to execute the command. The executor is 
-   * determined by the {@link RedisCommandType} and then the execution
-   * is started.
-   * 
-   * @param command Command to be executed
-   * @param cache The Cache instance of this server
-   * @param client The client data associated with the client
-   * @throws Exception 
-   */
-  public void executeCommand(ChannelHandlerContext ctx, Command command) throws Exception {
+  private void executeCommand(ChannelHandlerContext ctx, Command command) throws Exception {
     RedisCommandType type = command.getCommandType();
     Executor exec = type.getExecutor();
     if (isAuthenticated) {
@@ -150,7 +174,7 @@ public class ExecutionHandlerContext extends ChannelInboundHandlerAdapter {
       if (hasTransaction() && !(exec instanceof TransactionExecutor))
         executeWithTransaction(ctx, exec, command);
       else
-        executeWithoutTransaction(exec, command, MAXIMUM_NUM_RETRIES); 
+        executeWithoutTransaction(exec, command); 
 
       if (hasTransaction() && command.getCommandType() != RedisCommandType.MULTI) {
         writeToChannel(Coder.getSimpleStringResponse(this.byteBufAllocator, RedisConstants.COMMAND_QUEUED));
@@ -179,31 +203,23 @@ public class ExecutionHandlerContext extends ChannelInboundHandlerAdapter {
    * 
    * @param exec Executor to use
    * @param command Command to execute
-   * @param cache Cache instance
-   * @param client Client data associated with client
-   * @param n Recursive max depth of retries
    * @throws Exception Throws exception if exception is from within execution and not to be handled
    */
-  private void executeWithoutTransaction(final Executor exec, Command command, int n) throws Exception {
-    try {
-      exec.executeCommand(command, this);
-    } catch (RegionDestroyedException e) {
-      if (n > 0)
-        executeWithoutTransaction(exec, command, n - 1);
-      else
-        throw e;
+  private void executeWithoutTransaction(final Executor exec, Command command) throws Exception {
+    Exception cause = null;
+    for (int i = 0; i < MAXIMUM_NUM_RETRIES; i++) {
+      try {
+        exec.executeCommand(command, this);
+        return;
+      } catch (Exception e) {
+        cause = e;
+        if (e instanceof RegionDestroyedException || e.getCause() instanceof QueryInvocationTargetException)
+          Thread.sleep(WAIT_REGION_DSTRYD_MILLIS);
+      }
     }
+    throw cause;
   }
 
-  /**
-   * Private method to execute a command when a transaction has been started
-   * 
-   * @param exec Executor to use
-   * @param command Command to execute
-   * @param cache Cache instance
-   * @param client Client data associated with client
-   * @throws Exception Throws exception if exception is from within execution and unrelated to transactions
-   */
   private void executeWithTransaction(ChannelHandlerContext ctx, final Executor exec, Command command) throws Exception {
     CacheTransactionManager txm = cache.getCacheTransactionManager();
     TransactionId transactionId = getTransactionID();
@@ -276,34 +292,71 @@ public class ExecutionHandlerContext extends ChannelInboundHandlerAdapter {
     return this.transactionQueue;
   }
 
+  /**
+   * {@link ByteBuf} allocator for this context. All executors
+   * must use this pooled allocator as opposed to having unpooled buffers
+   * for maximum performance
+   * 
+   * @return allocator instance
+   */
   public ByteBufAllocator getByteBufAllocator() {
     return this.byteBufAllocator;
   }
 
-  public RegionCache getRegionCache() {
-    return this.regionCache;
+  /**
+   * Gets the provider of Regions
+   * 
+   * @return Provider
+   */
+  public RegionProvider getRegionProvider() {
+    return this.regionProvider;
   }
 
+  /**
+   * Getter for manager to allow pausing and resuming transactions
+   * @return Instance
+   */
   public CacheTransactionManager getCacheTransactionManager() {
     return this.cache.getCacheTransactionManager();
   }
 
+  /**
+   * Getter for logger
+   * @return instance
+   */
   public LogWriter getLogger() {
     return this.cache.getLogger();
   }
 
+  /**
+   * Get the channel for this context
+   * @return instance
+   *
   public Channel getChannel() {
     return this.channel;
   }
+   */
 
+  /**
+   * Get the authentication password, this will be same server wide.
+   * It is exposed here as opposed to {@link GemFireRedisServer}.
+   * @return password
+   */
   public byte[] getAuthPwd() {
     return this.authPwd;
   }
 
+  /**
+   * Checker if user has authenticated themselves
+   * @return True if no authentication required or authentication complete, false otherwise
+   */
   public boolean isAuthenticated() {
     return this.isAuthenticated;
   }
 
+  /**
+   * Lets this context know the authentication is complete
+   */
   public void setAuthenticationVerified() {
     this.isAuthenticated = true;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Executor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Executor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Executor.java
index d993589..304d19c 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Executor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Executor.java
@@ -15,7 +15,7 @@ public interface Executor {
    * a response
    * 
    * @param command The command to be executed
-   * @param context The client data associated with the client
+   * @param context The execution context by which this command is to be executed
    */
   public void executeCommand(Command command, ExecutionHandlerContext context);
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Extendable.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Extendable.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Extendable.java
index 889e5b2..f4dd122 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Extendable.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/Extendable.java
@@ -1,7 +1,7 @@
 package com.gemstone.gemfire.internal.redis;
 
 /**
- * This defines a command that can be extended,d there may need some level of abstraction
+ * This defines a command that can be extended, and there may need some level of abstraction
  * 
  * @author Vitaliy Gavrilov
  *
@@ -9,8 +9,8 @@ package com.gemstone.gemfire.internal.redis;
 public interface Extendable {
 
   /**
-   * 
-   * @return
+   * Getter for error message in case of argument arity mismatch
+   * @return Error string
    */
   public String getArgsError();
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RedisConstants.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RedisConstants.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RedisConstants.java
index 6fe77d7..9325820 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RedisConstants.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RedisConstants.java
@@ -3,7 +3,7 @@ package com.gemstone.gemfire.internal.redis;
 
 public class RedisConstants {
 
-  public static final int NUM_DEFAULT_KEYS = 4;
+  public static final int NUM_DEFAULT_KEYS = 3;
 
   /*
    * Responses

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RedisDataType.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RedisDataType.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RedisDataType.java
index 38febe2..febd371 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RedisDataType.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RedisDataType.java
@@ -23,7 +23,6 @@ public enum RedisDataType {
    * Strings Regions
    */
   REDIS_STRING {
-
     @Override
     public String toString() {
       return "string";
@@ -34,7 +33,6 @@ public enum RedisDataType {
    * Hashes Regions
    */
   REDIS_HASH {
-
     @Override
     public String toString() {
       return "hash";
@@ -45,7 +43,6 @@ public enum RedisDataType {
    * Lists Regions
    */
   REDIS_LIST {
-
     @Override
     public String toString() {
       return "list";
@@ -56,7 +53,6 @@ public enum RedisDataType {
    * Sets Regions
    */
   REDIS_SET {
-    
     @Override
     public String toString() {
       return "set";
@@ -67,7 +63,6 @@ public enum RedisDataType {
    * SortedSets Regions
    */
   REDIS_SORTEDSET {
-    
     @Override
     public String toString() {
       return "zset";
@@ -78,7 +73,6 @@ public enum RedisDataType {
    * HyperLogLog Regions
    */
   REDIS_HLL {
-    
     @Override
     public String toString() {
       return "hyperloglog";
@@ -89,7 +83,6 @@ public enum RedisDataType {
    * Regions protected from overwrite or deletion
    */
   REDIS_PROTECTED {
-    
     @Override
     public String toString() {
       return "protected";
@@ -100,8 +93,6 @@ public enum RedisDataType {
    * None
    */
   NONE {
-    
-    
     @Override
     public String toString() {
       return "none";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionCache.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionCache.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionCache.java
deleted file mode 100644
index 68f50ba..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionCache.java
+++ /dev/null
@@ -1,410 +0,0 @@
-package com.gemstone.gemfire.internal.redis;
-
-import java.io.Closeable;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-
-import com.gemstone.gemfire.LogWriter;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheTransactionManager;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.TransactionId;
-import com.gemstone.gemfire.cache.query.IndexNameConflictException;
-import com.gemstone.gemfire.cache.query.Query;
-import com.gemstone.gemfire.cache.query.QueryService;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.redis.executor.ExpirationExecutor;
-import com.gemstone.gemfire.internal.redis.executor.ListQuery;
-import com.gemstone.gemfire.internal.redis.executor.SortedSetQuery;
-import com.gemstone.gemfire.internal.redis.executor.hll.HyperLogLogPlus;
-import com.gemstone.gemfire.management.cli.Result;
-import com.gemstone.gemfire.management.cli.Result.Status;
-import com.gemstone.gemfire.management.internal.cli.commands.CreateAlterDestroyRegionCommands;
-import com.gemstone.gemfire.redis.GemFireRedisServer;
-
-
-public class RegionCache implements Closeable {
-
-  private final ConcurrentHashMap<ByteArrayWrapper, Region<?, ?>> regions;
-
-  /**
-   * This is the Redis meta data {@link Region} that holds the {@link RedisDataType}
-   * information for all Regions created. The mapping is a {@link String} key which is the name
-   * of the {@link Region} created to hold the data to the RedisDataType it contains.
-   */
-  private final Region<String, RedisDataType> redisMetaRegion;
-
-  /**
-   * This is the {@link RedisDataType#REDIS_LIST} meta data {@link Region}. Here each list
-   * stores the index of the head and tail of the list
-   */
-  private final Region<String, Integer> listsMetaRegion;
-
-  /**
-   * This is the {@link RedisDataType#REDIS_STRING} {@link Region}. This is the Region
-   * that stores all string contents
-   */
-  private final Region<ByteArrayWrapper, ByteArrayWrapper> stringsRegion;
-
-  /**
-   * This is the {@link RedisDataType#REDIS_HLL} {@link Region}. This is the Region
-   * that stores all HyperLogLog contents
-   */
-  private final Region<ByteArrayWrapper, HyperLogLogPlus> hLLRegion;
-
-  private final Cache cache;
-  private final QueryService queryService;
-  private final ConcurrentMap<ByteArrayWrapper, Map<Enum<?>, Query>> preparedQueries = new ConcurrentHashMap<ByteArrayWrapper, Map<Enum<?>, Query>>();
-  private final ConcurrentMap<ByteArrayWrapper, ScheduledFuture<?>> expirationsMap;
-  private final ScheduledExecutorService expirationExecutor;
-  private static final CreateAlterDestroyRegionCommands cliCmds = new CreateAlterDestroyRegionCommands();
-
-  public RegionCache(Region<ByteArrayWrapper, ByteArrayWrapper> stringsRegion, Region<ByteArrayWrapper, HyperLogLogPlus> hLLRegion, Region<String, RedisDataType> redisMetaRegion, Region<String, Integer> listsMetaRegion, ConcurrentMap<ByteArrayWrapper, ScheduledFuture<?>> expirationsMap, ScheduledExecutorService expirationExecutor) {
-    if (stringsRegion == null || hLLRegion == null || redisMetaRegion == null || listsMetaRegion == null)
-      throw new NullPointerException();
-    this.regions = new ConcurrentHashMap<ByteArrayWrapper, Region<?, ?>>();
-    this.stringsRegion = stringsRegion;
-    this.hLLRegion = hLLRegion;
-    this.redisMetaRegion = redisMetaRegion;
-    this.listsMetaRegion = listsMetaRegion;
-    this.cache = GemFireCacheImpl.getInstance();
-    this.queryService = cache.getQueryService();
-    this.expirationsMap = expirationsMap;
-    this.expirationExecutor = expirationExecutor;
-  }
-
-  public boolean existsKey(ByteArrayWrapper key) {
-    return this.redisMetaRegion.containsKey(key.toString());
-  }
-
-  public Set<String> metaKeySet() {
-    return this.redisMetaRegion.keySet();
-  }
-
-  public Set<Map.Entry<String, RedisDataType>> metaEntrySet() {
-    return this.redisMetaRegion.entrySet();
-  }
-
-  public int getMetaSize() {
-    return this.redisMetaRegion.size() - RedisConstants.NUM_DEFAULT_KEYS;
-  }
-
-  private boolean metaRemoveEntry(ByteArrayWrapper key) {
-    return this.redisMetaRegion.remove(key.toString()) != null;
-  }
-
-  public RedisDataType metaPutIfAbsent(ByteArrayWrapper key, RedisDataType value) {
-    return this.redisMetaRegion.putIfAbsent(key.toString(), value);
-  }
-
-  public RedisDataType metaPut(ByteArrayWrapper key, RedisDataType value) {
-    return this.redisMetaRegion.put(key.toString(), value);
-  }
-
-  public RedisDataType metaGet(ByteArrayWrapper key) {
-    return this.redisMetaRegion.get(key.toString());
-  }
-
-  public Region<?, ?> getRegion(ByteArrayWrapper key) {
-    return this.regions.get(key);
-  }
-
-  public void removeRegionReferenceLocally(ByteArrayWrapper key, RedisDataType type) {
-    cancelKeyExpiration(key);
-    this.regions.remove(key);
-  }
-
-  public boolean removeKey(ByteArrayWrapper key) {
-    RedisDataType type = getRedisDataType(key);
-    return removeKey(key, type);
-  }
-
-  public boolean removeKey(ByteArrayWrapper key, RedisDataType type) {
-    return removeKey(key, type, true);
-  }
-
-  public boolean removeKey(ByteArrayWrapper key, RedisDataType type, boolean cancelExpiration) {
-    if (type == null || type == RedisDataType.REDIS_PROTECTED)
-      return false;
-    metaRemoveEntry(key);
-    try {
-      if (type == RedisDataType.REDIS_STRING) {
-        return this.stringsRegion.remove(key) != null;
-      } else if (type == RedisDataType.REDIS_HLL) {
-        return this.hLLRegion.remove(key) != null;
-      } else {
-        return destroyRegion(key, type);
-      }
-    } catch (Exception exc) {
-      return false;
-    } finally {
-      if (cancelExpiration)
-        cancelKeyExpiration(key);
-      else
-        removeKeyExpiration(key);
-    }
-  }
-
-  public Region<?, ?> getOrCreateRegion(ByteArrayWrapper key, RedisDataType type, ExecutionHandlerContext context) {
-    return getOrCreateRegion0(key, type, context, true);
-  }
-
-  public Region<?, ?> createRemoteRegionLocally(ByteArrayWrapper key, RedisDataType type) {
-    return getOrCreateRegion0(key, type, null, false);
-  }
-
-  private Region<?, ?> getOrCreateRegion0(ByteArrayWrapper key, RedisDataType type, ExecutionHandlerContext context, boolean addToMeta) {
-    checkDataType(key, type);
-    Region<?, ?> r = this.regions.get(key);
-    if (r == null) {
-      String stringKey = key.toString();
-      synchronized (stringKey) { // This object will be interned across the vm
-        r = regions.get(key);
-        if (r == null) {
-          boolean hasTransaction = context != null && context.hasTransaction(); // Can create without context
-          CacheTransactionManager txm = null;
-          TransactionId transactionId = null;
-          try {
-            if (hasTransaction) {
-              txm = cache.getCacheTransactionManager();
-              transactionId = txm.suspend();
-            }
-            r = createRegionGlobally(stringKey);
-            if (addToMeta)
-              metaPut(key, type);
-            if (type == RedisDataType.REDIS_LIST)
-              doInitializeList(key, r.getFullPath());
-            else if (type == RedisDataType.REDIS_SORTEDSET)
-              doInitializeSortedSet(key, r);
-            this.regions.put(key, r);
-          } finally {
-            if (hasTransaction)
-              txm.resume(transactionId);
-          }
-        }
-      }
-    }
-    return r;
-  }
-
-  private boolean destroyRegion(ByteArrayWrapper key, RedisDataType type) {
-    String stringKey = key.toString();
-    Region<?, ?> r = this.regions.get(key);
-    if (r != null) {
-      synchronized (stringKey) { // This object will be interned across the vm
-        try {
-          r.destroyRegion();
-        } catch (Exception e) {
-          return false;
-        } finally {
-          this.preparedQueries.remove(key);
-          metaRemoveEntry(key);
-          if (type == RedisDataType.REDIS_LIST) {
-            this.listsMetaRegion.remove(stringKey + "head");
-            this.listsMetaRegion.remove(stringKey + "tail");
-          }
-          this.regions.remove(key);
-        }
-      }
-    }
-    return true;
-  }
-
-  private void doInitializeSortedSet(ByteArrayWrapper key, Region<?, ?> r) {
-    String fullpath = r.getFullPath();
-    try {
-      queryService.createIndex("scoreIndex", "value.score", r.getFullPath() + ".entrySet entry");
-      queryService.createIndex("scoreIndex2", "value.score", r.getFullPath() + ".values value");
-    } catch (Exception e) {
-      if (!(e instanceof IndexNameConflictException)) {
-        LogWriter logger = cache.getLogger();
-        if (logger.errorEnabled()) {
-          logger.error(e);
-        }
-      }
-    }
-    HashMap<Enum<?>, Query> queryList = new HashMap<Enum<?>, Query>();
-    for (SortedSetQuery lq: SortedSetQuery.values()) {
-      String queryString = lq.getQueryString(fullpath);
-      Query query = this.queryService.newQuery(queryString);
-      queryList.put(lq, query);
-    }
-    this.preparedQueries.put(key, queryList);
-  }
-
-  private void doInitializeList(ByteArrayWrapper key, String fullpath) {
-    listsMetaRegion.put(key + "head", Integer.valueOf(0));
-    listsMetaRegion.put(key + "tail", Integer.valueOf(0));
-    HashMap<Enum<?>, Query> queryList = new HashMap<Enum<?>, Query>();
-    for (ListQuery lq: ListQuery.values()) {
-      String queryString = lq.getQueryString(fullpath);
-      Query query = this.queryService.newQuery(queryString);
-      queryList.put(lq, query);
-    }
-    this.preparedQueries.put(key, queryList);
-  }
-
-  /**
-   * This method creates a Region globally with the given name. If
-   * there is an error in the creation, a runtime exception will
-   * be thrown.
-   * 
-   * @param key Name of Region to create
-   * @return Region Region created globally
-   */
-  private Region<?, ?> createRegionGlobally(String key) {
-    Region<?, ?> r = null;
-    Result result = cliCmds.createRegion(key, GemFireRedisServer.DEFAULT_REGION_TYPE, null, null, true, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null);
-    r = cache.getRegion(key);
-    if (result.getStatus() == Status.ERROR && r == null) {
-      String err = "";
-      while(result.hasNextLine())
-        err += result.nextLine();
-      throw new RegionCreationException(err);
-    }
-    if (r == null)
-      throw new RegionCreationException();
-    return r;
-  }
-
-  public Query getQuery(ByteArrayWrapper key, Enum<?> query) {
-    return this.preparedQueries.get(key).get(query);
-  }
-
-  /**
-   * Checks if the given key is associated with the passed data type.
-   * If there is a mismatch, a {@link RuntimeException} is thrown
-   * 
-   * @param key Key to check
-   * @param type Type to check to
-   */
-  protected void checkDataType(ByteArrayWrapper key, RedisDataType type) {
-    RedisDataType currentType = redisMetaRegion.get(key.toString());
-    if (currentType == null)
-      return;
-    if (currentType == RedisDataType.REDIS_PROTECTED)
-      throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is protected");
-    if (currentType != type)
-      throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is already used by a " + currentType.toString());
-  }
-
-  public boolean regionExists(ByteArrayWrapper key) {
-    return this.regions.containsKey(key);
-  }
-
-  public Region<ByteArrayWrapper, ByteArrayWrapper> getStringsRegion() {
-    return this.stringsRegion;
-  }
-
-  public Region<String, Integer> getListsMetaRegion() {
-    return this.listsMetaRegion;
-  }
-
-  public Region<ByteArrayWrapper, HyperLogLogPlus> gethLLRegion() {
-    return this.hLLRegion;
-  }
-
-  private RedisDataType getRedisDataType(String key) {
-    return this.redisMetaRegion.get(key);
-  }
-
-  public RedisDataType getRedisDataType(ByteArrayWrapper key) {
-    return getRedisDataType(key.toString());
-  }
-
-
-  /**
-   * Sets the expiration for a key. The setting and modifying of a key expiration can only be set by a delay,
-   * which means that both expiring after a time and at a time can be done but
-   * the delay to expire at a time must be calculated before these calls. It is
-   * also important to note that the delay is always handled in milliseconds
-   * 
-   * @param key The key to set the expiration for
-   * @param delay The delay in milliseconds of the expiration
-   * @return True is expiration set, false otherwise
-   */
-  public final boolean setExpiration(ByteArrayWrapper key, long delay) {
-    RedisDataType type = getRedisDataType(key);
-    if (type == null)
-      return false;
-    ScheduledFuture<?> future = this.expirationExecutor.schedule(new ExpirationExecutor(key, type, this), delay, TimeUnit.MILLISECONDS);
-    this.expirationsMap.put(key, future);
-    return true;
-  }
-
-  /**
-   * Modifies an expiration on a key
-   * 
-   * @param key String key to modify expiration on
-   * @param delay Delay in milliseconds to reset the expiration to
-   * @return True if reset, false if not
-   */
-  public final boolean modifyExpiration(ByteArrayWrapper key, long delay) {
-    /*
-     * Attempt to cancel future task
-     */
-    boolean canceled = cancelKeyExpiration(key);
-
-    if (!canceled)
-      return false;
-
-    RedisDataType type = getRedisDataType(key);
-    if (type == null)
-      return false;
-
-    ScheduledFuture<?> future = this.expirationExecutor.schedule(new ExpirationExecutor(key, type, this), delay, TimeUnit.MILLISECONDS);
-    this.expirationsMap.put(key, future);
-    return true;
-  }
-
-  /**
-   * Removes an expiration from a key
-   * 
-   * @param key Key
-   * @param context Context
-   * @return True is expiration cancelled on the key, false otherwise
-   */
-  public final boolean cancelKeyExpiration(ByteArrayWrapper key) {
-    ScheduledFuture<?> future = expirationsMap.remove(key);
-    if (future == null)
-      return false;
-    return future.cancel(false);
-  }
-
-  private boolean removeKeyExpiration(ByteArrayWrapper key) {
-    return expirationsMap.remove(key) != null;
-  }
-
-  /**
-   * Check method if key has expiration
-   * 
-   * @param key Key
-   * @return True if key has expiration, false otherwise
-   */
-  public boolean hasExpiration(ByteArrayWrapper key) {
-    return this.expirationsMap.containsKey(key);
-  }
-
-  /**
-   * Get remaining expiration time
-   * 
-   * @param key Key
-   * @return Remaining time in milliseconds or 0 if no delay or key doesn't exist
-   */
-  public final long getExpirationDelayMillis(ByteArrayWrapper key) {
-    ScheduledFuture<?> future = this.expirationsMap.get(key);
-    return future != null ? future.getDelay(TimeUnit.MILLISECONDS) : 0L;
-  }
-
-  @Override
-  public void close() {
-    this.preparedQueries.clear();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionCreationException.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionCreationException.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionCreationException.java
index 55fb701..5856a0c 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionCreationException.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionCreationException.java
@@ -1,5 +1,11 @@
 package com.gemstone.gemfire.internal.redis;
 
+/**
+ * This exception is used when an error happens while creating a {@link Region} globally
+ * 
+ * @author Vitaliy Gavrilov
+ *
+ */
 public class RegionCreationException extends RuntimeException {
 
   public RegionCreationException() {}
@@ -12,9 +18,6 @@ public class RegionCreationException extends RuntimeException {
     super(err, cause);
   }
 
-  /**
-   * 
-   */
   private static final long serialVersionUID = 8416820139078312997L;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionProvider.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionProvider.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionProvider.java
new file mode 100644
index 0000000..a95f853
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionProvider.java
@@ -0,0 +1,531 @@
+package com.gemstone.gemfire.internal.redis;
+
+import java.io.Closeable;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheTransactionManager;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.TransactionId;
+import com.gemstone.gemfire.cache.query.IndexNameConflictException;
+import com.gemstone.gemfire.cache.query.Query;
+import com.gemstone.gemfire.cache.query.QueryInvalidException;
+import com.gemstone.gemfire.cache.query.QueryService;
+import com.gemstone.gemfire.cache.query.RegionNotFoundException;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.redis.executor.ExpirationExecutor;
+import com.gemstone.gemfire.internal.redis.executor.ListQuery;
+import com.gemstone.gemfire.internal.redis.executor.SortedSetQuery;
+import com.gemstone.gemfire.internal.redis.executor.hll.HyperLogLogPlus;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.cli.Result.Status;
+import com.gemstone.gemfire.management.internal.cli.commands.CreateAlterDestroyRegionCommands;
+import com.gemstone.gemfire.redis.GemFireRedisServer;
+
+/**
+ * This class stands between {@link Executor} and {@link Cache#getRegion(String)}.
+ * This is needed because some keys for Redis represented as a {@link Region} in
+ * {@link GemFireRedisServer} come with additional state. Therefore getting, creating,
+ * or destroying a {@link Region} needs to be synchronized, which is done away with
+ * and abstracted by this class.
+ * 
+ * @author Vitaly Gavrilov
+ *
+ */
+public class RegionProvider implements Closeable {
+
+  private final ConcurrentHashMap<ByteArrayWrapper, Region<?, ?>> regions;
+
+  /**
+   * This is the Redis meta data {@link Region} that holds the {@link RedisDataType}
+   * information for all Regions created. The mapping is a {@link String} key which is the name
+   * of the {@link Region} created to hold the data to the RedisDataType it contains.
+   */
+  private final Region<String, RedisDataType> redisMetaRegion;
+
+  /**
+   * This is the {@link RedisDataType#REDIS_STRING} {@link Region}. This is the Region
+   * that stores all string contents
+   */
+  private final Region<ByteArrayWrapper, ByteArrayWrapper> stringsRegion;
+
+  /**
+   * This is the {@link RedisDataType#REDIS_HLL} {@link Region}. This is the Region
+   * that stores all HyperLogLog contents
+   */
+  private final Region<ByteArrayWrapper, HyperLogLogPlus> hLLRegion;
+
+  private final Cache cache;
+  private final QueryService queryService;
+  private final ConcurrentMap<ByteArrayWrapper, Map<Enum<?>, Query>> preparedQueries = new ConcurrentHashMap<ByteArrayWrapper, Map<Enum<?>, Query>>();
+  private final ConcurrentMap<ByteArrayWrapper, ScheduledFuture<?>> expirationsMap;
+  private final ScheduledExecutorService expirationExecutor;
+  private final RegionShortcut defaultRegionType;
+  private static final CreateAlterDestroyRegionCommands cliCmds = new CreateAlterDestroyRegionCommands();
+  private final ConcurrentHashMap<String, Lock> locks;
+  private final LogWriter logger;
+
+  public RegionProvider(Region<ByteArrayWrapper, ByteArrayWrapper> stringsRegion, Region<ByteArrayWrapper, HyperLogLogPlus> hLLRegion, Region<String, RedisDataType> redisMetaRegion, ConcurrentMap<ByteArrayWrapper, ScheduledFuture<?>> expirationsMap, ScheduledExecutorService expirationExecutor, RegionShortcut defaultShortcut) {
+    if (stringsRegion == null || hLLRegion == null || redisMetaRegion == null)
+      throw new NullPointerException();
+    this.regions = new ConcurrentHashMap<ByteArrayWrapper, Region<?, ?>>();
+    this.stringsRegion = stringsRegion;
+    this.hLLRegion = hLLRegion;
+    this.redisMetaRegion = redisMetaRegion;
+    this.cache = GemFireCacheImpl.getInstance();
+    this.queryService = cache.getQueryService();
+    this.expirationsMap = expirationsMap;
+    this.expirationExecutor = expirationExecutor;
+    this.defaultRegionType = defaultShortcut;
+    this.locks = new ConcurrentHashMap<String, Lock>();
+    this.logger = this.cache.getLogger();
+  }
+
+  public boolean existsKey(ByteArrayWrapper key) {
+    return this.redisMetaRegion.containsKey(key.toString());
+  }
+
+  public Set<String> metaKeySet() {
+    return this.redisMetaRegion.keySet();
+  }
+
+  public Set<Map.Entry<String, RedisDataType>> metaEntrySet() {
+    return this.redisMetaRegion.entrySet();
+  }
+
+  public int getMetaSize() {
+    return this.redisMetaRegion.size() - RedisConstants.NUM_DEFAULT_KEYS;
+  }
+
+  private boolean metaRemoveEntry(ByteArrayWrapper key) {
+    return this.redisMetaRegion.remove(key.toString()) != null;
+  }
+
+  public RedisDataType metaPutIfAbsent(ByteArrayWrapper key, RedisDataType value) {
+    return this.redisMetaRegion.putIfAbsent(key.toString(), value);
+  }
+
+  public RedisDataType metaPut(ByteArrayWrapper key, RedisDataType value) {
+    return this.redisMetaRegion.put(key.toString(), value);
+  }
+
+  public RedisDataType metaGet(ByteArrayWrapper key) {
+    return this.redisMetaRegion.get(key.toString());
+  }
+
+  public Region<?, ?> getRegion(ByteArrayWrapper key) {
+    return this.regions.get(key);
+  }
+
+  public void removeRegionReferenceLocally(ByteArrayWrapper key, RedisDataType type) {
+    Lock lock = this.locks.get(key.toString());
+    boolean locked = false;
+    try {
+      locked = lock.tryLock();
+      // If we cannot get the lock we ignore this remote event, this key has local event
+      // that started independently, ignore this event to prevent deadlock
+      if (locked) {
+        cancelKeyExpiration(key);
+        removeRegionState(key, type);
+      }
+    } finally {
+      if (locked) {
+        lock.unlock();
+      }
+    }
+  }
+
+  public boolean removeKey(ByteArrayWrapper key) {
+    RedisDataType type = getRedisDataType(key);
+    return removeKey(key, type);
+  }
+
+  public boolean removeKey(ByteArrayWrapper key, RedisDataType type) {
+    return removeKey(key, type, true);
+  }
+
+  public boolean removeKey(ByteArrayWrapper key, RedisDataType type, boolean cancelExpiration) {
+    if (type == null || type == RedisDataType.REDIS_PROTECTED)
+      return false;
+    Lock lock = this.locks.get(key.toString());
+    try {
+      if (lock != null)  {// Strings/hlls will not have locks
+        lock.lock();
+      }
+      metaRemoveEntry(key);
+      try {
+        if (type == RedisDataType.REDIS_STRING) {
+          return this.stringsRegion.remove(key) != null;
+        } else if (type == RedisDataType.REDIS_HLL) {
+          return this.hLLRegion.remove(key) != null;
+        } else {
+          return destroyRegion(key, type);
+        }
+      } catch (Exception exc) {
+        return false;
+      } finally {
+        if (cancelExpiration)
+          cancelKeyExpiration(key);
+        else
+          removeKeyExpiration(key);
+        if (lock != null)
+          this.locks.remove(key.toString());
+      }
+    } finally {
+      if (lock != null) {
+        lock.unlock();
+      }
+    }
+  }
+
+  public Region<?, ?> getOrCreateRegion(ByteArrayWrapper key, RedisDataType type, ExecutionHandlerContext context) {
+    return getOrCreateRegion0(key, type, context, true);
+  }
+
+  public void createRemoteRegionLocally(ByteArrayWrapper key, RedisDataType type) {
+    if (type == null || type == RedisDataType.REDIS_STRING || type == RedisDataType.REDIS_HLL)
+      return;
+    Region<?, ?> r = this.regions.get(key);
+    if (r != null)
+      return;
+    if (!this.regions.contains(key)) {
+      String stringKey = key.toString();
+      Lock lock = this.locks.get(stringKey);
+      if (lock == null) {
+        this.locks.putIfAbsent(stringKey, new ReentrantLock());
+        lock = this.locks.get(stringKey);
+      }
+      boolean locked = false;
+      try {
+        locked = lock.tryLock();
+        // If we cannot get the lock then this remote even may have been initialized
+        // independently on this machine, so if we wait on the lock it is more than
+        // likely we will deadlock just to do the same task, this even can be ignored
+        if (locked) {
+          r = cache.getRegion(key.toString());
+          if (type == RedisDataType.REDIS_LIST)
+            doInitializeList(key, r);
+          else if (type == RedisDataType.REDIS_SORTEDSET)
+            doInitializeSortedSet(key, r);
+          this.regions.put(key, r);
+        }
+      } finally {
+        if (locked) {
+          lock.unlock();
+        }
+      }
+    }
+  }
+
+  private Region<?, ?> getOrCreateRegion0(ByteArrayWrapper key, RedisDataType type, ExecutionHandlerContext context, boolean addToMeta) {
+    checkDataType(key, type);
+    Region<?, ?> r = this.regions.get(key);
+    if (r != null && r.isDestroyed()) {
+      removeKey(key, type);
+      r = null;
+    }
+    if (r == null) {
+      String stringKey = key.toString();
+      Lock lock = this.locks.get(stringKey);
+      if (lock == null) {
+        this.locks.putIfAbsent(stringKey, new ReentrantLock());
+        lock = this.locks.get(stringKey);
+      }
+
+      try {
+        lock.lock();
+        r = regions.get(key);
+        if (r == null) {
+          boolean hasTransaction = context != null && context.hasTransaction(); // Can create without context
+          CacheTransactionManager txm = null;
+          TransactionId transactionId = null;
+          try {
+            if (hasTransaction) {
+              txm = cache.getCacheTransactionManager();
+              transactionId = txm.suspend();
+            }
+            Exception concurrentCreateDestroyException = null;
+            do {
+              concurrentCreateDestroyException = null;
+              r = createRegionGlobally(stringKey);
+              try {
+                if (type == RedisDataType.REDIS_LIST)
+                  doInitializeList(key, r);
+                else if (type == RedisDataType.REDIS_SORTEDSET)
+                  doInitializeSortedSet(key, r);
+              } catch (QueryInvalidException e) {
+                if (e.getCause() instanceof RegionNotFoundException) {
+                  concurrentCreateDestroyException = e;
+                }
+              }
+            } while(concurrentCreateDestroyException != null);
+            this.regions.put(key, r);            
+            if (addToMeta) {
+              RedisDataType existingType = metaPutIfAbsent(key, type);
+              if (existingType != null && existingType != type)
+                throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is already used by a " + existingType.toString());
+            }
+          } finally {
+            if (hasTransaction)
+              txm.resume(transactionId);
+          }
+        }
+      } finally {
+        lock.unlock();
+      }
+    }
+    return r;
+  }
+
+  /**
+   * SYNCHRONIZE EXTERNALLY OF this.locks.get(key.toString())!!!!!
+   * 
+   * @param key Key of region to destroy
+   * @param type Type of region to destroyu
+   * @return Flag if destroyed
+   */
+  private boolean destroyRegion(ByteArrayWrapper key, RedisDataType type) {
+    Region<?, ?> r = this.regions.get(key);
+    if (r != null) {
+      try {
+        r.destroyRegion();
+      } catch (Exception e) {
+        return false;
+      } finally {
+        removeRegionState(key, type);
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Do not call this method if you are not synchronized on the lock associated with this key
+   * 
+   * @param key Key of region to remove
+   * @param type Type of key to remove all state
+   */
+  private void removeRegionState(ByteArrayWrapper key, RedisDataType type) {
+    this.preparedQueries.remove(key);
+    this.regions.remove(key);
+  }
+
+  private void doInitializeSortedSet(ByteArrayWrapper key, Region<?, ?> r) {
+    String fullpath = r.getFullPath();
+    try {
+      queryService.createIndex("scoreIndex", "entry.value.score", r.getFullPath() + ".entrySet entry");
+      queryService.createIndex("scoreIndex2", "value.score", r.getFullPath() + ".values value");
+    } catch (Exception e) {
+      if (!(e instanceof IndexNameConflictException)) {
+        if (logger.errorEnabled()) {
+          logger.error(e);
+        }
+      }
+    }
+    HashMap<Enum<?>, Query> queryList = new HashMap<Enum<?>, Query>();
+    for (SortedSetQuery lq: SortedSetQuery.values()) {
+      String queryString = lq.getQueryString(fullpath);
+      Query query = this.queryService.newQuery(queryString);
+      queryList.put(lq, query);
+    }
+    this.preparedQueries.put(key, queryList);
+  }
+
+  private void doInitializeList(ByteArrayWrapper key, Region r) {
+    r.put("head", Integer.valueOf(0));
+    r.put("tail", Integer.valueOf(0));
+    String fullpath = r.getFullPath();
+    HashMap<Enum<?>, Query> queryList = new HashMap<Enum<?>, Query>();
+    for (ListQuery lq: ListQuery.values()) {
+      String queryString = lq.getQueryString(fullpath);
+      Query query = this.queryService.newQuery(queryString);
+      queryList.put(lq, query);
+    }
+    this.preparedQueries.put(key, queryList);
+  }
+
+  /**
+   * This method creates a Region globally with the given name. If
+   * there is an error in the creation, a runtime exception will
+   * be thrown.
+   * 
+   * @param key Name of Region to create
+   * @return Region Region created globally
+   */
+  private Region<?, ?> createRegionGlobally(String key) {
+    Region<?, ?> r = null;
+    r = cache.getRegion(key);
+    if (r != null) return r;
+    do {
+      Result result = cliCmds.createRegion(key, defaultRegionType, null, null, true, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null);
+      r = cache.getRegion(key);
+      if (result.getStatus() == Status.ERROR && r == null) {
+        String err = "";
+        while(result.hasNextLine())
+          err += result.nextLine();
+        if (this.logger.errorEnabled()) {
+          this.logger.error("Region creation failure- "+ err);
+        }
+        throw new RegionCreationException(err);
+      }
+    } while(r == null); // The region can be null in the case that it is concurrently destroyed by
+    // a remote even triggered internally by Geode
+    return r;
+  }
+
+  public Query getQuery(ByteArrayWrapper key, Enum<?> query) {
+    return this.preparedQueries.get(key).get(query);
+    /*
+    if (query instanceof ListQuery) {
+      return this.queryService.newQuery(((ListQuery)query).getQueryString(this.regions.get(key).getFullPath()));
+    } else {
+      return this.queryService.newQuery(((SortedSetQuery)query).getQueryString(this.regions.get(key).getFullPath()));
+    }
+    */
+  }
+
+  /**
+   * Checks if the given key is associated with the passed data type.
+   * If there is a mismatch, a {@link RuntimeException} is thrown
+   * 
+   * @param key Key to check
+   * @param type Type to check to
+   */
+  protected void checkDataType(ByteArrayWrapper key, RedisDataType type) {
+    RedisDataType currentType = redisMetaRegion.get(key.toString());
+    if (currentType == null)
+      return;
+    if (currentType == RedisDataType.REDIS_PROTECTED)
+      throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is protected");
+    if (currentType != type)
+      throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is already used by a " + currentType.toString());
+  }
+
+  public boolean regionExists(ByteArrayWrapper key) {
+    return this.regions.containsKey(key);
+  }
+
+  public Region<ByteArrayWrapper, ByteArrayWrapper> getStringsRegion() {
+    return this.stringsRegion;
+  }
+
+  public Region<ByteArrayWrapper, HyperLogLogPlus> gethLLRegion() {
+    return this.hLLRegion;
+  }
+
+  private RedisDataType getRedisDataType(String key) {
+    return this.redisMetaRegion.get(key);
+  }
+
+  public RedisDataType getRedisDataType(ByteArrayWrapper key) {
+    return getRedisDataType(key.toString());
+  }
+
+  /**
+   * Sets the expiration for a key. The setting and modifying of a key expiration can only be set by a delay,
+   * which means that both expiring after a time and at a time can be done but
+   * the delay to expire at a time must be calculated before these calls. It is
+   * also important to note that the delay is always handled in milliseconds
+   * 
+   * @param key The key to set the expiration for
+   * @param delay The delay in milliseconds of the expiration
+   * @return True is expiration set, false otherwise
+   */
+  public final boolean setExpiration(ByteArrayWrapper key, long delay) {
+    RedisDataType type = getRedisDataType(key);
+    if (type == null)
+      return false;
+    ScheduledFuture<?> future = this.expirationExecutor.schedule(new ExpirationExecutor(key, type, this), delay, TimeUnit.MILLISECONDS);
+    this.expirationsMap.put(key, future);
+    return true;
+  }
+
+  /**
+   * Modifies an expiration on a key
+   * 
+   * @param key String key to modify expiration on
+   * @param delay Delay in milliseconds to reset the expiration to
+   * @return True if reset, false if not
+   */
+  public final boolean modifyExpiration(ByteArrayWrapper key, long delay) {
+    /*
+     * Attempt to cancel future task
+     */
+    boolean canceled = cancelKeyExpiration(key);
+
+    if (!canceled)
+      return false;
+
+    RedisDataType type = getRedisDataType(key);
+    if (type == null)
+      return false;
+
+    ScheduledFuture<?> future = this.expirationExecutor.schedule(new ExpirationExecutor(key, type, this), delay, TimeUnit.MILLISECONDS);
+    this.expirationsMap.put(key, future);
+    return true;
+  }
+
+  /**
+   * Removes an expiration from a key
+   * 
+   * @param key Key
+   * @param context Context
+   * @return True is expiration cancelled on the key, false otherwise
+   */
+  public final boolean cancelKeyExpiration(ByteArrayWrapper key) {
+    ScheduledFuture<?> future = expirationsMap.remove(key);
+    if (future == null)
+      return false;
+    return future.cancel(false);
+  }
+
+  private boolean removeKeyExpiration(ByteArrayWrapper key) {
+    return expirationsMap.remove(key) != null;
+  }
+
+  /**
+   * Check method if key has expiration
+   * 
+   * @param key Key
+   * @return True if key has expiration, false otherwise
+   */
+  public boolean hasExpiration(ByteArrayWrapper key) {
+    return this.expirationsMap.containsKey(key);
+  }
+
+  /**
+   * Get remaining expiration time
+   * 
+   * @param key Key
+   * @return Remaining time in milliseconds or 0 if no delay or key doesn't exist
+   */
+  public final long getExpirationDelayMillis(ByteArrayWrapper key) {
+    ScheduledFuture<?> future = this.expirationsMap.get(key);
+    return future != null ? future.getDelay(TimeUnit.MILLISECONDS) : 0L;
+  }
+
+  @Override
+  public void close() {
+    this.preparedQueries.clear();
+  }
+
+  public String dumpRegionsCache() {
+    StringBuilder builder = new StringBuilder();
+    for (Entry<ByteArrayWrapper, Region<?, ?>> e : this.regions.entrySet()) {
+      builder.append(e.getKey() + " --> {" + e.getValue() + "}\n");
+    }
+    return builder.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/AbstractExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/AbstractExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/AbstractExecutor.java
index 97fd4fc..d95d057 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/AbstractExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/AbstractExecutor.java
@@ -7,7 +7,7 @@ import com.gemstone.gemfire.internal.redis.ExecutionHandlerContext;
 import com.gemstone.gemfire.internal.redis.Executor;
 import com.gemstone.gemfire.internal.redis.RedisDataType;
 import com.gemstone.gemfire.internal.redis.RedisDataTypeMismatchException;
-import com.gemstone.gemfire.internal.redis.RegionCache;
+import com.gemstone.gemfire.internal.redis.RegionProvider;
 import com.gemstone.gemfire.redis.GemFireRedisServer;
 
 /**
@@ -22,7 +22,7 @@ public abstract class AbstractExecutor implements Executor {
   /**
    * Number of Regions used by GemFireRedisServer internally
    */
-  public static final int NUM_DEFAULT_REGIONS = 4;
+  public static final int NUM_DEFAULT_REGIONS = 3;
 
   /**
    * Max length of a list
@@ -53,7 +53,7 @@ public abstract class AbstractExecutor implements Executor {
    * @return Region with name key
    */
   protected Region<?, ?> getOrCreateRegion(ExecutionHandlerContext context, ByteArrayWrapper key, RedisDataType type) {
-    return context.getRegionCache().getOrCreateRegion(key, type, context);
+    return context.getRegionProvider().getOrCreateRegion(key, type, context);
   }
 
   /**
@@ -65,7 +65,7 @@ public abstract class AbstractExecutor implements Executor {
    * @param context context
    */
   protected void checkDataType(ByteArrayWrapper key, RedisDataType type, ExecutionHandlerContext context) {
-    RedisDataType currentType = context.getRegionCache().getRedisDataType(key);
+    RedisDataType currentType = context.getRegionProvider().getRedisDataType(key);
     if (currentType == null)
       return;
     if (currentType == RedisDataType.REDIS_PROTECTED)
@@ -83,13 +83,13 @@ public abstract class AbstractExecutor implements Executor {
    * @return The Query of this key and QueryType
    */
   protected Query getQuery(ByteArrayWrapper key, Enum<?> type, ExecutionHandlerContext context) {
-    return context.getRegionCache().getQuery(key, type);
+    return context.getRegionProvider().getQuery(key, type);
   }
 
   protected boolean removeEntry(ByteArrayWrapper key, RedisDataType type, ExecutionHandlerContext context) {
     if (type == null || type == RedisDataType.REDIS_PROTECTED)
       return false;
-    RegionCache rC = context.getRegionCache();
+    RegionProvider rC = context.getRegionProvider();
     return rC.removeKey(key, type);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/AbstractScanExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/AbstractScanExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/AbstractScanExecutor.java
index 09b1d76..e4a9a53 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/AbstractScanExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/AbstractScanExecutor.java
@@ -4,7 +4,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.regex.Pattern;
 
-import com.gemstone.gemfire.internal.redis.executor.org.apache.hadoop.fs.GlobPattern;
+import com.gemstone.gemfire.internal.redis.org.apache.hadoop.fs.GlobPattern;
 
 
 public abstract class AbstractScanExecutor extends AbstractExecutor {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/DBSizeExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/DBSizeExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/DBSizeExecutor.java
index 954e3e0..e434a07 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/DBSizeExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/DBSizeExecutor.java
@@ -8,7 +8,7 @@ public class DBSizeExecutor extends AbstractExecutor {
 
   @Override
   public void executeCommand(Command command, ExecutionHandlerContext context) {
-    int size = context.getRegionCache().getMetaSize();
+    int size = context.getRegionProvider().getMetaSize();
     command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), size));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/DelExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/DelExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/DelExecutor.java
index 28e5f40..7d54883 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/DelExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/DelExecutor.java
@@ -28,7 +28,7 @@ public class DelExecutor extends AbstractExecutor {
     for (int i = 1; i < commandElems.size(); i++) {
       byte[] byteKey = commandElems.get(i);
       ByteArrayWrapper key = new ByteArrayWrapper(byteKey);
-      RedisDataType type = context.getRegionCache().getRedisDataType(key); 
+      RedisDataType type = context.getRegionProvider().getRedisDataType(key); 
       if (removeEntry(key, type, context))
         numRemoved++;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/EchoExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/EchoExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/EchoExecutor.java
index 03fa868..2d0268e 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/EchoExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/EchoExecutor.java
@@ -2,8 +2,8 @@ package com.gemstone.gemfire.internal.redis.executor;
 
 import java.util.List;
 
-import com.gemstone.gemfire.internal.redis.Command;
 import com.gemstone.gemfire.internal.redis.Coder;
+import com.gemstone.gemfire.internal.redis.Command;
 import com.gemstone.gemfire.internal.redis.ExecutionHandlerContext;
 import com.gemstone.gemfire.internal.redis.RedisConstants.ArityDef;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExistsExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExistsExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExistsExecutor.java
index ec8fef6..5adc554 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExistsExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExistsExecutor.java
@@ -22,7 +22,7 @@ public class ExistsExecutor extends AbstractExecutor {
     }
 
     ByteArrayWrapper key = command.getKey();
-    boolean exists = context.getRegionCache().existsKey(key);
+    boolean exists = context.getRegionProvider().existsKey(key);
 
     if (exists)
       command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), EXISTS));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpirationExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpirationExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpirationExecutor.java
index 3ed78d8..9fa7595 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpirationExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpirationExecutor.java
@@ -2,15 +2,15 @@ package com.gemstone.gemfire.internal.redis.executor;
 
 import com.gemstone.gemfire.internal.redis.ByteArrayWrapper;
 import com.gemstone.gemfire.internal.redis.RedisDataType;
-import com.gemstone.gemfire.internal.redis.RegionCache;
+import com.gemstone.gemfire.internal.redis.RegionProvider;
 
 
 public class ExpirationExecutor implements Runnable {
   private final ByteArrayWrapper key;
   private final RedisDataType type;
-  private final RegionCache rC;
+  private final RegionProvider rC;
 
-  public ExpirationExecutor(ByteArrayWrapper k, RedisDataType type, RegionCache rC) {
+  public ExpirationExecutor(ByteArrayWrapper k, RedisDataType type, RegionProvider rC) {
     this.key = k;
     this.type = type;
     this.rC = rC;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpireAtExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpireAtExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpireAtExecutor.java
index 695d41d..7c40ae4 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpireAtExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpireAtExecutor.java
@@ -8,7 +8,7 @@ import com.gemstone.gemfire.internal.redis.Command;
 import com.gemstone.gemfire.internal.redis.ExecutionHandlerContext;
 import com.gemstone.gemfire.internal.redis.Extendable;
 import com.gemstone.gemfire.internal.redis.RedisConstants.ArityDef;
-import com.gemstone.gemfire.internal.redis.RegionCache;
+import com.gemstone.gemfire.internal.redis.RegionProvider;
 
 public class ExpireAtExecutor extends AbstractExecutor implements Extendable {
 
@@ -28,7 +28,7 @@ public class ExpireAtExecutor extends AbstractExecutor implements Extendable {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), getArgsError()));
       return;
     }
-    RegionCache rC = context.getRegionCache();
+    RegionProvider rC = context.getRegionProvider();
     ByteArrayWrapper wKey = command.getKey();
 
     byte[] timestampByteArray = commandElems.get(TIMESTAMP_INDEX);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d2296bb/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpireExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpireExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpireExecutor.java
index b237f01..dc5ad3d 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpireExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ExpireExecutor.java
@@ -8,7 +8,7 @@ import com.gemstone.gemfire.internal.redis.Command;
 import com.gemstone.gemfire.internal.redis.ExecutionHandlerContext;
 import com.gemstone.gemfire.internal.redis.Extendable;
 import com.gemstone.gemfire.internal.redis.RedisConstants.ArityDef;
-import com.gemstone.gemfire.internal.redis.RegionCache;
+import com.gemstone.gemfire.internal.redis.RegionProvider;
 
 public class ExpireExecutor extends AbstractExecutor implements Extendable {
 
@@ -29,7 +29,7 @@ public class ExpireExecutor extends AbstractExecutor implements Extendable {
       return;
     }
     ByteArrayWrapper wKey = command.getKey();
-    RegionCache rC = context.getRegionCache();
+    RegionProvider rC = context.getRegionProvider();
         byte[] delayByteArray = commandElems.get(SECONDS_INDEX);
     long delay;
     try {