You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2016/07/25 17:26:57 UTC

incubator-geode git commit: GEODE-1617: Regions can be created with a variety of characters that are unsupported

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 40f95887b -> 7ca571c56


GEODE-1617: Regions can be created with a variety of characters that are unsupported

* Test for valid characters when creating non-internal regions
* Updated unit tests and added a new isInternalRegion field to InternalRegionAttributes
* Fixed issue with HA region.
* Found a potential issue with Attributes for a region.
* This closes #201


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

Branch: refs/heads/develop
Commit: 7ca571c56e8d49e35b9e4f445bd3ca4fbdea974e
Parents: 40f9588
Author: Kevin J. Duling <kd...@pivotal.io>
Authored: Mon Jul 11 14:20:56 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Jul 25 10:25:17 2016 -0700

----------------------------------------------------------------------
 .../modules/util/CreateRegionFunction.java      |  33 +++-
 .../gemfire/cache/DynamicRegionFactory.java     |   1 +
 .../gemstone/gemfire/cache/RegionFactory.java   |   1 +
 .../internal/cache/GemFireCacheImpl.java        |  11 +-
 .../gemfire/internal/cache/HARegion.java        |   9 +-
 .../internal/cache/InternalRegionArguments.java |  11 +-
 .../gemfire/internal/cache/LocalRegion.java     | 187 ++++++++++++++++---
 .../internal/cache/PartitionedRegionHelper.java |  34 +++-
 .../parallel/ParallelGatewaySenderQueue.java    |  10 +-
 .../wan/serial/SerialGatewaySenderQueue.java    |  65 +++++--
 .../apache/geode/redis/GeodeRedisServer.java    |  80 +++++---
 .../gemfire/cache/query/QueryJUnitTest.java     | 104 +----------
 .../gemfire/cache/query/RegionJUnitTest.java    |  16 +-
 .../gemfire/distributed/LocatorDUnitTest.java   |  48 +++--
 .../PartitionedRegionCreationDUnitTest.java     |  33 +---
 .../cache/tier/sockets/Bug36829DUnitTest.java   |  13 +-
 .../cache/tier/sockets/Bug37805DUnitTest.java   |   2 +-
 .../RegionCreateDestroyDUnitTest.java           | 174 +++++++++++++++++
 .../management/RegionManagementDUnitTest.java   |  23 +--
 19 files changed, 564 insertions(+), 291 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/extensions/geode-modules/src/main/java/com/gemstone/gemfire/modules/util/CreateRegionFunction.java
----------------------------------------------------------------------
diff --git a/extensions/geode-modules/src/main/java/com/gemstone/gemfire/modules/util/CreateRegionFunction.java b/extensions/geode-modules/src/main/java/com/gemstone/gemfire/modules/util/CreateRegionFunction.java
index 564ccb7..342968c 100644
--- a/extensions/geode-modules/src/main/java/com/gemstone/gemfire/modules/util/CreateRegionFunction.java
+++ b/extensions/geode-modules/src/main/java/com/gemstone/gemfire/modules/util/CreateRegionFunction.java
@@ -16,28 +16,34 @@
 */
 package com.gemstone.gemfire.modules.util;
 
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.Properties;
+
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.Declarable;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
 import com.gemstone.gemfire.cache.RegionFactory;
 import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.FunctionContext;
 import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
 import com.gemstone.gemfire.distributed.DistributedLockService;
 import com.gemstone.gemfire.distributed.internal.locks.DistributedMemberLock;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlGenerator;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.util.Properties;
-
 public class CreateRegionFunction implements Function, Declarable {
 
   private static final long serialVersionUID = -9210226844302128969L;
@@ -195,7 +201,7 @@ public class CreateRegionFunction implements Function, Declarable {
       // Unlock the distributed lock
       try {
         dml.unlock();
-      } catch (Exception e) {
+      } catch (Exception ignore) {
       }
     }
     return status;
@@ -214,7 +220,18 @@ public class CreateRegionFunction implements Function, Declarable {
     }
     RegionFactory<String, RegionConfiguration> factory = this.cache.createRegionFactory(RegionShortcut.REPLICATE);
     factory.addCacheListener(new RegionConfigurationCacheListener());
-    return factory.create(REGION_CONFIGURATION_METADATA_REGION);
+    GemFireCacheImpl gemFireCache = (GemFireCacheImpl) cache;
+    InternalRegionArguments ira = new InternalRegionArguments().setInternalRegion(true);
+    AttributesFactory af = new AttributesFactory();
+    af.setScope(Scope.LOCAL);
+    RegionAttributes ra = af.create();
+    try {
+      return gemFireCache.createVMRegion(REGION_CONFIGURATION_METADATA_REGION, ra, ira);
+    } catch (IOException | ClassNotFoundException e) {
+      InternalGemFireError assErr = new InternalGemFireError(LocalizedStrings.GemFireCache_UNEXPECTED_EXCEPTION.toLocalizedString());
+      assErr.initCause(e);
+      throw assErr;
+    }
   }
 
   private void writeCacheXml() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/main/java/com/gemstone/gemfire/cache/DynamicRegionFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/DynamicRegionFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/DynamicRegionFactory.java
index 9a59833..1f449cd 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/DynamicRegionFactory.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/DynamicRegionFactory.java
@@ -232,6 +232,7 @@ public abstract class DynamicRegionFactory  {
       if (this.dynamicRegionList == null) {
         InternalRegionArguments ira = new InternalRegionArguments()
         .setDestroyLockFlag(true)
+        .setInternalRegion(true)
         .setSnapshotInputStream(null)
         .setImageTarget(null);
         AttributesFactory af = new AttributesFactory ();       

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/main/java/com/gemstone/gemfire/cache/RegionFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/RegionFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/RegionFactory.java
index 3fd383a..4fe8ee2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/RegionFactory.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/RegionFactory.java
@@ -822,6 +822,7 @@ public class RegionFactory<K,V>
     RegionAttributes<K,V> ra = this.attrsFactory.create();
     return getCache().createRegion(name, ra);
   }
+
   /**
    * Creates a sub-region in the {@link Cache} using
    * the configuration contained in this RegionFactory. Validation of the

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index a9e1631..84ba926 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -3015,12 +3015,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
         ira.setIndexes(((UserSpecifiedRegionAttributes) attrs).getIndexes());
       }
       return createVMRegion(name, attrs, ira);
-    } catch (IOException e) {
-      // only if loading snapshot, not here
-      InternalGemFireError assErr = new InternalGemFireError(LocalizedStrings.GemFireCache_UNEXPECTED_EXCEPTION.toLocalizedString());
-      assErr.initCause(e);
-      throw assErr;
-    } catch (ClassNotFoundException e) {
+    } catch (IOException | ClassNotFoundException e) {
       // only if loading snapshot, not here
       InternalGemFireError assErr = new InternalGemFireError(LocalizedStrings.GemFireCache_UNEXPECTED_EXCEPTION.toLocalizedString());
       assErr.initCause(e);
@@ -3036,7 +3031,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
       }
     }
     stopper.checkCancelInProgress(null);
-    LocalRegion.validateRegionName(name);
+    LocalRegion.validateRegionName(name, internalRegionArgs);
     RegionAttributes<K, V> attrs = p_attrs;
     attrs = invokeRegionBefore(null, name, attrs, internalRegionArgs);
     if (attrs == null) {
@@ -3049,7 +3044,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     InternalDistributedMember imageTarget = internalRegionArgs.getImageTarget();
     final boolean recreate = internalRegionArgs.getRecreateFlag();
 
-    final boolean isPartitionedRegion = (attrs.getPartitionAttributes() == null) ? false : true;
+    final boolean isPartitionedRegion = attrs.getPartitionAttributes() != null;
     final boolean isReinitCreate = snapshotInputStream != null || imageTarget != null || recreate;
 
     final String regionPath = LocalRegion.calcFullPath(name, null);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/HARegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/HARegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/HARegion.java
index cac3d0a..d79cb72 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/HARegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/HARegion.java
@@ -273,10 +273,11 @@ public final class HARegion extends DistributedRegion
 
     HARegion haRegion = new HARegion(regionName, ra, null, cache);
     haRegion.setOwner(hrq);
-    Region region = cache.createVMRegion(regionName, ra,
-        new InternalRegionArguments().setInternalMetaRegion(haRegion)
-            .setDestroyLockFlag(true).setSnapshotInputStream(null)
-            .setImageTarget(null));
+    Region region = cache.createVMRegion(regionName, ra, new InternalRegionArguments().setInternalMetaRegion(haRegion)
+                                                                                      .setDestroyLockFlag(true)
+                                                                                      .setSnapshotInputStream(null)
+                                                                                      .setInternalRegion(true)
+                                                                                      .setImageTarget(null));
 
     return (HARegion)region;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InternalRegionArguments.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InternalRegionArguments.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InternalRegionArguments.java
index c403231..54c9194 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InternalRegionArguments.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/InternalRegionArguments.java
@@ -20,7 +20,6 @@ import java.io.InputStream;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.LocalRegion.TestCallable;
@@ -39,6 +38,7 @@ public final class InternalRegionArguments
   private boolean isUsedForPartitionedRegionAdmin;
   private boolean isUsedForSerialGatewaySenderQueue;
   private boolean isUsedForParallelGatewaySenderQueue;
+  private boolean isInternalRegion;
   private int bucketRedundancy;
   private boolean isUsedForPartitionedRegionBucket;
   private RegionAdvisor partitionedRegionAdvisor;
@@ -311,4 +311,13 @@ public final class InternalRegionArguments
   public Map<String,CacheServiceProfile> getCacheServiceProfiles() {
     return this.cacheServiceProfiles;
   }
+
+  public boolean isInternalRegion() {
+    return isInternalRegion;
+  }
+
+  public InternalRegionArguments setInternalRegion(final boolean internalRegion) {
+    isInternalRegion = internalRegion;
+    return this;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index 7da2b45..46ccd47 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -17,10 +17,96 @@
 
 package com.gemstone.gemfire.internal.cache;
 
-import com.gemstone.gemfire.*;
+import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.*;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.AbstractSet;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Hashtable;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.logging.log4j.Logger;
+
+import com.gemstone.gemfire.CancelCriterion;
+import com.gemstone.gemfire.CancelException;
+import com.gemstone.gemfire.CopyHelper;
+import com.gemstone.gemfire.DataSerializable;
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.DeltaSerializationException;
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.InternalGemFireException;
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.SystemFailure;
 import com.gemstone.gemfire.admin.internal.SystemMemberCacheEventProcessor;
-import com.gemstone.gemfire.cache.*;
+import com.gemstone.gemfire.cache.AttributesMutator;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheClosedException;
+import com.gemstone.gemfire.cache.CacheEvent;
+import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.CacheListener;
+import com.gemstone.gemfire.cache.CacheLoader;
+import com.gemstone.gemfire.cache.CacheLoaderException;
+import com.gemstone.gemfire.cache.CacheRuntimeException;
+import com.gemstone.gemfire.cache.CacheStatistics;
+import com.gemstone.gemfire.cache.CacheWriter;
+import com.gemstone.gemfire.cache.CacheWriterException;
+import com.gemstone.gemfire.cache.CustomExpiry;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.DiskAccessException;
+import com.gemstone.gemfire.cache.DiskStoreFactory;
+import com.gemstone.gemfire.cache.DiskWriteAttributes;
+import com.gemstone.gemfire.cache.DiskWriteAttributesFactory;
+import com.gemstone.gemfire.cache.EntryDestroyedException;
+import com.gemstone.gemfire.cache.EntryExistsException;
+import com.gemstone.gemfire.cache.EntryNotFoundException;
+import com.gemstone.gemfire.cache.EvictionAttributes;
+import com.gemstone.gemfire.cache.ExpirationAttributes;
+import com.gemstone.gemfire.cache.FailedSynchronizationException;
+import com.gemstone.gemfire.cache.InterestRegistrationEvent;
+import com.gemstone.gemfire.cache.InterestResultPolicy;
+import com.gemstone.gemfire.cache.LoaderHelper;
+import com.gemstone.gemfire.cache.LowMemoryException;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.PartitionedRegionStorageException;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionDestroyedException;
+import com.gemstone.gemfire.cache.RegionEvent;
+import com.gemstone.gemfire.cache.RegionExistsException;
+import com.gemstone.gemfire.cache.RegionMembershipListener;
+import com.gemstone.gemfire.cache.RegionReinitializedException;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.StatisticsDisabledException;
 import com.gemstone.gemfire.cache.TimeoutException;
+import com.gemstone.gemfire.cache.TransactionException;
+import com.gemstone.gemfire.cache.TransactionId;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.client.ServerOperationException;
 import com.gemstone.gemfire.cache.client.SubscriptionNotEnabledException;
@@ -32,7 +118,17 @@ import com.gemstone.gemfire.cache.control.ResourceManager;
 import com.gemstone.gemfire.cache.execute.Function;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
 import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
-import com.gemstone.gemfire.cache.query.*;
+import com.gemstone.gemfire.cache.query.FunctionDomainException;
+import com.gemstone.gemfire.cache.query.Index;
+import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
+import com.gemstone.gemfire.cache.query.IndexType;
+import com.gemstone.gemfire.cache.query.MultiIndexCreationException;
+import com.gemstone.gemfire.cache.query.NameResolutionException;
+import com.gemstone.gemfire.cache.query.QueryException;
+import com.gemstone.gemfire.cache.query.QueryInvocationTargetException;
+import com.gemstone.gemfire.cache.query.QueryService;
+import com.gemstone.gemfire.cache.query.SelectResults;
+import com.gemstone.gemfire.cache.query.TypeMismatchException;
 import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
 import com.gemstone.gemfire.cache.query.internal.DefaultQueryService;
 import com.gemstone.gemfire.cache.query.internal.ExecutionContext;
@@ -45,11 +141,22 @@ import com.gemstone.gemfire.cache.util.ObjectSizer;
 import com.gemstone.gemfire.cache.wan.GatewaySender;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.DistributedSystem;
-import com.gemstone.gemfire.distributed.internal.*;
+import com.gemstone.gemfire.distributed.internal.DM;
+import com.gemstone.gemfire.distributed.internal.DistributionAdvisor;
 import com.gemstone.gemfire.distributed.internal.DistributionAdvisor.Profile;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.ResourceEvent;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.i18n.StringId;
-import com.gemstone.gemfire.internal.*;
+import com.gemstone.gemfire.internal.Assert;
+import com.gemstone.gemfire.internal.ClassLoadUtil;
+import com.gemstone.gemfire.internal.HeapDataOutputStream;
+import com.gemstone.gemfire.internal.InternalStatisticsDisabledException;
+import com.gemstone.gemfire.internal.NanoTimer;
+import com.gemstone.gemfire.internal.Version;
+import com.gemstone.gemfire.internal.admin.ClientHealthMonitoringRegion;
 import com.gemstone.gemfire.internal.cache.CacheDistributionAdvisor.CacheProfile;
 import com.gemstone.gemfire.internal.cache.DiskInitFile.DiskRegionFlag;
 import com.gemstone.gemfire.internal.cache.FilterRoutingInfo.FilterInfo;
@@ -60,7 +167,11 @@ import com.gemstone.gemfire.internal.cache.control.InternalResourceManager.Resou
 import com.gemstone.gemfire.internal.cache.control.MemoryEvent;
 import com.gemstone.gemfire.internal.cache.control.MemoryThresholds;
 import com.gemstone.gemfire.internal.cache.control.ResourceListener;
-import com.gemstone.gemfire.internal.cache.execute.*;
+import com.gemstone.gemfire.internal.cache.execute.DistributedRegionFunctionExecutor;
+import com.gemstone.gemfire.internal.cache.execute.DistributedRegionFunctionResultSender;
+import com.gemstone.gemfire.internal.cache.execute.LocalResultCollector;
+import com.gemstone.gemfire.internal.cache.execute.RegionFunctionContextImpl;
+import com.gemstone.gemfire.internal.cache.execute.ServerToClientFunctionResultSender;
 import com.gemstone.gemfire.internal.cache.ha.ThreadIdentifier;
 import com.gemstone.gemfire.internal.cache.lru.LRUEntry;
 import com.gemstone.gemfire.internal.cache.partitioned.RedundancyAlreadyMetException;
@@ -71,8 +182,19 @@ import com.gemstone.gemfire.internal.cache.persistence.PersistentMemberID;
 import com.gemstone.gemfire.internal.cache.persistence.query.IndexMap;
 import com.gemstone.gemfire.internal.cache.persistence.query.mock.IndexMapImpl;
 import com.gemstone.gemfire.internal.cache.tier.InterestType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
-import com.gemstone.gemfire.internal.cache.versions.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientNotifier;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientHealthMonitor;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientTombstoneMessage;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ClientUpdateMessage;
+import com.gemstone.gemfire.internal.cache.tier.sockets.HandShake;
+import com.gemstone.gemfire.internal.cache.tier.sockets.VersionedObjectList;
+import com.gemstone.gemfire.internal.cache.versions.ConcurrentCacheModificationException;
+import com.gemstone.gemfire.internal.cache.versions.RegionVersionHolder;
+import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
+import com.gemstone.gemfire.internal.cache.versions.VersionSource;
+import com.gemstone.gemfire.internal.cache.versions.VersionStamp;
+import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.cache.wan.AbstractGatewaySender;
 import com.gemstone.gemfire.internal.cache.wan.GatewaySenderEventCallbackArgument;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
@@ -89,18 +211,6 @@ import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
 import com.gemstone.gemfire.internal.util.concurrent.FutureResult;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableCountDownLatch;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReadWriteLock;
-import org.apache.logging.log4j.Logger;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-import java.util.regex.Pattern;
-
-import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ENTRY_EVENT_NEW_VALUE;
 
 /**
  * Implementation of a local scoped-region. Note that this class has a different
@@ -111,12 +221,13 @@ import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifie
  *
  */
 @SuppressWarnings("deprecation")
-public class LocalRegion extends AbstractRegion 
+public class LocalRegion extends AbstractRegion
   implements LoaderHelperFactory, ResourceListener<MemoryEvent>,
              DiskExceptionHandler, DiskRecoveryStore
 {
   private static final Logger logger = LogService.getLogger();
-  
+  private static final Pattern NAME_PATTERN = Pattern.compile("[aA-zZ0-9-_.]+");
+
   /**
    * Internal interface used to simulate failures when performing entry operations
    * @since GemFire 5.7
@@ -809,7 +920,7 @@ public class LocalRegion extends AbstractRegion
           }
           throw new RegionDestroyedException(toString(), getFullPath());
         }
-        validateRegionName(subregionName);
+        validateRegionName(subregionName, internalRegionArgs);
 
         validateSubregionAttributes(regionAttributes);
         String regionPath = calcFullPath(subregionName, this);
@@ -7769,17 +7880,41 @@ public class LocalRegion extends AbstractRegion
     this.entries.removeEntry(event.getKey(), re, false) ;      
   }
 
-  static void validateRegionName(String name)
+  static void validateRegionName(String name, InternalRegionArguments internalRegionArgs)
   {
     if (name == null) {
       throw new IllegalArgumentException(LocalizedStrings.LocalRegion_NAME_CANNOT_BE_NULL.toLocalizedString());
     }
-    if (name.length() == 0) {
+    if (name.isEmpty()) {
       throw new IllegalArgumentException(LocalizedStrings.LocalRegion_NAME_CANNOT_BE_EMPTY.toLocalizedString());
     }
-    if (name.indexOf(SEPARATOR) >= 0) {
+    if (name.contains(SEPARATOR)) {
       throw new IllegalArgumentException(LocalizedStrings.LocalRegion_NAME_CANNOT_CONTAIN_THE_SEPARATOR_0.toLocalizedString(SEPARATOR));
     }
+
+    // Validate the name of the region only if it isn't an internal region
+    if (internalRegionArgs.isInternalRegion()){
+      return;
+    }
+    if (internalRegionArgs.isUsedForMetaRegion()) {
+      return;
+    }
+    if (internalRegionArgs.isUsedForPartitionedRegionAdmin()) {
+      return;
+    }
+    if (internalRegionArgs.isUsedForPartitionedRegionBucket()) {
+      return;
+    }
+
+    if (name.startsWith("__")) {
+      throw new IllegalArgumentException("Region names may not begin with a double-underscore: " + name);
+    }
+
+    // Ensure the region only contains valid characters
+    Matcher matcher = NAME_PATTERN.matcher(name);
+    if (!matcher.matches()) {
+      throw new IllegalArgumentException("Region names may only be alphanumeric and may contain hyphens or underscores: " + name);
+    }
   }
 
   private void checkCacheClosed()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelper.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelper.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelper.java
index e0d4f48..397f35f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelper.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/PartitionedRegionHelper.java
@@ -17,8 +17,35 @@
 
 package com.gemstone.gemfire.internal.cache;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.logging.log4j.Logger;
+
 import com.gemstone.gemfire.CancelException;
-import com.gemstone.gemfire.cache.*;
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheWriterException;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.EntryDestroyedException;
+import com.gemstone.gemfire.cache.EntryEvent;
+import com.gemstone.gemfire.cache.EntryNotFoundException;
+import com.gemstone.gemfire.cache.EntryOperation;
+import com.gemstone.gemfire.cache.FixedPartitionAttributes;
+import com.gemstone.gemfire.cache.FixedPartitionResolver;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.PartitionResolver;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionExistsException;
+import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.partition.PartitionNotAvailableException;
 import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
 import com.gemstone.gemfire.cache.util.CacheWriterAdapter;
@@ -37,10 +64,6 @@ import com.gemstone.gemfire.internal.cache.partitioned.RegionAdvisor;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
-import org.apache.logging.log4j.Logger;
-
-import java.io.IOException;
-import java.util.*;
 
 /**
  */
@@ -287,6 +310,7 @@ public class PartitionedRegionHelper
         root = (DistributedRegion) gemCache.createVMRegion(PR_ROOT_REGION_NAME, ra, 
             new InternalRegionArguments()
             .setIsUsedForPartitionedRegionAdmin(true)
+            .setInternalRegion(true)
             .setCachePerfStatsHolder(prMetaStatsHolder));
         root.getDistributionAdvisor().addMembershipListener(new MemberFailureListener());
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
index d703c6e..d21d6dc 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
@@ -16,8 +16,6 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.parallel;
 
-import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -38,6 +36,7 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 
+import it.unimi.dsi.fastutil.objects.ObjectOpenHashSet;
 import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.CancelException;
@@ -557,7 +556,7 @@ public class ParallelGatewaySenderQueue implements RegionQueue {
         try {
           prQ = (PartitionedRegion)cache
               .createVMRegion(prQName, ra, new InternalRegionArguments()
-                  .setInternalMetaRegion(meta).setDestroyLockFlag(true)
+                  .setInternalMetaRegion(meta).setDestroyLockFlag(true).setInternalRegion(true)
                   .setSnapshotInputStream(null).setImageTarget(null));
           // at this point we should be able to assert prQ == meta; 
           
@@ -570,12 +569,9 @@ public class ParallelGatewaySenderQueue implements RegionQueue {
           //Wait for buckets to be recovered.
           prQ.shadowPRWaitForBucketRecovery();
 
-        } catch (IOException veryUnLikely) {
+        } catch (IOException | ClassNotFoundException veryUnLikely) {
           logger.fatal(LocalizedMessage.create(LocalizedStrings.SingleWriteSingleReadRegionQueue_UNEXPECTED_EXCEPTION_DURING_INIT_OF_0,
                   this.getClass()), veryUnLikely);
-        } catch (ClassNotFoundException alsoUnlikely) {
-          logger.fatal(LocalizedMessage.create(LocalizedStrings.SingleWriteSingleReadRegionQueue_UNEXPECTED_EXCEPTION_DURING_INIT_OF_0,
-                  this.getClass()), alsoUnlikely);
         }
         if (logger.isDebugEnabled()) {
           logger.debug("{}: Created queue region: {}", this, prQ);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialGatewaySenderQueue.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialGatewaySenderQueue.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialGatewaySenderQueue.java
index 57d6e34..efa7870 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialGatewaySenderQueue.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/serial/SerialGatewaySenderQueue.java
@@ -16,14 +16,51 @@
  */
 package com.gemstone.gemfire.internal.cache.wan.serial;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.logging.log4j.Logger;
+
 import com.gemstone.gemfire.CancelException;
 import com.gemstone.gemfire.SystemFailure;
-import com.gemstone.gemfire.cache.*;
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.AttributesMutator;
+import com.gemstone.gemfire.cache.CacheException;
+import com.gemstone.gemfire.cache.CacheListener;
+import com.gemstone.gemfire.cache.CacheWriterException;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.EntryNotFoundException;
+import com.gemstone.gemfire.cache.EvictionAction;
+import com.gemstone.gemfire.cache.EvictionAttributes;
+import com.gemstone.gemfire.cache.Operation;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionDestroyedException;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.TimeoutException;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEvent;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.cache.*;
+import com.gemstone.gemfire.internal.cache.CachedDeserializable;
+import com.gemstone.gemfire.internal.cache.Conflatable;
+import com.gemstone.gemfire.internal.cache.DistributedRegion;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
+import com.gemstone.gemfire.internal.cache.LocalRegion;
+import com.gemstone.gemfire.internal.cache.RegionQueue;
+import com.gemstone.gemfire.internal.cache.Token;
 import com.gemstone.gemfire.internal.cache.versions.RegionVersionVector;
 import com.gemstone.gemfire.internal.cache.versions.VersionSource;
 import com.gemstone.gemfire.internal.cache.wan.AbstractGatewaySender;
@@ -34,13 +71,6 @@ import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.offheap.OffHeapRegionEntryHelper;
 import com.gemstone.gemfire.pdx.internal.PeerTypeRegistration;
-import org.apache.logging.log4j.Logger;
-
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
 
 /**
  * @since GemFire 7.0
@@ -979,15 +1009,14 @@ public class SerialGatewaySenderQueue implements RegionQueue {
         SerialGatewaySenderQueueMetaRegion meta = new SerialGatewaySenderQueueMetaRegion(
             this.regionName, ra, null, gemCache, sender);
         try {
-          this.region = gemCache.createVMRegion(
-              this.regionName,
-              ra,
-              new InternalRegionArguments().setInternalMetaRegion(meta)
-                  .setDestroyLockFlag(true).setSnapshotInputStream(null)
-                  .setImageTarget(null)
-                  .setIsUsedForSerialGatewaySenderQueue(true)
-                  .setSerialGatewaySender(sender));
-          
+          this.region = gemCache.createVMRegion(this.regionName, ra, new InternalRegionArguments().setInternalMetaRegion(meta)
+                                                                                                  .setDestroyLockFlag(true)
+                                                                                                  .setSnapshotInputStream(null)
+                                                                                                  .setImageTarget(null)
+                                                                                                  .setIsUsedForSerialGatewaySenderQueue(true)
+                                                                                                  .setInternalRegion(true)
+                                                                                                  .setSerialGatewaySender(sender));
+
         } catch (IOException veryUnLikely) {
           logger.fatal(LocalizedMessage.create(LocalizedStrings.SingleWriteSingleReadRegionQueue_UNEXPECTED_EXCEPTION_DURING_INIT_OF_0,
                   this.getClass()), veryUnLikely);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/main/java/org/apache/geode/redis/GeodeRedisServer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/redis/GeodeRedisServer.java b/geode-core/src/main/java/org/apache/geode/redis/GeodeRedisServer.java
index e586829..6f6ae6d 100644
--- a/geode-core/src/main/java/org/apache/geode/redis/GeodeRedisServer.java
+++ b/geode-core/src/main/java/org/apache/geode/redis/GeodeRedisServer.java
@@ -32,27 +32,6 @@ import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import com.gemstone.gemfire.LogWriter;
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.CacheFactory;
-import com.gemstone.gemfire.cache.EntryEvent;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionDestroyedException;
-import com.gemstone.gemfire.cache.RegionFactory;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
-import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
-import com.gemstone.gemfire.internal.SocketCreator;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.hll.HyperLogLogPlus;
-
-import org.apache.geode.redis.internal.ByteArrayWrapper;
-import org.apache.geode.redis.internal.ByteToCommandDecoder;
-import org.apache.geode.redis.internal.Coder;
-import org.apache.geode.redis.internal.ExecutionHandlerContext;
-import org.apache.geode.redis.internal.RedisDataType;
-import org.apache.geode.redis.internal.RegionProvider;
-
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.buffer.PooledByteBufAllocator;
 import io.netty.channel.Channel;
@@ -68,6 +47,32 @@ import io.netty.channel.socket.SocketChannel;
 import io.netty.channel.socket.nio.NioServerSocketChannel;
 import io.netty.channel.socket.oio.OioServerSocketChannel;
 import io.netty.util.concurrent.Future;
+import org.apache.geode.redis.internal.ByteArrayWrapper;
+import org.apache.geode.redis.internal.ByteToCommandDecoder;
+import org.apache.geode.redis.internal.Coder;
+import org.apache.geode.redis.internal.ExecutionHandlerContext;
+import org.apache.geode.redis.internal.RedisDataType;
+import org.apache.geode.redis.internal.RegionProvider;
+
+import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.LogWriter;
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.EntryEvent;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionDestroyedException;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.util.CacheListenerAdapter;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.internal.SocketCreator;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.InternalRegionArguments;
+import com.gemstone.gemfire.internal.hll.HyperLogLogPlus;
+import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 
 /**
  * The GeodeRedisServer is a server that understands the Redis protocol. As
@@ -419,19 +424,32 @@ public class GeodeRedisServer {
 
   private void initializeRedis() {
     synchronized (this.cache) {
-      RegionFactory<String, RedisDataType> rfMeta = cache.createRegionFactory(RegionShortcut.REPLICATE);
-      rfMeta.addCacheListener(this.metaListener);
-      RegionFactory<ByteArrayWrapper, ByteArrayWrapper> rfString = cache.createRegionFactory(DEFAULT_REGION_TYPE);
-      RegionFactory<ByteArrayWrapper, HyperLogLogPlus> rfHLL = cache.createRegionFactory(DEFAULT_REGION_TYPE);
       Region<ByteArrayWrapper, ByteArrayWrapper> stringsRegion;
-      if ((stringsRegion = this.cache.getRegion(STRING_REGION)) == null)
-        stringsRegion = rfString.create(GeodeRedisServer.STRING_REGION);
+      InternalRegionArguments ira = new InternalRegionArguments().setInternalRegion(true);
+      AttributesFactory af = new AttributesFactory();
+      af.setScope(Scope.LOCAL);
+      RegionAttributes ra = af.create();
       Region<ByteArrayWrapper, HyperLogLogPlus> hLLRegion;
-      if ((hLLRegion = this.cache.getRegion(HLL_REGION)) == null)
-        hLLRegion = rfHLL.create(HLL_REGION);
       Region<String, RedisDataType> redisMetaData;
-      if ((redisMetaData = this.cache.getRegion(REDIS_META_DATA_REGION)) == null)
-        redisMetaData = rfMeta.create(REDIS_META_DATA_REGION);
+      GemFireCacheImpl gemFireCache = (GemFireCacheImpl) cache;
+      try {
+        if ((stringsRegion = cache.getRegion(STRING_REGION)) == null) {
+          stringsRegion = gemFireCache.createVMRegion(GeodeRedisServer.STRING_REGION, ra, ira);
+        }
+        if ((hLLRegion = cache.getRegion(HLL_REGION)) == null) {
+          hLLRegion = gemFireCache.createVMRegion(HLL_REGION, ra, ira);
+        }
+        if ((redisMetaData = cache.getRegion(REDIS_META_DATA_REGION)) == null) {
+          af.addCacheListener(metaListener);
+          af.setDataPolicy(DataPolicy.REPLICATE);
+          redisMetaData = gemFireCache.createVMRegion(REDIS_META_DATA_REGION, af.create(), ira);
+        }
+      } catch (IOException | ClassNotFoundException e) {
+        // only if loading snapshot, not here
+        InternalGemFireError assErr = new InternalGemFireError(LocalizedStrings.GemFireCache_UNEXPECTED_EXCEPTION.toLocalizedString());
+        assErr.initCause(e);
+        throw assErr;
+      }
       this.regionCache = new RegionProvider(stringsRegion, hLLRegion, redisMetaData, expirationFutures, expirationExecutor, this.DEFAULT_REGION_TYPE);
       redisMetaData.put(REDIS_META_DATA_REGION, RedisDataType.REDIS_PROTECTED);
       redisMetaData.put(HLL_REGION, RedisDataType.REDIS_PROTECTED);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryJUnitTest.java
index d677b17..10e14d4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/QueryJUnitTest.java
@@ -117,101 +117,13 @@ public class QueryJUnitTest {
   }
 
   @Test
-  public void test003UnicodeInRegionNameAndQuery() {
-    CacheUtils.log("testUnicodeInQuery");
-    try{
-      Region region = CacheUtils.createRegion("\u4e2d��", Portfolio.class);
-      Portfolio p = new Portfolio(0);
-      p.unicode\u1e64tring = "\u4e2d��";
-      region.put("1",p);
-      region.put("2",new Portfolio(1));
-      String queryStr = "SELECT DISTINCT * FROM " + region.getFullPath() +" WHERE unicode\u1e64tring = '\u4e2d��'";
-      Query q = CacheUtils.getQueryService().newQuery(queryStr);
-      SelectResults results = (SelectResults)q.execute();
-      assertEquals(results.size(), 1);
-      p = (Portfolio)results.iterator().next();
-      assertEquals(p.unicode\u1e64tring, "\u4e2d��");
-    }catch(Exception e){
-      e.printStackTrace();
-      fail("Exception during Query.execute");
-    }
-  }
-
-  @Test
-  public void test004UnicodeInRegionNameAndQueryWithIndex() {
-    try {
-      String unicode = "\u2030\u220f\u2260���±��";
-      Region region = CacheUtils.createRegion(unicode, Portfolio.class);
-      CacheUtils.getQueryService().createIndex("unicodeIndex", "unicode\u1e64tring",
-          "/'" + unicode + "'");
-      Portfolio p = new Portfolio(0);
-      p.unicode\u1e64tring = unicode;
-      region.put("1", p);
-      region.put("2", new Portfolio(1));
-      String queryStr = "SELECT DISTINCT * FROM /'" + unicode
-          + "' WHERE unicode\u1e64tring = '" + unicode + "'";
-      Query q = CacheUtils.getQueryService().newQuery(queryStr);
-      SelectResults results = (SelectResults) q.execute();
-      assertEquals(results.size(), 1);
-      p = (Portfolio) results.iterator().next();
-      assertEquals(p.unicode\u1e64tring, unicode);
-
-      String unicode2 = "\u2030�±�\u220f\u2260��";
-      CacheUtils.createRegion(unicode2, Portfolio.class);
-      queryStr = "SELECT DISTINCT * FROM /'" + unicode + "' u1, /'" + unicode2
-          + "' u2 WHERE u1.unicode\u1e64tring = '" + unicode
-          + "' order by u1.unicode\u1e64tring  limit 1";
-      CacheUtils.log(queryStr);
-      CacheUtils.getQueryService().newQuery(queryStr).execute();
-
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail("Exception during Query.execute");
-    }
-  }
-
-
-  @Test
-  public void test005UnicodeInRegionNameAndQueryWithIndexUsingQuotesAsDelim() {
-    try {
-      String unicode = "\u2030\u220f\u2260���±*+|<?>=. !@#$%^&*()_+,;:��[]{}?";
-      Region region = CacheUtils.createRegion(unicode, Portfolio.class);
-      CacheUtils.getQueryService().createIndex("unicodeIndex", "unicode\u1e64tring",
-          "/'" + unicode + "'");
-      Portfolio p = new Portfolio(0);
-      p.unicode\u1e64tring = unicode;
-      region.put("1", p);
-      region.put("2", new Portfolio(1));
-      String queryStr = "SELECT DISTINCT * FROM /'" + unicode
-          + "' WHERE unicode\u1e64tring = '" + unicode + "'";
-      Query q = CacheUtils.getQueryService().newQuery(queryStr);
-      SelectResults results = (SelectResults) q.execute();
-      assertEquals(results.size(), 1);
-      p = (Portfolio) results.iterator().next();
-      assertEquals(p.unicode\u1e64tring, unicode);
-
-      String unicode2 = "!@#$%^&*(|)_?+,;: ��\u2030\u220f\u2260���±*+<>=.[]{}?";
-      CacheUtils.createRegion(unicode2, Portfolio.class);
-      queryStr = "SELECT DISTINCT * FROM /'" + unicode + "' u1, /'" + unicode2
-          + "' u2 WHERE u1.unicode\u1e64tring = '" + unicode
-          + "' order by u1.unicode\u1e64tring  limit 1";
-      CacheUtils.log(queryStr);
-      CacheUtils.getQueryService().newQuery(queryStr).execute();
-
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail("Exception during Query.execute");
-    }
-  }
-
-  @Test
-  public void test006Compile() {
+  public void test003Compile() {
     CacheUtils.log("testCompile");
     //fail("The test case is empty.");
   }
   
   @Test
-  public void test007IsCompiled() {
+  public void test004IsCompiled() {
     CacheUtils.log("testIsCompiled");
     String queryStr = "SELECT DISTINCT * FROM /root";
     Query q = CacheUtils.getQueryService().newQuery(queryStr);
@@ -220,7 +132,7 @@ public class QueryJUnitTest {
   }
     
   @Test
-  public void test008GetStatistics() {
+  public void test005GetStatistics() {
     CacheUtils.log("testGetStatistics");
     String queryStr = "SELECT DISTINCT * FROM /Portfolios where status='active'";
     Query q = CacheUtils.getQueryService().newQuery(queryStr);
@@ -253,7 +165,7 @@ public class QueryJUnitTest {
   }
   
   @Test
-  public void test009GetRegionsInQuery() {
+  public void test006GetRegionsInQuery() {
     
     String queryStrs[] = new String[] {
         "SELECT DISTINCT * FROM /Portfolios where status='active'",
@@ -300,7 +212,7 @@ public class QueryJUnitTest {
   }
 
   @Test
-  public void test010UndefinedResults() {
+  public void test007UndefinedResults() {
     CacheUtils.log("testQueryExceptionLogMessage");
     Region region = CacheUtils.createRegion("Portfolios", Portfolio.class);
     region.put("1",new Portfolio(1));
@@ -334,7 +246,7 @@ public class QueryJUnitTest {
   }  
 
   @Test
-  public void test011NullCollectionField() {
+  public void test008NullCollectionField() {
     Region region = CacheUtils.createRegion("Portfolios", Portfolio.class);
     for (int i = 0; i < 10; i++) {
       Portfolio p = new Portfolio(i);
@@ -365,7 +277,7 @@ public class QueryJUnitTest {
     } catch (Exception e) {
       fail("Query execution failed " + e);
     }
-    assertEquals("Incorrect result size ", 1, sr.size());
+    assertEquals("Incorrect result size ", 2, sr.size());
   }
 
   @Test
@@ -448,6 +360,4 @@ public class QueryJUnitTest {
       return exceptionsThrown.size() == 0;
     }
   }
-
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/test/java/com/gemstone/gemfire/cache/query/RegionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/RegionJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/RegionJUnitTest.java
index 9564a5f..c35847e 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/cache/query/RegionJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/cache/query/RegionJUnitTest.java
@@ -169,25 +169,17 @@ public class RegionJUnitTest{
     
     String queryStrs[] = new String[] {
         "SELECT * FROM /pos",
-        "SELECT * FROM /pos where status='active'",
-        "SELECT * FROM /p:os", // Using ':' in region name.
-        "SELECT * FROM /p@os", // Using '@' in region name.
-        "SELECT * FROM /p-os", // Using '-' in region name.
-        "SELECT * FROM /p#os", // Using '#' in region name.
-        "SELECT * FROM /p+os", // Using '+' in region name.
-    };
+        "SELECT * FROM /pos where status='active'"
+  };
     
     CacheUtils.startCache();
     cache = CacheUtils.getCache();
     AttributesFactory attributesFactory = new AttributesFactory();
     attributesFactory.setValueConstraint(Portfolio.class);
     RegionAttributes regionAttributes = attributesFactory.create();
-    
-    cache.createRegion("p:os",regionAttributes);
-    cache.createRegion("p@os",regionAttributes);
+
+    cache.createRegion("p_os",regionAttributes);
     cache.createRegion("p-os",regionAttributes);
-    cache.createRegion("p#os",regionAttributes);
-    cache.createRegion("p+os",regionAttributes);
 
     for(int i=0; i<queryStrs.length;++i) {
        Query q = CacheUtils.getQueryService().newQuery(queryStrs[i]);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
index 5e68b9e..954846f 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/distributed/LocatorDUnitTest.java
@@ -16,13 +16,18 @@
  */
 package com.gemstone.gemfire.distributed;
 
-import org.junit.experimental.categories.Category;
-import org.junit.Test;
-
+import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
 import static org.junit.Assert.*;
 
-import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.ForcedDisconnectException;
 import com.gemstone.gemfire.GemFireConfigException;
@@ -32,7 +37,12 @@ import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.distributed.internal.*;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
+import com.gemstone.gemfire.distributed.internal.DistributionException;
+import com.gemstone.gemfire.distributed.internal.DistributionManager;
+import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
+import com.gemstone.gemfire.distributed.internal.InternalLocator;
+import com.gemstone.gemfire.distributed.internal.MembershipListener;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipManager;
 import com.gemstone.gemfire.distributed.internal.membership.MembershipTestHook;
@@ -46,16 +56,20 @@ import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.logging.InternalLogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.internal.tcp.Connection;
-import com.gemstone.gemfire.test.dunit.*;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Properties;
-import java.util.Set;
-
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.dunit.DistributedTestUtils;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.dunit.LogWriterUtils;
+import com.gemstone.gemfire.test.dunit.NetworkUtils;
+import com.gemstone.gemfire.test.dunit.SerializableCallable;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.ThreadUtils;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.Wait;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * Tests the ability of the {@link Locator} API to start and stop
@@ -167,7 +181,7 @@ public class LocatorDUnitTest extends JUnit4DistributedTestCase {
 
     // ensure that I, as a collocated locator owner, can create a cache region
     Cache cache = CacheFactory.create(system);
-    Region r = cache.createRegionFactory(RegionShortcut.REPLICATE).create("test region");
+    Region r = cache.createRegionFactory(RegionShortcut.REPLICATE).create("test-region");
     assertNotNull("expected to create a region", r);
 
     // create a lock service and have every vm get a lock

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionCreationDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionCreationDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionCreationDUnitTest.java
index 6c89ab7..7c93604 100755
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionCreationDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionCreationDUnitTest.java
@@ -284,38 +284,7 @@ public class PartitionedRegionCreationDUnitTest extends PartitionedRegionDUnitTe
     vm3.invoke(getCacheSerializableRunnableForPRValidate(regionNamePrefix));
   }
 
-  /**
-   * Test Partitioned Region names that contain spaces
-   * @throws Exception
-   */
-  @Test
-  public void testPartitionedRegionNameWithSpaces() throws Exception 
-  {
-    final String rName = getUniqueName() + " with some spaces";
-    
-    CacheSerializableRunnable csr = new CacheSerializableRunnable("validateNoExceptionWhenUsingNameWithSpaces") {
-      @Override
-      public void run2() throws CacheException
-      {
-        Cache cache = getCache();
-        Region partitionedregion = null;
-        AttributesFactory attr = new AttributesFactory();
-        attr.setPartitionAttributes(new PartitionAttributesFactory()
-            .setRedundantCopies(0).create());
-        partitionedregion = cache.createRegion(rName, attr
-            .create());
-        assertNotNull("Partitioned Region ref null", partitionedregion);
-        assertNotNull("Cache does not contain PR " + rName, cache
-            .getRegion(rName));
-        assertTrue("Partitioned Region ref claims to be destroyed",
-            !partitionedregion.isDestroyed());
-      }
-    };
-    Host.getHost(0).getVM(2).invoke(csr);
-    Host.getHost(0).getVM(3).invoke(csr);
-  }
-
-  /**
+   /**
    * Test whether partition region creation is preveented when
    * an instance is created that has the incorrect redundancy
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36829DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36829DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36829DUnitTest.java
index fa00bef..7d719be 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36829DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug36829DUnitTest.java
@@ -46,7 +46,7 @@ public class Bug36829DUnitTest extends JUnit4DistributedTestCase {
 
   private VM ClientVM;
 
-  private String regionName;
+  private static final String REGION_NAME = "Bug36829_region";
 
   private int PORT;
 
@@ -55,7 +55,7 @@ public class Bug36829DUnitTest extends JUnit4DistributedTestCase {
     Host host = Host.getHost(0);
     this.serverVM = host.getVM(0);
     this.ClientVM = host.getVM(1);
-    regionName = Bug36829DUnitTest.class.getName() + "_region";
+
     CacheServerTestUtil.disableShufflingOfEndpoints();
   }
 
@@ -70,8 +70,7 @@ public class Bug36829DUnitTest extends JUnit4DistributedTestCase {
           ))).intValue();
 
     this.ClientVM.invoke(() -> CacheServerTestUtil.createCacheClient(
-            getClientPool(NetworkUtils.getServerHostName(ClientVM.getHost()), PORT, true, 0),
-            regionName,
+            getClientPool(NetworkUtils.getServerHostName(ClientVM.getHost()), PORT, true, 0), REGION_NAME,
             getClientDistributedSystemProperties(durableClientId,
                 durableClientTimeout), Boolean.TRUE ));
 
@@ -87,7 +86,7 @@ public class Bug36829DUnitTest extends JUnit4DistributedTestCase {
     this.ClientVM.invoke(() -> Bug36829DUnitTest.registerKey( "Key1" ));
 
     // creating Region on the Server
-/*    this.serverVM.invoke(() -> CacheServerTestUtil.createRegion( regionName ));
+/*    this.serverVM.invoke(() -> CacheServerTestUtil.createRegion( REGION_NAME ));
      // should be successful.
     this.ClientVM.invoke(() -> Bug36829DUnitTest.registerKeyAfterRegionCreation( "Key1" ));*/
 
@@ -100,7 +99,7 @@ public class Bug36829DUnitTest extends JUnit4DistributedTestCase {
 
   private static void registerKey(String key) throws Exception {
     // Get the region
-    Region region = CacheServerTestUtil.getCache().getRegion(Bug36829DUnitTest.class.getName() + "_region");
+    Region region = CacheServerTestUtil.getCache().getRegion(REGION_NAME);
     assertNotNull(region);
     try {
       region.registerInterest(key, InterestResultPolicy.NONE);
@@ -112,7 +111,7 @@ public class Bug36829DUnitTest extends JUnit4DistributedTestCase {
 
   private static void registerKeyAfterRegionCreation(String key) throws Exception {
     // Get the region
-    Region region = CacheServerTestUtil.getCache().getRegion(Bug36829DUnitTest.class.getName() + "_region");
+    Region region = CacheServerTestUtil.getCache().getRegion(REGION_NAME);
     assertNotNull(region);
 
     region.registerInterest(key, InterestResultPolicy.NONE);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37805DUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37805DUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37805DUnitTest.java
index ee4471f..3d18e23 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37805DUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/Bug37805DUnitTest.java
@@ -60,7 +60,7 @@ public class Bug37805DUnitTest extends JUnit4DistributedTestCase {
     Host host = Host.getHost(0);
     this.server1VM = host.getVM(0);
     this.durableClientVM = host.getVM(1);
-    regionName = Bug37805DUnitTest.class.getName() + "_region";
+    regionName = "Bug37805_region";
     CacheServerTestUtil.disableShufflingOfEndpoints();
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/test/java/com/gemstone/gemfire/management/RegionCreateDestroyDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/RegionCreateDestroyDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/RegionCreateDestroyDUnitTest.java
new file mode 100644
index 0000000..8b75f1b
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/RegionCreateDestroyDUnitTest.java
@@ -0,0 +1,174 @@
+/*
+ * 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.management;
+
+import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.Assert.*;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.client.ClientCache;
+import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.Invoke;
+import com.gemstone.gemfire.test.dunit.RMIException;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+import com.gemstone.gemfire.test.junit.categories.SecurityTest;
+
+@Category({ DistributedTest.class, SecurityTest.class })
+public class RegionCreateDestroyDUnitTest extends JUnit4CacheTestCase {
+
+  private static final String GOOD_REGION_NAME = "Good-Region";
+  private static final String BAD_REGION_NAME = "Bad@Region";
+  private static final String RESERVED_REGION_NAME = "__ReservedRegion";
+
+  protected VM client1 = null;
+  protected VM client2 = null;
+  protected VM client3 = null;
+  protected int serverPort;
+
+  @Before
+  public void before() throws Exception {
+    final Host host = Host.getHost(0);
+    this.client1 = host.getVM(1);
+    this.client2 = host.getVM(2);
+    this.client3 = host.getVM(3);
+
+    Properties props = new Properties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "");
+
+    getSystem(props);
+
+  }
+
+  private void startServer(final Cache cache) throws IOException {
+    CacheServer server1 = cache.addCacheServer();
+    server1.setPort(0);
+    server1.start();
+
+    this.serverPort = server1.getPort();
+  }
+
+  @Override
+  public void preTearDownCacheTestCase() throws Exception {
+    Invoke.invokeInEveryVM(() -> closeCache());
+    closeCache();
+  }
+
+  protected Properties createClientProperties() {
+    Properties props = new Properties();
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "");
+    props.setProperty(SECURITY_LOG_LEVEL, "finest");
+    return props;
+  }
+
+  @Test
+  public void testCreateDestroyValidRegion() throws InterruptedException {
+    Cache serverCache = getCache();
+    serverCache.createRegionFactory(RegionShortcut.REPLICATE).create(GOOD_REGION_NAME);
+
+    try {
+      startServer(serverCache);
+    } catch (IOException e) {
+      fail(e.getMessage());
+    }
+    client1.invoke(() -> {
+      ClientCache cache = new ClientCacheFactory(createClientProperties()).setPoolSubscriptionEnabled(true)
+                                                                          .addPoolServer("localhost", serverPort)
+                                                                          .create();
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(GOOD_REGION_NAME);
+      region.destroyRegion();
+      assertThat(region.isDestroyed()).isTrue();
+    });
+  }
+
+  @Test
+  public void testCreateInvalidRegion() throws InterruptedException {
+    Cache serverCache = getCache();
+    try {
+      serverCache.createRegionFactory(RegionShortcut.REPLICATE).create(BAD_REGION_NAME);
+    } catch (IllegalArgumentException iae) {
+      assertEquals("Region names may only be alphanumeric and may contain hyphens or underscores: Bad@Region", iae.getMessage());
+    }
+
+    try {
+      startServer(serverCache);
+    } catch (IOException e) {
+      fail(e.getMessage());
+    }
+    client1.invoke(() -> {
+      ClientCache cache = new ClientCacheFactory(createClientProperties()).setPoolSubscriptionEnabled(true)
+                                                                          .addPoolServer("localhost", serverPort)
+                                                                          .create();
+      try {
+        cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(BAD_REGION_NAME);
+        fail("Should have thrown an IllegalArgumentException");
+      } catch (IllegalArgumentException iae) {
+        assertEquals("Region names may only be alphanumeric and may contain hyphens or underscores: Bad@Region", iae.getMessage());
+      }
+    });
+  }
+
+  @Test
+  public void testCreateDestroyReservedRegion() throws InterruptedException {
+    Cache serverCache = getCache();
+    try {
+      serverCache.createRegionFactory(RegionShortcut.REPLICATE).create(RESERVED_REGION_NAME);
+      fail("Should have thrown an IllegalArgumentException");
+    } catch (IllegalArgumentException arg) {
+      assertEquals("Region names may not begin with a double-underscore: __ReservedRegion", arg.getMessage());
+    }
+    try {
+      startServer(serverCache);
+    } catch (IOException e) {
+      fail(e.getMessage());
+    }
+
+    try {
+      client1.invoke(() -> {
+        ClientCache cache = new ClientCacheFactory(createClientProperties()).setPoolSubscriptionEnabled(true)
+                                                                            .addPoolServer("localhost", serverPort)
+                                                                            .create();
+        try {
+          cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(RESERVED_REGION_NAME);
+          fail("Should have thrown an IllegalArgumentException");
+        } catch (IllegalArgumentException e) {
+          assertEquals("Region names may not begin with a double-underscore: __ReservedRegion", e.getMessage());
+        }
+      });
+    } catch (RMIException rmi) {
+      rmi.getCause();
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ca571c5/geode-core/src/test/java/com/gemstone/gemfire/management/RegionManagementDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/RegionManagementDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/RegionManagementDUnitTest.java
index 9b72ff7..081be54 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/RegionManagementDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/RegionManagementDUnitTest.java
@@ -16,15 +16,8 @@
  */
 package com.gemstone.gemfire.management;
 
-import org.junit.experimental.categories.Category;
-import org.junit.Test;
-
 import static org.junit.Assert.*;
 
-import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
-import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -37,6 +30,9 @@ import javax.management.Notification;
 import javax.management.NotificationListener;
 import javax.management.ObjectName;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.CacheListener;
@@ -64,6 +60,7 @@ import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.dunit.VM;
 import com.gemstone.gemfire.test.dunit.Wait;
 import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 /**
  * This class checks and verifies various data and operations exposed through
@@ -362,12 +359,8 @@ public class RegionManagementDUnitTest extends ManagementTestBase {
           attributesFactory.setValueConstraint(Portfolio.class);
           RegionAttributes regionAttributes = attributesFactory.create();
           
-          cache.createRegion("p:os",regionAttributes);
-          cache.createRegion("p@os",regionAttributes);
           cache.createRegion("p-os",regionAttributes);
-          cache.createRegion("p#os",regionAttributes);
-          cache.createRegion("p+os",regionAttributes);
-          cache.createRegion("p?os",regionAttributes);
+          cache.createRegion("p_os",regionAttributes);
         }
       });
 
@@ -384,12 +377,8 @@ public class RegionManagementDUnitTest extends ManagementTestBase {
           ManagementService service = getManagementService();
           
           try {
-            MBeanUtil.getDistributedRegionMbean("/p:os", 1);
-            MBeanUtil.getDistributedRegionMbean("/p@os", 1);
             MBeanUtil.getDistributedRegionMbean("/p-os", 1);
-            MBeanUtil.getDistributedRegionMbean("/p#os", 1);
-            MBeanUtil.getDistributedRegionMbean("/p+os", 1);
-            MBeanUtil.getDistributedRegionMbean("/p?os", 1);
+            MBeanUtil.getDistributedRegionMbean("/p_os", 1);
 
           } catch (Exception e) {
             InternalDistributedSystem.getLoggerI18n().fine(