You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by bs...@apache.org on 2015/08/14 22:39:50 UTC

[01/50] [abbrv] incubator-geode git commit: GEODE-124: Add logic to compute getTotalSize

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-77 ab68f4e68 -> e0d1c4f90


GEODE-124: Add logic to compute getTotalSize

getTotalSize collects current size of a partitioned region on all members. The
sum of these sizes is used to check if rebalance is needed.


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

Branch: refs/heads/feature/GEODE-77
Commit: d307eda9253e9d8929806f5421ef12059eed71eb
Parents: a0ada76
Author: Ashvin Agrawal <as...@apache.org>
Authored: Mon Jul 27 10:02:24 2015 -0700
Committer: Ashvin Agrawal <as...@apache.org>
Committed: Mon Jul 27 10:02:24 2015 -0700

----------------------------------------------------------------------
 .../gemfire/cache/util/AutoBalancer.java        |  21 +++-
 .../cache/util/AutoBalancerJUnitTest.java       | 103 ++++++++++++++++++-
 2 files changed, 117 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d307eda9/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java b/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
index decbd89..1de7031 100644
--- a/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
+++ b/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
@@ -2,6 +2,7 @@ package com.gemstone.gemfire.cache.util;
 
 import java.util.Date;
 import java.util.Properties;
+import java.util.Set;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
@@ -17,9 +18,13 @@ import com.gemstone.gemfire.cache.Declarable;
 import com.gemstone.gemfire.cache.GemFireCache;
 import com.gemstone.gemfire.cache.control.RebalanceOperation;
 import com.gemstone.gemfire.cache.control.RebalanceResults;
+import com.gemstone.gemfire.cache.partition.PartitionMemberInfo;
 import com.gemstone.gemfire.distributed.DistributedLockService;
 import com.gemstone.gemfire.distributed.internal.locks.DLockService;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
+import com.gemstone.gemfire.internal.cache.partitioned.InternalPRInfo;
+import com.gemstone.gemfire.internal.cache.partitioned.LoadProbe;
 import com.gemstone.gemfire.internal.logging.LogService;
 
 /**
@@ -271,8 +276,20 @@ public class AutoBalancer implements Declarable {
   static class GeodeCacheFacade implements CacheOperationFacade {
     @Override
     public long getTotalDataSize() {
-      // TODO Auto-generated method stub
-      return getTotalTransferSize();
+      long totalSize = 0;
+      GemFireCacheImpl cache = getCache();
+      for (PartitionedRegion region : cache.getPartitionedRegions()) {
+        LoadProbe probe = cache.getResourceManager().getLoadProbe();
+        InternalPRInfo info = region.getRedundancyProvider().buildPartitionedRegionInfo(true, probe);
+        Set<PartitionMemberInfo> membersInfo = info.getPartitionMemberInfo();
+        for (PartitionMemberInfo member : membersInfo) {
+          if (logger.isDebugEnabled()) {
+            logger.debug("Region:{}, Member: {}, Size: {}", region.getFullPath(), member, member.getSize());
+          }
+          totalSize += member.getSize();
+        }
+      }
+      return totalSize;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d307eda9/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
index 90ddbc3..888ea20 100644
--- a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
+++ b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
@@ -6,6 +6,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
+import java.util.HashSet;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -27,6 +28,7 @@ import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.control.RebalanceFactory;
 import com.gemstone.gemfire.cache.control.RebalanceOperation;
 import com.gemstone.gemfire.cache.control.RebalanceResults;
+import com.gemstone.gemfire.cache.partition.PartitionMemberInfo;
 import com.gemstone.gemfire.cache.util.AutoBalancer.AuditScheduler;
 import com.gemstone.gemfire.cache.util.AutoBalancer.CacheOperationFacade;
 import com.gemstone.gemfire.cache.util.AutoBalancer.GeodeCacheFacade;
@@ -36,7 +38,11 @@ import com.gemstone.gemfire.cache.util.AutoBalancer.TimeProvider;
 import com.gemstone.gemfire.distributed.DistributedLockService;
 import com.gemstone.gemfire.distributed.internal.locks.DLockService;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.internal.cache.PRHARedundancyProvider;
+import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.control.InternalResourceManager;
+import com.gemstone.gemfire.internal.cache.partitioned.InternalPRInfo;
+import com.gemstone.gemfire.internal.cache.partitioned.LoadProbe;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -570,7 +576,7 @@ public class AutoBalancerJUnitTest {
   public void testFacadeTotalTransferSize() throws Exception {
     assertEquals(12345, getFacadeForResourceManagerOps(true).getTotalTransferSize());
   }
-  
+
   @Test
   public void testFacadeRebalance() throws Exception {
     getFacadeForResourceManagerOps(false).rebalance();
@@ -582,7 +588,7 @@ public class AutoBalancerJUnitTest {
     final RebalanceFactory mockRebalanceFactory = mockContext.mock(RebalanceFactory.class);
     final RebalanceOperation mockRebalanceOperation = mockContext.mock(RebalanceOperation.class);
     final RebalanceResults mockRebalanceResults = mockContext.mock(RebalanceResults.class);
-    
+
     mockContext.checking(new Expectations() {
       {
         oneOf(mockCache).getResourceManager();
@@ -604,17 +610,104 @@ public class AutoBalancerJUnitTest {
         allowing(mockRebalanceResults);
       }
     });
-    
+
     GeodeCacheFacade facade = new GeodeCacheFacade() {
       @Override
       GemFireCacheImpl getCache() {
         return mockCache;
       }
     };
-    
+
     return facade;
   }
-  
+
+  @Test
+  public void testFacadeTotalBytesNoRegion() {
+    final GemFireCacheImpl mockCache = mockContext.mock(GemFireCacheImpl.class);
+    mockContext.checking(new Expectations() {
+      {
+        oneOf(mockCache).getPartitionedRegions();
+        will(returnValue(new HashSet<PartitionedRegion>()));
+      }
+    });
+
+    GeodeCacheFacade facade = new GeodeCacheFacade() {
+      @Override
+      GemFireCacheImpl getCache() {
+        return mockCache;
+      }
+    };
+
+    assertEquals(0, facade.getTotalDataSize());
+  }
+
+  @Test
+  public void testFacadeTotalBytes2Regions() {
+    cache = createBasicCache();
+
+    final GemFireCacheImpl mockCache = mockContext.mock(GemFireCacheImpl.class);
+
+    final PartitionedRegion mockR1 = mockContext.mock(PartitionedRegion.class, "r1");
+    final PartitionedRegion mockR2 = mockContext.mock(PartitionedRegion.class, "r2");
+    final HashSet<PartitionedRegion> regions = new HashSet<>();
+    regions.add(mockR1);
+    regions.add(mockR2);
+
+    final PRHARedundancyProvider mockRedundancyProviderR1 = mockContext.mock(PRHARedundancyProvider.class, "prhaR1");
+    final InternalPRInfo mockR1PRInfo = mockContext.mock(InternalPRInfo.class, "prInforR1");
+    final PartitionMemberInfo mockR1M1Info = mockContext.mock(PartitionMemberInfo.class, "r1M1");
+    final PartitionMemberInfo mockR1M2Info = mockContext.mock(PartitionMemberInfo.class, "r1M2");
+    final HashSet<PartitionMemberInfo> r1Members = new HashSet<>();
+    r1Members.add(mockR1M1Info);
+    r1Members.add(mockR1M2Info);
+
+    final PRHARedundancyProvider mockRedundancyProviderR2 = mockContext.mock(PRHARedundancyProvider.class, "prhaR2");
+    final InternalPRInfo mockR2PRInfo = mockContext.mock(InternalPRInfo.class, "prInforR2");
+    final PartitionMemberInfo mockR2M1Info = mockContext.mock(PartitionMemberInfo.class, "r2M1");
+    final HashSet<PartitionMemberInfo> r2Members = new HashSet<>();
+    r2Members.add(mockR2M1Info);
+
+    mockContext.checking(new Expectations() {
+      {
+        oneOf(mockCache).getPartitionedRegions();
+        will(returnValue(regions));
+        exactly(2).of(mockCache).getResourceManager();
+        will(returnValue(cache.getResourceManager()));
+        allowing(mockR1).getFullPath();
+        oneOf(mockR1).getRedundancyProvider();
+        will(returnValue(mockRedundancyProviderR1));
+        allowing(mockR2).getFullPath();
+        oneOf(mockR2).getRedundancyProvider();
+        will(returnValue(mockRedundancyProviderR2));
+
+        oneOf(mockRedundancyProviderR1).buildPartitionedRegionInfo(with(true), with(any(LoadProbe.class)));
+        will(returnValue(mockR1PRInfo));
+        oneOf(mockR1PRInfo).getPartitionMemberInfo();
+        will(returnValue(r1Members));
+        atLeast(1).of(mockR1M1Info).getSize();
+        will(returnValue(123L));
+        atLeast(1).of(mockR1M2Info).getSize();
+        will(returnValue(74L));
+
+        oneOf(mockRedundancyProviderR2).buildPartitionedRegionInfo(with(true), with(any(LoadProbe.class)));
+        will(returnValue(mockR2PRInfo));
+        oneOf(mockR2PRInfo).getPartitionMemberInfo();
+        will(returnValue(r2Members));
+        atLeast(1).of(mockR2M1Info).getSize();
+        will(returnValue(3475L));
+      }
+    });
+
+    GeodeCacheFacade facade = new GeodeCacheFacade() {
+      @Override
+      GemFireCacheImpl getCache() {
+        return mockCache;
+      }
+    };
+
+    assertEquals(123 + 74 + 3475, facade.getTotalDataSize());
+  }
+
   private Properties getBasicConfig() {
     Properties props = new Properties();
     // every second schedule


[17/50] [abbrv] incubator-geode git commit: GEODE-162: remove race condititions and long sleeps from region expiration test methods in RegionTestCase

Posted by bs...@apache.org.
GEODE-162: remove race condititions and long sleeps from region expiration test methods in RegionTestCase


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

Branch: refs/heads/feature/GEODE-77
Commit: 3393f5a314b9bcd2de0d4c173703e9a5f791e197
Parents: 0ff54f3
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Jul 30 16:40:14 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Mon Aug 3 14:28:59 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/LocalRegion.java     |   7 +
 .../gemfire/cache30/RegionTestCase.java         | 132 ++++++++-----------
 2 files changed, 60 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3393f5a3/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index 9ea26a7..495e992 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -8815,6 +8815,13 @@ public class LocalRegion extends AbstractRegion
     }
     return this.entryExpiryTasks.get(re);
   }
+  /**
+   * Used by unit tests to get access to the RegionIdleExpiryTask
+   * of this region. Returns null if no task exists.
+   */
+  public RegionIdleExpiryTask getRegionIdleExpiryTask() {
+    return this.regionIdleExpiryTask;
+  }
   
   private void addExpiryTask(RegionEntry re, boolean ifAbsent)
   {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3393f5a3/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
index a20cfc5..69eeec0 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
@@ -2085,7 +2085,7 @@ public abstract class RegionTestCase extends CacheTestCase {
       if (now >= tilt)
         break;
       if (!region.isDestroyed()) {
-        pause(100);
+        pause(10);
         continue;
       }
       if (now >= tilt - SLOP) {
@@ -2107,7 +2107,7 @@ public abstract class RegionTestCase extends CacheTestCase {
         break;
       Assert.assertTrue(System.currentTimeMillis() <= tilt,
           "Region failed to destroy");
-      pause(100);
+      pause(10);
     }
   }  
 
@@ -2704,7 +2704,7 @@ public abstract class RegionTestCase extends CacheTestCase {
     
     vm0.invoke(new CacheSerializableRunnable("testRegionTtlInvalidate") {
       public void run2() throws CacheException {
-        final int timeout = 50; // ms
+        final int timeout = 22; // ms
         final Object key = "KEY";
         final Object value = "VALUE";
         
@@ -2736,8 +2736,8 @@ public abstract class RegionTestCase extends CacheTestCase {
         finally {
           ExpiryTask.permitExpiration();
         }
-        waitForInvalidate(entry, tilt);
-        waitForInvalidate(region.getEntry("k2"), tilt);
+        waitForInvalidate(entry, tilt, 10);
+        waitForInvalidate(region.getEntry("k2"), tilt, 10);
       }
     });
   }
@@ -2753,7 +2753,7 @@ public abstract class RegionTestCase extends CacheTestCase {
       return;
     
     final String name = this.getUniqueName();
-    final int timeout = 700; // ms
+    final int timeout = 22; // ms
     final Object key = "KEY";
     final Object value = "VALUE";
     
@@ -3741,42 +3741,38 @@ public abstract class RegionTestCase extends CacheTestCase {
   throws CacheException, InterruptedException {
 
     final String name = this.getUniqueName();
-    final int timeout = 20; // ms
-    final int hugeTimeout = Integer.MAX_VALUE;
-    final ExpirationAttributes expire =
-            new ExpirationAttributes(timeout, ExpirationAction.INVALIDATE);
-    final ExpirationAttributes hugeExpire =
-            new ExpirationAttributes(hugeTimeout, ExpirationAction.INVALIDATE);
+            ;
     final Object key = "KEY";
     final Object value = "VALUE";
     
     AttributesFactory factory = new AttributesFactory(getRegionAttributes());
     factory.setStatisticsEnabled(true);
     RegionAttributes attrs = factory.create();
-    Region region = null;
-    long tilt;
+    LocalRegion region = null;
     System.setProperty(LocalRegion.EXPIRY_MS_PROPERTY, "true");
     try {
-      region = createRegion(name, attrs);
+      region = (LocalRegion) createRegion(name, attrs);
 
       region.create(key, value);
-      tilt = System.currentTimeMillis() + timeout;
 
       // Now go from no timeout to a timeout
       Region.Entry entry = region.getEntry(key);
       assertEquals(value, entry.getValue());
-      region.getAttributesMutator().setRegionIdleTimeout(expire);
-      waitForInvalidate(entry, tilt);
-
-      // Now go from a big timeout to a short one
-      region.getAttributesMutator().setRegionIdleTimeout(hugeExpire);
+      region.getAttributesMutator().setRegionIdleTimeout(new ExpirationAttributes(12000/*ms*/, ExpirationAction.INVALIDATE));
       region.put(key, value);
-      tilt = System.currentTimeMillis() + timeout;
-      entry = region.getEntry(key);
-      pause(timeout * 2);
-      assertEquals(value, entry.getValue());
-      region.getAttributesMutator().setRegionIdleTimeout(expire);
-      waitForInvalidate(entry, tilt);
+      long tilt = System.currentTimeMillis();
+
+      ExpiryTask expiryTask = region.getRegionIdleExpiryTask();
+      long mediumExpiryTime = expiryTask.getExpirationTime();
+      region.getAttributesMutator().setRegionIdleTimeout(new ExpirationAttributes(999000/*ms*/, ExpirationAction.INVALIDATE));
+      expiryTask = region.getRegionIdleExpiryTask();
+      long hugeExpiryTime = expiryTask.getExpirationTime();
+      region.getAttributesMutator().setRegionIdleTimeout(new ExpirationAttributes(20/*ms*/, ExpirationAction.INVALIDATE));
+      expiryTask = region.getRegionIdleExpiryTask();
+      long shortExpiryTime = expiryTask.getExpirationTime();
+      waitForInvalidate(entry, tilt+20, 10);
+      assertTrue("expected hugeExpiryTime=" + hugeExpiryTime + " to be > than mediumExpiryTime=" + mediumExpiryTime, (hugeExpiryTime - mediumExpiryTime) > 0);
+      assertTrue("expected mediumExpiryTime=" + mediumExpiryTime + " to be > than shortExpiryTime=" + shortExpiryTime, (mediumExpiryTime - shortExpiryTime) > 0);
     }
     finally {
       System.getProperties().remove(LocalRegion.EXPIRY_MS_PROPERTY);
@@ -3791,8 +3787,10 @@ public abstract class RegionTestCase extends CacheTestCase {
   public void testRegionIdleInvalidate()
   throws InterruptedException, CacheException {
 
-    if(getRegionAttributes().getPartitionAttributes() != null)
+    if (getRegionAttributes().getPartitionAttributes() != null) {
+      // PR does not support INVALID ExpirationAction
       return;
+    }
     
     final String name = this.getUniqueName();
     final String subname = this.getUniqueName() + "-SUB";
@@ -3840,70 +3838,46 @@ public abstract class RegionTestCase extends CacheTestCase {
           tilt = System.currentTimeMillis() + timeout;
           entry = region.getEntry(key);
           assertEquals(value, entry.getValue());
-          if(!region.getAttributes().getDataPolicy().withPartitioning()){
-            // Do not create subregionsif parent region is a Partitioned Region 
-            sub = region.createSubregion(subname, subRegAttrs);
-          }
+          sub = region.createSubregion(subname, subRegAttrs);
         } 
         finally {
           System.getProperties().remove(LocalRegion.EXPIRY_MS_PROPERTY);
           ExpiryTask.permitExpiration();
         }
-        waitForInvalidate(entry, tilt);
+        waitForInvalidate(entry, tilt, 10);
         
         assertTrue(list.waitForInvocation(333));
         
-        pause(timeout);
-        assertFalse(list.wasInvoked());
 
+        // The next phase of the test verifies that a get will cause the
+        // expiration time to be extended.
+        // For this phase we don't worry about actually expiring but just
+        // making sure the expiration time gets extended.
+        region.getAttributesMutator().setRegionIdleTimeout(new ExpirationAttributes(9000/*ms*/, ExpirationAction.INVALIDATE));
+        
+        LocalRegion lr = (LocalRegion) region;
         {
-          long endTime = System.currentTimeMillis() + (timeout * 2);
+          ExpiryTask expiryTask = lr.getRegionIdleExpiryTask();
           region.put(key, value);
-          do {
-            tilt = System.currentTimeMillis() + timeout; // *earliest* time it can expire
-            region.get(key); // access *might* prevent idle expiration, if it occurs early enough.
-
-            // ...but a huge delay could cause an invalidation before we
-            // had a chance to do the get.
-            boolean wasInvoked = list.wasInvoked();
-            if (!value.equals(entry.getValue()) || wasInvoked) {
-              long now = System.currentTimeMillis(); // Did it take too long?
-              assertTrue("Entry invalidated " + (tilt - now) 
-                  + " ms prematurely (" + entry.getValue() + ")" + " value="+ value + " entryValue=" + entry.getValue() + " wasInvoked=" + wasInvoked, 
-                  now >= tilt);
-              
-              // So it invalidated due to machine latencies.  Get out.
-              break;
-            }
-          } while (System.currentTimeMillis() < endTime);
-        }
-        waitForInvalidate(entry, tilt);
-        assertTrue(list.waitForInvocation(333));
+          long createExpiry = expiryTask.getExpirationTime();
+          waitForExpiryClockToChange(lr);
+          region.put(key, "VALUE2");
+          long putExpiry = expiryTask.getExpirationTime();
+          assertTrue("expected putExpiry=" + putExpiry + " to be > than createExpiry=" + createExpiry, (putExpiry - createExpiry) > 0);
+          waitForExpiryClockToChange(lr);
+          region.get(key);
+          long getExpiry = expiryTask.getExpirationTime();
+          assertTrue("expected getExpiry=" + getExpiry + " to be > than putExpiry=" + putExpiry, (getExpiry - putExpiry) > 0);
         
-        region.put(key, value);
-        assertEquals(value, entry.getValue());
-        if(!region.getAttributes().getDataPolicy().withPartitioning()){
-          long endTime = System.currentTimeMillis() + (timeout * 2);
+          waitForExpiryClockToChange(lr);
           sub.put(key, value);
-          do {
-            tilt = System.currentTimeMillis() + timeout;
-            sub.get(key, value); // get should prevent idle expiration
-            
-            // ...but a huge delay could cause an invalidation before we
-            // had a chance to do the get.
-            if (!value.equals(entry.getValue()) || list.wasInvoked()) {
-              long now = System.currentTimeMillis(); // Did it take too long?
-              assertTrue("Entry invalidated " + (tilt - now) 
-                  + " ms prematurely (" + entry.getValue() + ")", 
-                  now >= tilt);
-              
-              // So it invalidated due to machine latencies.  Get out.
-              break;
-            }
-          } while (System.currentTimeMillis() < endTime);
+          long subPutExpiry = expiryTask.getExpirationTime();
+          assertTrue("expected subPutExpiry=" + subPutExpiry + " to be > than getExpiry=" + getExpiry, (subPutExpiry - getExpiry) > 0);
+          waitForExpiryClockToChange(lr);
+          sub.get(key);
+          long subGetExpiry = expiryTask.getExpirationTime();
+          assertTrue("expected subGetExpiry=" + subGetExpiry + " to be > than subPutExpiry=" + subPutExpiry, (subGetExpiry - subPutExpiry) > 0);
         }
-        waitForInvalidate(entry, tilt);
-        assertTrue(list.waitForInvocation(333));
       }
     });
   }
@@ -3920,7 +3894,7 @@ public abstract class RegionTestCase extends CacheTestCase {
       return;
     
     final String name = this.getUniqueName();
-    final int timeout = 800; // ms
+    final int timeout = 22; // ms
     final Object key = "KEY";
     final Object value = "VALUE";
     


[38/50] [abbrv] incubator-geode git commit: GEODE-179: fix race in testEntryTtlDestroyEvent

Posted by bs...@apache.org.
GEODE-179: fix race in testEntryTtlDestroyEvent

A race existed that caused no-ack regions to fail because they
did not see the entry expire.
This race was fixed by suspending expiration until the create
is seen on the remote side and then after permitting expiration
using a wait for the expire to happen.


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

Branch: refs/heads/feature/GEODE-77
Commit: e255c75e9be33e3d274c42822923ba3db0629739
Parents: 1c7cbb2
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Aug 5 16:46:45 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Mon Aug 10 14:06:32 2015 -0700

----------------------------------------------------------------------
 .../gemfire/cache30/MultiVMRegionTestCase.java  | 94 +++++++++++++-------
 1 file changed, 63 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e255c75e/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
index 5e605dd..ecec932 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/MultiVMRegionTestCase.java
@@ -45,6 +45,7 @@ import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.AttributesMutator;
 import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheEvent;
 import com.gemstone.gemfire.cache.CacheException;
 import com.gemstone.gemfire.cache.CacheFactory;
 import com.gemstone.gemfire.cache.CacheListener;
@@ -84,6 +85,7 @@ import com.gemstone.gemfire.internal.InternalDataSerializer;
 import com.gemstone.gemfire.internal.InternalInstantiator;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.EntryExpiryTask;
+import com.gemstone.gemfire.internal.cache.ExpiryTask;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
@@ -3822,7 +3824,7 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
         return;
       
       final String name = this.getUniqueName();
-      final int timeout = 40; // ms
+      final int timeout = 22; // ms
       final Object key = "KEY";
       final Object value = "VALUE";
 
@@ -3857,81 +3859,111 @@ public abstract class MultiVMRegionTestCase extends RegionTestCase {
       }
 
 
-      SerializableRunnable create = new CacheSerializableRunnable("Populate") {
+      SerializableRunnable createRegion = new CacheSerializableRunnable("Create with Listener") {
         public void run2() throws CacheException {
           AttributesFactory fac = new AttributesFactory(getRegionAttributes());
           fac.addCacheListener(destroyListener = new DestroyListener());
-          Region region = null;
-          System.setProperty(LocalRegion.EXPIRY_MS_PROPERTY, "true");
-          try {
-            region = createRegion(name, fac.create());
-          } 
-          finally {
-            System.getProperties().remove(LocalRegion.EXPIRY_MS_PROPERTY);
-          }
+          createRegion(name, fac.create());
         }
       };
 
-      vm1.invoke(create);
+      vm1.invoke(createRegion);
 
       vm0.invoke(new CacheSerializableRunnable("Create with TTL") {
           public void run2() throws CacheException {
             AttributesFactory factory = new AttributesFactory(getRegionAttributes());
-            final boolean partitioned = getRegionAttributes().getPartitionAttributes() != null ||
-            getRegionAttributes().getDataPolicy().withPartitioning();
             factory.setStatisticsEnabled(true);
             ExpirationAttributes expire =
               new ExpirationAttributes(timeout,
                                        ExpirationAction.DESTROY);
             factory.setEntryTimeToLive(expire);
-            if (!getRegionAttributes().getDataPolicy().withReplication()&& ! partitioned) {
+            if (!getRegionAttributes().getDataPolicy().withReplication()) {
               factory.setDataPolicy(DataPolicy.NORMAL);
               factory.setSubscriptionAttributes(new SubscriptionAttributes(InterestPolicy.ALL));
             }
             System.setProperty(LocalRegion.EXPIRY_MS_PROPERTY, "true");
             try {
               createRegion(name, factory.create());
+              ExpiryTask.suspendExpiration();
+              // suspend to make sure we can see that the put is distributed to this member
             } 
             finally {
               System.getProperties().remove(LocalRegion.EXPIRY_MS_PROPERTY);
             }
           }
         });
+      
+      try {
 
-      // let create finish before setting up other cache
+      // let region create finish before doing put
       //pause(10);
 
       vm1.invoke(new SerializableCallable() {
         public Object call() throws Exception {
           Region region = getRootRegion().getSubregion(name);
+          DestroyListener dl = (DestroyListener)region.getAttributes().getCacheListeners()[0];
+          dl.enableEventHistory();
           region.put(key, value);
           // reset listener after create event
-          assertTrue(((DestroyListener)region.getAttributes().
-            getCacheListeners()[0]).wasInvoked());
+          assertTrue(dl.wasInvoked());
+          List<CacheEvent> history = dl.getEventHistory();
+          CacheEvent ce = history.get(0);
+          dl.disableEventHistory();
+          assertEquals(Operation.CREATE, ce.getOperation());
           return null;
         }
       });
-
+      vm0.invoke(new CacheSerializableRunnable("Check create received from vm1") {
+        public void run2() throws CacheException {
+          final Region region = getRootRegion().getSubregion(name);
+          WaitCriterion waitForCreate = new WaitCriterion() {
+            public boolean done() {
+              return region.getEntry(key) != null;
+            }
+            public String description() {
+              return "never saw create of " + key;
+            }
+          };
+          DistributedTestCase.waitForCriterion(waitForCreate, 3000, 10, true);
+        }
+      });
+      
+      } finally {
+        vm0.invoke(new CacheSerializableRunnable("resume expiration") {
+          public void run2() throws CacheException {
+            ExpiryTask.permitExpiration();
+          }
+        });
+      }
+      
+      // now wait for it to expire
       vm0.invoke(new CacheSerializableRunnable("Check local destroy") {
           public void run2() throws CacheException {
-            Region region =
-              getRootRegion().getSubregion(name);
-            int retries = 10;
-            while (region.getEntry(key) != null && retries-- > 0) {
-              pause(timeout);
-            }
-            assertNull(region.getEntry(key));
+            final Region region = getRootRegion().getSubregion(name);
+            WaitCriterion waitForExpire = new WaitCriterion() {
+              public boolean done() {
+                return region.getEntry(key) == null;
+              }
+              public String description() {
+                return "never saw expire of " + key + " entry=" + region.getEntry(key);
+              }
+            };
+            DistributedTestCase.waitForCriterion(waitForExpire, 4000, 10, true);
           }
         });
 
       vm1.invoke(new CacheSerializableRunnable("Verify destroyed and event") {
           public void run2() throws CacheException {
-            Region region = getRootRegion().getSubregion(name);
-            int retries = 10;
-            while (region.getEntry(key) != null && retries-- > 0) {
-              pause(timeout);
-            }
-            assertNull(region.getEntry(key));
+            final Region region = getRootRegion().getSubregion(name);
+            WaitCriterion waitForExpire = new WaitCriterion() {
+              public boolean done() {
+                return region.getEntry(key) == null;
+              }
+              public String description() {
+                return "never saw expire of " + key + " entry=" + region.getEntry(key);
+              }
+            };
+            DistributedTestCase.waitForCriterion(waitForExpire, 4000, 10, true);
             assertTrue(destroyListener.waitForInvocation(555));
             assertTrue(((DestroyListener)destroyListener).eventIsExpiration);
           }


[07/50] [abbrv] incubator-geode git commit: GEODE-135: Added UnclosableInputStream for SAX parser

Posted by bs...@apache.org.
GEODE-135: Added UnclosableInputStream for SAX parser

This closes #12

In order to block the parser from closing the stream, we wrap the
InputStream in a filter, i.e., UnclosableInputStream, whose close()
function does nothing.


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

Branch: refs/heads/feature/GEODE-77
Commit: 405daf57eea94560e5700c026095b837cf823c93
Parents: 94939c1
Author: lzh <lz...@ca.ibm.com>
Authored: Tue Jul 28 13:46:48 2015 -0400
Committer: Dan Smith <ds...@pivotal.io>
Committed: Wed Jul 29 09:46:31 2015 -0700

----------------------------------------------------------------------
 .../internal/cache/xmlcache/CacheXmlParser.java | 26 +++++++++++++++++++-
 1 file changed, 25 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/405daf57/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
index 9f2cbc0..9b2f5ed 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheXmlParser.java
@@ -182,13 +182,37 @@ public class CacheXmlParser extends CacheXml implements ContentHandler {
    *
    */
   public static CacheXmlParser parse(InputStream is) {
+	  
+    /**
+     * The API doc
+     * http://java.sun.com/javase/6/docs/api/org/xml/sax/InputSource.html for
+     * the SAX InputSource says: "... standard processing of both byte and
+     * character streams is to close them on as part of end-of-parse cleanup, so
+     * applications should not attempt to re-use such streams after they have
+     * been handed to a parser."
+     *
+     * In order to block the parser from closing the stream, we wrap the
+     * InputStream in a filter, i.e., UnclosableInputStream, whose close()
+     * function does nothing.
+     * 
+     */
+    class UnclosableInputStream extends BufferedInputStream {
+      public UnclosableInputStream(InputStream stream) {
+        super(stream);
+      }
+
+      @Override
+      public void close() {
+      }
+    }
+	 
     CacheXmlParser handler = new CacheXmlParser();
     try {
       SAXParserFactory factory = SAXParserFactory.newInstance();
       factory.setFeature(DISALLOW_DOCTYPE_DECL_FEATURE, true);
       factory.setValidating(true);
       factory.setNamespaceAware(true);     
-      BufferedInputStream bis = new BufferedInputStream(is);
+      UnclosableInputStream bis = new UnclosableInputStream(is);
       try {
         SAXParser parser = factory.newSAXParser();
         // Parser always reads one buffer plus a little extra worth before


[21/50] [abbrv] incubator-geode git commit: GEODE-181: Add new unit and integration tests for FastLogger

Posted by bs...@apache.org.
GEODE-181: Add new unit and integration tests for FastLogger

Add new TestSuites for logging tests. Extract default config testing to
FastLoggerWithDefaultConfigJUnitTest. Move FastLoggerJUnitTest integration
tests to FastLoggerIntegrationJUnitTest. Create new unit tests in
FastLoggerJUnitTest. Update tests to full JUnit 4 syntax and use Rules
(GEODE-89).

Add mockito dependency to build for unit testing.

Completes: GEODE-89, GEODE-181, GEODE-10


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

Branch: refs/heads/feature/GEODE-77
Commit: 1c4cd8f7909ba20f3c321001d26424e6853fe6a9
Parents: fa9bd37
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue Aug 4 11:39:44 2015 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Aug 4 14:35:24 2015 -0700

----------------------------------------------------------------------
 build.gradle                                    |   1 +
 .../gemfire/internal/logging/LogService.java    |  53 +-
 .../internal/logging/log4j/Configurator.java    |  65 ++-
 .../internal/logging/log4j/FastLogger.java      |  48 +-
 .../DistributedSystemLogFileJUnitTest.java      |  20 +-
 .../logging/LoggingIntegrationTestSuite.java    |  14 +
 .../internal/logging/LoggingUnitTestSuite.java  |  12 +
 .../log4j/FastLoggerIntegrationJUnitTest.java   | 557 +++++++++++++++++++
 .../logging/log4j/FastLoggerJUnitTest.java      | 274 ++++-----
 .../FastLoggerWithDefaultConfigJUnitTest.java   |  74 +++
 .../log4j/Log4jIntegrationTestSuite.java        |  12 +
 .../logging/log4j/Log4jUnitTestSuite.java       |  16 +
 12 files changed, 914 insertions(+), 232 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index c82e82a..42b4a14 100755
--- a/build.gradle
+++ b/build.gradle
@@ -278,6 +278,7 @@ subprojects {
     testCompile 'com.github.stefanbirkner:system-rules:1.9.0'
     testCompile 'edu.umd.cs.mtc:multithreadedtc:1.01'
     testCompile 'junit:junit:4.12'
+    testCompile 'org.mockito:mockito-core:1.10.19'
     testCompile 'org.hamcrest:hamcrest-all:1.3'
     testCompile 'org.jmock:jmock:2.8.1'
     testCompile 'org.jmock:jmock-junit4:2.8.1'

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
index 6298cf6..a4a399d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
@@ -12,6 +12,7 @@ import java.beans.PropertyChangeListener;
 import java.io.File;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -21,9 +22,11 @@ import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.Filter.Result;
 import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.AppenderRef;
 import org.apache.logging.log4j.core.config.Configuration;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
 import org.apache.logging.log4j.core.config.LoggerConfig;
+import org.apache.logging.log4j.core.filter.AbstractFilterable;
 import org.apache.logging.log4j.core.lookup.Interpolator;
 import org.apache.logging.log4j.core.lookup.StrSubstitutor;
 import org.apache.logging.log4j.status.StatusLogger;
@@ -78,10 +81,9 @@ public class LogService extends LogManager {
   private static void init() {
     setLog4jConfigFileProperty();
     LoggerContext context = ((org.apache.logging.log4j.core.Logger) LogManager.getLogger(BASE_LOGGER_NAME, GemFireParameterizedMessageFactory.INSTANCE)).getContext();
-    context.reconfigure();
     context.removePropertyChangeListener(propertyChangeListener);
     context.addPropertyChangeListener(propertyChangeListener);
-    setFastLoggerDebugAvailableFlag();
+    context.reconfigure(); // propertyChangeListener invokes configureFastLoggerDelegating
     configureLoggers(false, false);
   }
   
@@ -117,15 +119,17 @@ public class LogService extends LogManager {
   /**
    * Check to see if the user has specified a Log4j configuration file.  If not, attempt
    * to find a GemFire Log4j configuration file in various locations.
+   * 
+   * @return true if log4j.configurationFile property was set; false if it was unchanged
    */
-  private static final void setLog4jConfigFileProperty() {
+  private static final boolean setLog4jConfigFileProperty() {
     // fix bug #52175
     final URL configInClasspath = ConfigLocator.findConfigInClasspath();
     if (configInClasspath != null ) {
       // Log4J 2 will find the configuration file in classpath so do nothing
       configFileInformation = "Using log4j configuration found in classpath: '" + configInClasspath.toString() + "'";
       StatusLogger.getLogger().info(configFileInformation);
-      return;
+      return false;
     }
     
     // If the user set the log4j system property then there's nothing else to do.
@@ -137,16 +141,14 @@ public class LogService extends LogManager {
         //We will let log4j2 handle the null case and just log what file we are attempting to use
         configFileInformation = "Using log4j configuration file specified by " + ConfigurationFactory.CONFIGURATION_FILE_PROPERTY + ": '" + configFileName + "'";
         StatusLogger.getLogger().info(configFileInformation);
-        return;
+        return false;
       }
       else {
         //If the resource can be found and in cases where the resource is in gemfire jar,
         //we set the log location to the file that was found
-        String configFilePropertyValue = configUrl.toString();
-        System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, configFilePropertyValue);
-        configFileInformation = "Using log4j configuration file specified by " + ConfigurationFactory.CONFIGURATION_FILE_PROPERTY + ": '" + configFilePropertyValue + "'";
+        configFileInformation = "Using log4j configuration file specified by " + ConfigurationFactory.CONFIGURATION_FILE_PROPERTY + ": '" + configFileName + "'";
         StatusLogger.getLogger().info(configFileInformation);
-        return;
+        return true;
       }
     }
     
@@ -164,7 +166,7 @@ public class LogService extends LogManager {
       System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, configFilePropertyValue);
       configFileInformation = "Setting " + ConfigurationFactory.CONFIGURATION_FILE_PROPERTY + " to specify log4j configuration file in current directory: '" + configFilePropertyValue + "'";
       StatusLogger.getLogger().debug(configFileInformation);
-      return;
+      return true;
     }
 
     // Use the log4j config file found on the classpath in the gemfire jar file.
@@ -173,7 +175,7 @@ public class LogService extends LogManager {
     System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, configFilePropertyValue);
     configFileInformation = "Setting " + ConfigurationFactory.CONFIGURATION_FILE_PROPERTY + " to specify log4j configuration file: '" + configFilePropertyValue + "'";
     StatusLogger.getLogger().info(configFileInformation);
-    return;
+    return true;
   }
   
   public static String getConfigInformation() {
@@ -244,25 +246,18 @@ public class LogService extends LogManager {
     return new Throwable().getStackTrace()[depth].getClassName();
   }
 
-  public static void setFastLoggerDebugAvailableFlag() {
+  public static void configureFastLoggerDelegating() {
     final Configuration config = ((org.apache.logging.log4j.core.Logger)
-        LogManager.getLogger(BASE_LOGGER_NAME, GemFireParameterizedMessageFactory.INSTANCE)).getContext().getConfiguration();
+        LogManager.getLogger(ROOT_LOGGER_NAME, GemFireParameterizedMessageFactory.INSTANCE)).getContext().getConfiguration();
     
-    // Check for debug/trace and filters on each logger
-    for (LoggerConfig loggerConfig : config.getLoggers().values()) {
-      if (loggerConfig.getName().startsWith(BASE_LOGGER_NAME) 
-          && ((loggerConfig.hasFilter() && !GEMFIRE_VERBOSE_FILTER.equals(loggerConfig.getFilter().toString())) 
-          || loggerConfig.getLevel().isLessSpecificThan(Level.DEBUG))){
-        FastLogger.setDebugAvailable(true);
-        return;
-      }
-    }
-    
-    // Check for context filters
-    if (config.hasFilter()) {
-      FastLogger.setDebugAvailable(true);
+    if (Configurator.hasContextWideFilter(config) || 
+        Configurator.hasAppenderFilter(config) || 
+        Configurator.hasDebugOrLower(config) || 
+        Configurator.hasLoggerFilter(config) || 
+        Configurator.hasAppenderRefFilter(config)) {
+      FastLogger.setDelegating(true);
     } else {
-      FastLogger.setDebugAvailable(false);
+      FastLogger.setDelegating(false);
     }
   }
   
@@ -274,11 +269,11 @@ public class LogService extends LogManager {
           evt.getPropertyName());
       
       if (evt.getPropertyName().equals(LoggerContext.PROPERTY_CONFIG)) {
-        setFastLoggerDebugAvailableFlag();
+        configureFastLoggerDelegating();
       }
     }
   }
-
+  
   public static void setBaseLogLevel(Level level) {
     if (isUsingGemFireDefaultConfig()) {
       Configurator.setLevel(ROOT_LOGGER_NAME, level);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/Configurator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/Configurator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/Configurator.java
index c7ae945..ccb2639 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/Configurator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/Configurator.java
@@ -6,11 +6,13 @@ import java.util.Set;
 
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.LoggerContext;
 import org.apache.logging.log4j.core.config.AppenderRef;
 import org.apache.logging.log4j.core.config.Configuration;
 import org.apache.logging.log4j.core.config.LoggerConfig;
 import org.apache.logging.log4j.core.config.Property;
+import org.apache.logging.log4j.core.filter.AbstractFilterable;
 
 import com.gemstone.gemfire.internal.logging.LogService;
 
@@ -34,7 +36,7 @@ public class Configurator {
     context.updateLoggers();
     
     if (level.isLessSpecificThan(Level.DEBUG)) {
-      LogService.setFastLoggerDebugAvailableFlag();
+      LogService.configureFastLoggerDelegating();
     }
   }
   
@@ -110,4 +112,65 @@ public class Configurator {
     }
     return logConfig;
   }
+
+  public static boolean hasContextWideFilter(final Configuration config) {
+    return config.hasFilter();
+  }
+  
+  public static boolean hasAppenderFilter(final Configuration config) {
+    for (Appender appender : config.getAppenders().values()) {
+      if (appender instanceof AbstractFilterable) {
+        if (((AbstractFilterable) appender).hasFilter()) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+  
+  public static boolean hasDebugOrLower(final Configuration config) {
+    for (LoggerConfig loggerConfig : config.getLoggers().values()) {
+      boolean isDebugOrLower = loggerConfig.getLevel().isLessSpecificThan(Level.DEBUG);
+      if (isDebugOrLower) {
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  public static boolean hasLoggerFilter(final Configuration config) {
+    for (LoggerConfig loggerConfig : config.getLoggers().values()) {
+      boolean isRoot = loggerConfig.getName().equals("");
+      boolean isGemFire = loggerConfig.getName().startsWith(LogService.BASE_LOGGER_NAME);
+      boolean hasFilter = loggerConfig.hasFilter();
+      boolean isGemFireVerboseFilter = hasFilter && LogService.GEMFIRE_VERBOSE_FILTER.equals(loggerConfig.getFilter().toString());
+      
+      if (isRoot || isGemFire) {
+        // check for Logger Filter
+        if (hasFilter && !isGemFireVerboseFilter) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+  
+  public static boolean hasAppenderRefFilter(final Configuration config) {
+    for (LoggerConfig loggerConfig : config.getLoggers().values()) {
+      boolean isRoot = loggerConfig.getName().equals("");
+      boolean isGemFire = loggerConfig.getName().startsWith(LogService.BASE_LOGGER_NAME);
+      boolean hasFilter = loggerConfig.hasFilter();
+      boolean isGemFireVerboseFilter = hasFilter && LogService.GEMFIRE_VERBOSE_FILTER.equals(loggerConfig.getFilter().toString());
+      
+      if (isRoot || isGemFire) {
+        // check for AppenderRef Filter
+        for (AppenderRef appenderRef : loggerConfig.getAppenderRefs()) {
+          if (appenderRef.getFilter() != null) {
+            return true;
+          }
+        }
+      }
+    }
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/FastLogger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/FastLogger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/FastLogger.java
index 21d7965..9583429 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/FastLogger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/FastLogger.java
@@ -1,6 +1,5 @@
 package com.gemstone.gemfire.internal.logging.log4j;
 
-import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.Logger;
 import org.apache.logging.log4j.Marker;
 import org.apache.logging.log4j.message.MessageFactory;
@@ -9,8 +8,9 @@ import org.apache.logging.log4j.spi.ExtendedLoggerWrapper;
 import org.apache.logging.log4j.status.StatusLogger;
 
 /**
- * Overrides is enabled checks for log levels below INFO to avoid performance
- * penalties when the log level is INFO or above.
+ * Overrides is-enabled checks for log levels below INFO to avoid performance
+ * penalties when the log level is INFO or above. If delegating is true
+ * then it will always delegate to ExtendedLoggerWrapper for is-enabled checks.
  * 
  * @author Kirk Lund
  * @author David Hoots
@@ -18,7 +18,7 @@ import org.apache.logging.log4j.status.StatusLogger;
 public class FastLogger extends ExtendedLoggerWrapper {
   private static final long serialVersionUID = 7084130827962463327L;
 
-  private static volatile boolean debugAvailable = true;
+  private static volatile boolean delegating = true;
   
   public FastLogger(final Logger logger) {
     this((ExtendedLogger) logger, logger.getName(), logger.getMessageFactory());
@@ -28,55 +28,33 @@ public class FastLogger extends ExtendedLoggerWrapper {
     super(logger, name, messageFactory);
   }
 
-  public static void setDebugAvailable(final boolean newValue) {
-    StatusLogger.getLogger().debug("Setting debugAvailable to {}", newValue);
-    debugAvailable = newValue;
+  public static void setDelegating(final boolean newValue) {
+    StatusLogger.getLogger().debug("Setting delegating to {}", newValue);
+    delegating = newValue;
   }
   
-  /**
-   * Checks whether this Logger is enabled for the {@link Level#DEBUG DEBUG} Level.
-   *
-   * @return boolean - {@code true} if this Logger is enabled for level DEBUG, {@code false} otherwise.
-   */
   @Override
   public boolean isDebugEnabled() {
-    return debugAvailable && super.isDebugEnabled();
+    return delegating && super.isDebugEnabled();
   }
 
-  /**
-   * Checks whether this Logger is enabled for the {@link Level#DEBUG DEBUG} Level.
-   *
-   * @param marker The marker data specific to this log statement.
-   * @return boolean - {@code true} if this Logger is enabled for level DEBUG, {@code false} otherwise.
-   */
   @Override
   public boolean isDebugEnabled(final Marker marker) {
-    return debugAvailable && super.isDebugEnabled(marker);
+    return delegating && super.isDebugEnabled(marker);
   }
 
-  /**
-   * Checks whether this Logger is enabled for the {@link Level#TRACE TRACE} level.
-   *
-   * @return boolean - {@code true} if this Logger is enabled for level TRACE, {@code false} otherwise.
-   */
   @Override
   public boolean isTraceEnabled() {
-    return debugAvailable && super.isTraceEnabled();
+    return delegating && super.isTraceEnabled();
   }
 
-  /**
-   * Checks whether this Logger is enabled for the {@link Level#TRACE TRACE} level.
-   *
-   * @param marker The marker data specific to this log statement.
-   * @return boolean - {@code true} if this Logger is enabled for level TRACE, {@code false} otherwise.
-   */
   @Override
   public boolean isTraceEnabled(final Marker marker) {
-    return debugAvailable && super.isTraceEnabled(marker);
+    return delegating && super.isTraceEnabled(marker);
   }
   
-  public boolean isDebugAvailable() {
-    return debugAvailable;
+  public boolean isDelegating() {
+    return delegating;
   }
   
   public Logger getExtendedLogger() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
index 9c7ba58..8dfd5a4 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/DistributedSystemLogFileJUnitTest.java
@@ -487,7 +487,7 @@ public class DistributedSystemLogFileJUnitTest {
     assertTrue(logWriter.fineEnabled());
     assertTrue(((LogWriterLogger)logWriter).isDebugEnabled());
     assertTrue(logWriter instanceof FastLogger);
-    assertTrue(((FastLogger)logWriter).isDebugAvailable());
+    assertTrue(((FastLogger)logWriter).isDelegating());
 
     DistributedTestCase.waitForCriterion(new WaitCriterion() {
       @Override
@@ -703,7 +703,7 @@ public class DistributedSystemLogFileJUnitTest {
     assertTrue(logWriter.fineEnabled());
     assertTrue(((LogWriterLogger)logWriter).isDebugEnabled());
     assertTrue(logWriter instanceof FastLogger);
-    assertTrue(((FastLogger)logWriter).isDebugAvailable());
+    assertTrue(((FastLogger)logWriter).isDelegating());
 
     DistributedTestCase.waitForCriterion(new WaitCriterion() {
       @Override
@@ -942,8 +942,8 @@ public class DistributedSystemLogFileJUnitTest {
     //Because debug available is a static volatile, it is shared between the two writers
     //However we should not see any debug level logging due to the config level set in 
     //the log writer itself
-    assertTrue(((FastLogger)securityLogWriter).isDebugAvailable());
-    assertTrue(((FastLogger)logWriter).isDebugAvailable());
+    assertTrue(((FastLogger)securityLogWriter).isDelegating());
+    assertTrue(((FastLogger)logWriter).isDelegating());
 
     
     DistributedTestCase.waitForCriterion(new WaitCriterion() {
@@ -1058,8 +1058,8 @@ public class DistributedSystemLogFileJUnitTest {
     assertTrue(((LogWriterLogger)logWriter).isDebugEnabled());
     assertTrue(securityLogWriter instanceof FastLogger);
     assertTrue(logWriter instanceof FastLogger);
-    assertTrue(((FastLogger)securityLogWriter).isDebugAvailable());
-    assertTrue(((FastLogger)logWriter).isDebugAvailable());
+    assertTrue(((FastLogger)securityLogWriter).isDelegating());
+    assertTrue(((FastLogger)logWriter).isDelegating());
 
     DistributedTestCase.waitForCriterion(new WaitCriterion() {
       @Override
@@ -1230,8 +1230,8 @@ public class DistributedSystemLogFileJUnitTest {
     assertTrue(((LogWriterLogger)logWriter).isDebugEnabled());
     assertTrue(securityLogWriter instanceof FastLogger);
     assertTrue(logWriter instanceof FastLogger);
-    assertTrue(((FastLogger)securityLogWriter).isDebugAvailable());
-    assertTrue(((FastLogger)logWriter).isDebugAvailable());
+    assertTrue(((FastLogger)securityLogWriter).isDelegating());
+    assertTrue(((FastLogger)logWriter).isDelegating());
 
     DistributedTestCase.waitForCriterion(new WaitCriterion() {
       @Override
@@ -1379,8 +1379,8 @@ public class DistributedSystemLogFileJUnitTest {
     assertFalse(((LogWriterLogger)logWriter).isDebugEnabled());
     assertTrue(securityLogWriter instanceof FastLogger);
     assertTrue(logWriter instanceof FastLogger);
-    assertTrue(((FastLogger)securityLogWriter).isDebugAvailable());
-    assertTrue(((FastLogger)logWriter).isDebugAvailable());
+    assertTrue(((FastLogger)securityLogWriter).isDelegating());
+    assertTrue(((FastLogger)logWriter).isDelegating());
 
     DistributedTestCase.waitForCriterion(new WaitCriterion() {
       @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingIntegrationTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingIntegrationTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingIntegrationTestSuite.java
new file mode 100755
index 0000000..1efaa59
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingIntegrationTestSuite.java
@@ -0,0 +1,14 @@
+package com.gemstone.gemfire.internal.logging;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+  DistributedSystemLogFileJUnitTest.class,
+  LocatorLogFileJUnitTest.class,
+  LogServiceJUnitTest.class,
+  MergeLogFilesJUnitTest.class,
+})
+public class LoggingIntegrationTestSuite {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingUnitTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingUnitTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingUnitTestSuite.java
new file mode 100755
index 0000000..cd39a9d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/LoggingUnitTestSuite.java
@@ -0,0 +1,12 @@
+package com.gemstone.gemfire.internal.logging;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+  LogWriterImplJUnitTest.class,
+  SortLogFileJUnitTest.class
+})
+public class LoggingUnitTestSuite {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerIntegrationJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerIntegrationJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerIntegrationJUnitTest.java
new file mode 100644
index 0000000..f1c7ab9
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerIntegrationJUnitTest.java
@@ -0,0 +1,557 @@
+package com.gemstone.gemfire.internal.logging.log4j;
+
+import static org.hamcrest.CoreMatchers.*;
+import static org.junit.Assert.*;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+
+import org.apache.logging.log4j.Level;
+import org.apache.logging.log4j.Logger;
+import org.apache.logging.log4j.Marker;
+import org.apache.logging.log4j.MarkerManager;
+import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TemporaryFolder;
+
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Tests FastLogger isDebugEnabled and isTraceEnabled with various configurations.
+ * 
+ * For filters see https://logging.apache.org/log4j/2.0/manual/filters.html
+ * 
+ * @author Kirk Lund
+ * @author David Hoots
+ */
+@Category(IntegrationTest.class)
+public class FastLoggerIntegrationJUnitTest {
+
+  private static final String TEST_LOGGER_NAME = FastLogger.class.getPackage().getName();
+  private static final String ENABLED_MARKER_NAME = "ENABLED";
+  private static final String UNUSED_MARKER_NAME = "UNUSED";
+  
+  @Rule
+  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  
+  @Rule
+  public final TemporaryFolder temporaryFolder = new TemporaryFolder();
+  
+  private File configFile;
+  private Logger logger;
+  private LoggerContext appenderContext;
+  private Marker enabledMarker;
+  private Marker unusedMarker;
+  
+  @Before
+  public void setUp() throws Exception {
+    System.clearProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
+    this.configFile = new File(this.temporaryFolder.getRoot(), "log4j2-test.xml");
+    this.enabledMarker = MarkerManager.getMarker(ENABLED_MARKER_NAME);
+    this.unusedMarker = MarkerManager.getMarker(UNUSED_MARKER_NAME);
+    setUpLogService();
+  }
+  
+  @After
+  public void tearDown() {
+    System.clearProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
+    LogService.reconfigure();
+  }
+  
+  private void setUpLogService() throws Exception {
+    // Load a base config and do some sanity checks
+    writeSimpleConfigFile(this.configFile, Level.WARN);
+    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, this.configFile.toURI().toURL().toString());
+
+    LogService.reconfigure();
+    LogService.getLogger().getName(); // This causes the config file to be loaded
+    this.logger = LogService.getLogger(TEST_LOGGER_NAME);
+    this.appenderContext = ((org.apache.logging.log4j.core.Logger) LogService.getRootLogger()).getContext();
+    
+    assertThat(LogService.getLogger(LogService.BASE_LOGGER_NAME).getLevel(), is(Level.FATAL));
+    assertThat(this.logger, is(instanceOf(FastLogger.class)));
+    assertThat(this.logger.getLevel(), is(Level.WARN));
+  }
+  
+  @Test
+  public void debugConfigIsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+  }
+  @Test
+  public void traceConfigIsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.TRACE, expectDelegating(true));
+  }
+  @Test
+  public void infoConfigIsNotDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+  }
+  @Test
+  public void warnConfigIsNotDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.WARN, expectDelegating(false));
+  }
+  @Test
+  public void errorConfigIsNotDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.ERROR, expectDelegating(false));
+  }
+  @Test
+  public void fatalConfigIsNotDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.FATAL, expectDelegating(false));
+  }
+
+  @Test
+  public void fromDebugToInfoSetsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+  }
+  @Test
+  public void fromInfoToDebugUnsetsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+  }
+  
+  @Test
+  public void fromDebugToContextWideFilterKeepsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+    verifyIsDelegatingForContextWideFilter(Level.INFO, expectDelegating(true));
+  }
+  @Test
+  public void fromInfoToContextWideFilterSetsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+    verifyIsDelegatingForContextWideFilter(Level.INFO, expectDelegating(true));
+  }
+  @Test
+  public void fromContextWideFilterToInfoUnsetsDelegating() throws Exception {
+    verifyIsDelegatingForContextWideFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+  }
+  @Test
+  public void fromContextWideFilterToDebugKeepsDelegating() throws Exception {
+    verifyIsDelegatingForContextWideFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+  }
+
+  @Test
+  public void fromDebugToAppenderFilterKeepsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+    verifyIsDelegatingForAppenderFilter(Level.INFO, expectDelegating(true));
+  }
+  @Test
+  public void fromInfoToAppenderFilterSetsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+    verifyIsDelegatingForAppenderFilter(Level.INFO, expectDelegating(true));
+  }
+  @Test
+  public void fromAppenderFilterToInfoUnsetsDelegating() throws Exception {
+    verifyIsDelegatingForAppenderFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+  }
+  @Test
+  public void fromAppenderFilterToDebugKeepsDelegating() throws Exception {
+    verifyIsDelegatingForAppenderFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+  }
+  
+  @Test
+  public void fromDebugToLoggerFilterKeepsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+    verifyIsDelegatingForLoggerFilter(Level.INFO, expectDelegating(true));
+  }
+  @Test
+  public void fromInfoToLoggerFilterSetsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+    verifyIsDelegatingForLoggerFilter(Level.INFO, expectDelegating(true));
+  }
+  @Test
+  public void fromLoggerFilterToInfoUnsetsDelegating() throws Exception {
+    verifyIsDelegatingForLoggerFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+  }
+  @Test
+  public void fromLoggerFilterToDebugKeepsDelegating() throws Exception {
+    verifyIsDelegatingForLoggerFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+  }
+
+  @Test
+  public void fromDebugToAppenderRefFilterKeepsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+    verifyIsDelegatingForAppenderRefFilter(Level.INFO, expectDelegating(true));
+  }
+  @Test
+  public void fromInfoToAppenderRefFilterSetsDelegating() throws Exception {
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+    verifyIsDelegatingForAppenderRefFilter(Level.INFO, expectDelegating(true));
+  }
+  @Test
+  public void fromAppenderRefFilterToInfoUnsetsDelegating() throws Exception {
+    verifyIsDelegatingForAppenderRefFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForDebugOrLower(Level.INFO, expectDelegating(false));
+  }
+  @Test
+  public void fromAppenderRefFilterToDebugKeepsDelegating() throws Exception {
+    verifyIsDelegatingForAppenderRefFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForDebugOrLower(Level.DEBUG, expectDelegating(true));
+  }
+    
+  @Test
+  public void fromContextWideFilterToLoggerFilterKeepsDelegating() throws Exception {
+    verifyIsDelegatingForContextWideFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForLoggerFilter(Level.INFO, expectDelegating(true));
+  }
+  @Test
+  public void fromLoggerFilterToContextWideFilterKeepsDelegating() throws Exception {
+    verifyIsDelegatingForLoggerFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForContextWideFilter(Level.INFO, expectDelegating(true));
+  }
+  
+  @Test
+  public void contextWideFilterIsDelegating() throws Exception {
+    verifyIsDelegatingForContextWideFilter(Level.TRACE, expectDelegating(true));
+    verifyIsDelegatingForContextWideFilter(Level.DEBUG, expectDelegating(true));
+    verifyIsDelegatingForContextWideFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForContextWideFilter(Level.WARN, expectDelegating(true));
+    verifyIsDelegatingForContextWideFilter(Level.ERROR, expectDelegating(true));
+    verifyIsDelegatingForContextWideFilter(Level.FATAL, expectDelegating(true));
+  }
+  
+  @Test
+  public void loggerFilterIsDelegating() throws Exception {
+    verifyIsDelegatingForLoggerFilter(Level.TRACE, expectDelegating(true));
+    verifyIsDelegatingForLoggerFilter(Level.DEBUG, expectDelegating(true));
+    verifyIsDelegatingForLoggerFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForLoggerFilter(Level.WARN, expectDelegating(true));
+    verifyIsDelegatingForLoggerFilter(Level.ERROR, expectDelegating(true));
+    verifyIsDelegatingForLoggerFilter(Level.FATAL, expectDelegating(true));
+  }
+  
+  @Test
+  public void appenderFilterIsDelegating() throws Exception {
+    verifyIsDelegatingForAppenderFilter(Level.TRACE, expectDelegating(true));
+    verifyIsDelegatingForAppenderFilter(Level.DEBUG, expectDelegating(true));
+    verifyIsDelegatingForAppenderFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForAppenderFilter(Level.WARN, expectDelegating(true));
+    verifyIsDelegatingForAppenderFilter(Level.ERROR, expectDelegating(true));
+    verifyIsDelegatingForAppenderFilter(Level.FATAL, expectDelegating(true));
+  }
+  
+  @Test
+  public void appenderRefFilterIsDelegating() throws Exception {
+    verifyIsDelegatingForAppenderRefFilter(Level.TRACE, expectDelegating(true));
+    verifyIsDelegatingForAppenderRefFilter(Level.DEBUG, expectDelegating(true));
+    verifyIsDelegatingForAppenderRefFilter(Level.INFO, expectDelegating(true));
+    verifyIsDelegatingForAppenderRefFilter(Level.WARN, expectDelegating(true));
+    verifyIsDelegatingForAppenderRefFilter(Level.ERROR, expectDelegating(true));
+    verifyIsDelegatingForAppenderRefFilter(Level.FATAL, expectDelegating(true));
+  }
+  
+  /**
+   * Verifies FastLogger isDelegating if Level is DEBUG or TRACE.
+   * 
+   * @param level the log Level
+   * @param expectIsDelegating true if expecting FastLogger.isDelegating to be true
+   */
+  private void verifyIsDelegatingForDebugOrLower(final Level level, final boolean expectIsDelegating) throws Exception {
+    writeSimpleConfigFile(this.configFile, level);
+    this.appenderContext.reconfigure();
+    
+    assertThat(this.logger.getLevel(), is(level));
+    
+    assertThat(this.logger.isTraceEnabled(), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(), is(level.isLessSpecificThan(Level.FATAL)));
+    
+    assertThat(this.logger.isTraceEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.FATAL)));
+
+    final boolean delegating = ((FastLogger)this.logger).isDelegating();
+    assertThat(delegating, is(expectIsDelegating));
+    assertThat(delegating, is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(delegating, is(expectIsDelegating));
+  }
+  
+  /**
+   * Verifies FastLogger isDelegating if there is a Logger Filter.
+   * 
+   * @param level the log Level
+   * @param expectIsDelegating true if expecting FastLogger.isDelegating to be true
+   */
+  private void verifyIsDelegatingForLoggerFilter(final Level level, final boolean expectIsDelegating) throws Exception {
+    assertThat(expectIsDelegating, is(true)); // always true for Logger Filter
+
+    writeLoggerFilterConfigFile(this.configFile, level);
+    this.appenderContext.reconfigure();
+    
+    assertThat(this.logger.getLevel(), is(level));
+    
+    assertThat(this.logger.isTraceEnabled(), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(), is(level.isLessSpecificThan(Level.FATAL)));
+    
+    assertThat(this.logger.isTraceEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.FATAL)));
+    
+    assertThat(this.logger.isTraceEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.FATAL)));
+    
+    assertThat(((FastLogger)this.logger).isDelegating(), is(expectIsDelegating));
+  }
+
+  /**
+   * Verifies FastLogger isDelegating if there is a Context-wide Filter.
+   * 
+   * @param level the log Level
+   * @param expectIsDelegating true if expecting FastLogger.isDelegating to be true
+   */
+  private void verifyIsDelegatingForContextWideFilter(final Level level, final boolean expectIsDelegating) throws Exception {
+    assertThat(expectIsDelegating, is(true)); // always true for Context-wide Filter
+    
+    writeContextWideFilterConfigFile(this.configFile, level);
+    this.appenderContext.reconfigure();
+    
+    assertThat(this.logger.getLevel(), is(level));
+    
+    // note: unlike other filters, Context-wide filters are processed BEFORE isEnabled checks
+    
+    assertThat(this.logger.isTraceEnabled(), is(false));
+    assertThat(this.logger.isDebugEnabled(), is(false));
+    assertThat(this.logger.isInfoEnabled(), is(false));
+    assertThat(this.logger.isWarnEnabled(), is(false));
+    assertThat(this.logger.isErrorEnabled(), is(false));
+    assertThat(this.logger.isFatalEnabled(), is(false));
+    
+    assertThat(this.logger.isTraceEnabled(this.enabledMarker), is(true));
+    assertThat(this.logger.isDebugEnabled(this.enabledMarker), is(true));
+    assertThat(this.logger.isInfoEnabled(this.enabledMarker), is(true));
+    assertThat(this.logger.isWarnEnabled(this.enabledMarker), is(true));
+    assertThat(this.logger.isErrorEnabled(this.enabledMarker), is(true));
+    assertThat(this.logger.isFatalEnabled(this.enabledMarker), is(true));
+    
+    assertThat(this.logger.isTraceEnabled(this.unusedMarker), is(false));
+    assertThat(this.logger.isDebugEnabled(this.unusedMarker), is(false));
+    assertThat(this.logger.isInfoEnabled(this.unusedMarker), is(false));
+    assertThat(this.logger.isWarnEnabled(this.unusedMarker), is(false));
+    assertThat(this.logger.isErrorEnabled(this.unusedMarker), is(false));
+    assertThat(this.logger.isFatalEnabled(this.unusedMarker), is(false));
+
+    assertThat(((FastLogger)this.logger).isDelegating(), is(expectIsDelegating));
+  }
+  
+  /**
+   * Verifies FastLogger isDelegating if there is a Appender Filter.
+   * 
+   * @param level the log Level
+   * @param expectIsDelegating true if expecting FastLogger.isDelegating to be true
+   */
+  private void verifyIsDelegatingForAppenderFilter(final Level level, final boolean expectIsDelegating) throws Exception {
+    assertThat(expectIsDelegating, is(true)); // always true for Appender Filter
+
+    writeAppenderFilterConfigFile(this.configFile, level);
+    this.appenderContext.reconfigure();
+    
+    assertThat(this.logger.getLevel(), is(level));
+    
+    assertThat(this.logger.isTraceEnabled(), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(), is(level.isLessSpecificThan(Level.FATAL)));
+    
+    assertThat(this.logger.isTraceEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.FATAL)));
+    
+    assertThat(this.logger.isTraceEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.FATAL)));
+    
+    assertThat(((FastLogger)this.logger).isDelegating(), is(expectIsDelegating));
+  }
+
+  /**
+   * Verifies FastLogger isDelegating if there is a AppenderRef Filter.
+   * 
+   * @param level the log Level
+   * @param expectIsDelegating true if expecting FastLogger.isDelegating to be true
+   */
+  private void verifyIsDelegatingForAppenderRefFilter(final Level level, final boolean expectIsDelegating) throws Exception {
+    assertThat(expectIsDelegating, is(true)); // always true for AppenderRef Filter
+
+    writeAppenderRefFilterConfigFile(this.configFile, level);
+    this.appenderContext.reconfigure();
+    
+    assertThat(this.logger.getLevel(), is(level));
+    
+    assertThat(this.logger.isTraceEnabled(), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(), is(level.isLessSpecificThan(Level.FATAL)));
+    
+    assertThat(this.logger.isTraceEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(this.enabledMarker), is(level.isLessSpecificThan(Level.FATAL)));
+    
+    assertThat(this.logger.isTraceEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.TRACE)));
+    assertThat(this.logger.isDebugEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.DEBUG)));
+    assertThat(this.logger.isInfoEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.INFO)));
+    assertThat(this.logger.isWarnEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.WARN)));
+    assertThat(this.logger.isErrorEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.ERROR)));
+    assertThat(this.logger.isFatalEnabled(this.unusedMarker), is(level.isLessSpecificThan(Level.FATAL)));
+    
+    assertThat(((FastLogger)this.logger).isDelegating(), is(expectIsDelegating));
+  }
+
+  private boolean expectDelegating(final boolean value) {
+    return value;
+  }
+  
+  private static String writeSimpleConfigFile(final File configFile, final Level level) throws IOException {
+    final String xml = 
+        "<?xml version=\"1.0\" encoding=\"UTF-8\"?>" +
+            "<Configuration monitorInterval=\"5\">" +
+            "<Appenders><Console name=\"STDOUT\" target=\"SYSTEM_OUT\"/></Appenders>" +
+            "<Loggers>" +
+              "<Logger name=\"" + TEST_LOGGER_NAME + "\" level=\"" + level.name() + "\" additivity=\"true\">" +
+                "<AppenderRef ref=\"STDOUT\"/>" +
+              "</Logger>" +
+              "<Root level=\"FATAL\"/>" +
+            "</Loggers>" +
+           "</Configuration>";
+    final BufferedWriter writer = new BufferedWriter(new FileWriter(configFile));
+    writer.write(xml);
+    writer.close();
+    return xml;
+  }
+  
+  private static String writeLoggerFilterConfigFile(final File configFile, final Level level) throws IOException {
+    final String xml = 
+        "<?xml version=\"1.0\" encoding=\"UTF-8\"?>" +
+            "<Configuration monitorInterval=\"5\">" +
+            "<Appenders><Console name=\"STDOUT\" target=\"SYSTEM_OUT\"/></Appenders>" +
+            "<Loggers>" +
+              "<Logger name=\"" + TEST_LOGGER_NAME + "\" level=\"" + level.name() + "\" additivity=\"true\">" +
+                "<filters>" +
+                  "<MarkerFilter marker=\"" + ENABLED_MARKER_NAME + "\" onMatch=\"ACCEPT\" onMismatch=\"DENY\"/>" +
+                "</filters>" +
+                "<AppenderRef ref=\"STDOUT\"/>" +
+              "</Logger>" +
+              "<Root level=\"FATAL\"/>" +
+            "</Loggers>" +
+           "</Configuration>";
+    final BufferedWriter writer = new BufferedWriter(new FileWriter(configFile));
+    writer.write(xml);
+    writer.close();
+    return xml;
+  }
+  
+  private static String writeContextWideFilterConfigFile(final File configFile, final Level level) throws IOException {
+    final String xml = 
+        "<?xml version=\"1.0\" encoding=\"UTF-8\"?>" +
+          "<Configuration monitorInterval=\"5\">" +
+            "<Appenders><Console name=\"STDOUT\" target=\"SYSTEM_OUT\"/></Appenders>" +
+            "<Loggers>" +
+              "<Logger name=\"" + TEST_LOGGER_NAME + "\" level=\"" + level.name() + "\" additivity=\"true\">" +
+              "</Logger>" +
+              "<Root level=\"FATAL\">" +
+                "<AppenderRef ref=\"STDOUT\"/>" +
+              "</Root>" +
+            "</Loggers>" +
+            "<filters>" +
+              "<MarkerFilter marker=\"" + ENABLED_MARKER_NAME + "\" onMatch=\"ACCEPT\" onMismatch=\"DENY\"/>" +
+            "</filters>" +
+           "</Configuration>";
+    final BufferedWriter writer = new BufferedWriter(new FileWriter(configFile));
+    writer.write(xml);
+    writer.close();
+    return xml;
+  }
+
+  private static String writeAppenderFilterConfigFile(final File configFile, final Level level) throws IOException {
+    final String xml = 
+        "<?xml version=\"1.0\" encoding=\"UTF-8\"?>" +
+          "<Configuration monitorInterval=\"5\">" +
+            "<Appenders>" +
+              "<Console name=\"STDOUT\" target=\"SYSTEM_OUT\">" +
+              "<filters>" +
+                "<MarkerFilter marker=\"" + ENABLED_MARKER_NAME + "\" onMatch=\"ACCEPT\" onMismatch=\"DENY\"/>" +
+              "</filters>" +
+              "</Console>" +
+            "</Appenders>" +
+            "<Loggers>" +
+              "<Logger name=\"" + TEST_LOGGER_NAME + "\" level=\"" + level.name() + "\" additivity=\"true\">" +
+              "</Logger>" +
+              "<Root level=\"FATAL\">" +
+                "<AppenderRef ref=\"STDOUT\"/>" +
+              "</Root>" +
+            "</Loggers>" +
+           "</Configuration>";
+    final BufferedWriter writer = new BufferedWriter(new FileWriter(configFile));
+    writer.write(xml);
+    writer.close();
+    return xml;
+  }
+
+  private static String writeAppenderRefFilterConfigFile(final File configFile, final Level level) throws IOException {
+    final String xml = 
+        "<?xml version=\"1.0\" encoding=\"UTF-8\"?>" +
+          "<Configuration monitorInterval=\"5\">" +
+            "<Appenders>" +
+              "<Console name=\"STDOUT\" target=\"SYSTEM_OUT\">" +
+              "</Console>" +
+            "</Appenders>" +
+            "<Loggers>" +
+              "<Logger name=\"" + TEST_LOGGER_NAME + "\" level=\"" + level.name() + "\" additivity=\"true\">" +
+              "</Logger>" +
+              "<Root level=\"FATAL\">" +
+                "<AppenderRef ref=\"STDOUT\">" +
+                  "<filters>" +
+                    "<MarkerFilter marker=\"" + ENABLED_MARKER_NAME + "\" onMatch=\"ACCEPT\" onMismatch=\"DENY\"/>" +
+                  "</filters>" +
+                "</AppenderRef>" +
+              "</Root>" +
+            "</Loggers>" +
+           "</Configuration>";
+    final BufferedWriter writer = new BufferedWriter(new FileWriter(configFile));
+    writer.write(xml);
+    writer.close();
+    return xml;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerJUnitTest.java
old mode 100644
new mode 100755
index 2aab5df..96d8b93
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerJUnitTest.java
@@ -1,202 +1,162 @@
 package com.gemstone.gemfire.internal.logging.log4j;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.lang.reflect.Field;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.CoreMatchers.sameInstance;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
 
 import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.Logger;
-import org.apache.logging.log4j.core.LoggerContext;
-import org.apache.logging.log4j.core.config.ConfigurationFactory;
-import org.junit.After;
+import org.apache.logging.log4j.Marker;
+import org.apache.logging.log4j.message.MessageFactory;
+import org.apache.logging.log4j.message.ParameterizedMessageFactory;
+import org.apache.logging.log4j.spi.ExtendedLogger;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 /**
- * Tests FastLogger isDebugEnabled and isTraceEnabled.
- * 
- * @author Kirk Lund
- * @author David Hoots
+ * Unit tests the FastLogger class which wraps and delegates to an actual 
+ * Logger with optimizations for isDebugEnabled and isTraceEnabled.
  */
 @Category(UnitTest.class)
 public class FastLoggerJUnitTest {
 
-  private static final String TEST_LOGGER_NAME = "com.gemstone.gemfire.cache.internal";
-  
-  private File configFile;
+  private MessageFactory messageFactory;
+  private ExtendedLogger mockedLogger;
+  private Marker mockedMarker;
   
   @Before
   public void setUp() {
-    System.clearProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
-  }
-  
-  @After
-  public void tearDown() {
-    System.clearProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
-    LogService.reconfigure();
-    if (this.configFile != null && this.configFile.exists()) {
-      this.configFile.delete();
-    }
+    this.messageFactory = new ParameterizedMessageFactory();
+    this.mockedLogger = mock(ExtendedLogger.class);
+    this.mockedMarker = mock(Marker.class);
+
+    when(this.mockedLogger.getMessageFactory()).thenReturn(this.messageFactory);
+    when(this.mockedMarker.getName()).thenReturn("MARKER");
   }
   
   /**
-   * Verifies that when the configuration is changed the FastLogger
-   * debugAvailable field is changed.
+   * FastLogger should return isDelegating after setDelegating
    */
   @Test
-  public final void testRespondToConfigChange() throws Exception {
-    final File configFile = new File(System.getProperty("java.io.tmpdir"), "log4j2-test.xml");
-    
-    // Load a base config and do some sanity checks
-    writeBaseConfigFile(configFile, "WARN");
-    System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, configFile.toURI().toURL().toString());
+  public void returnIsDelegatingAfterSetDelegating() {
+    FastLogger.setDelegating(true);
 
-    LogService.reconfigure();
+    FastLogger fastLogger = new FastLogger(this.mockedLogger);
     
-    LogService.getLogger().getName(); // This causes the config file to be loaded
-    final Logger testLogger = LogService.getLogger(TEST_LOGGER_NAME);
-    
-    final LoggerContext appenderContext = ((org.apache.logging.log4j.core.Logger) LogService.getRootLogger()).getContext();
-    assertEquals(Level.FATAL, LogService.getLogger(LogService.BASE_LOGGER_NAME).getLevel());
-    assertEquals(Level.WARN, LogService.getLogger(TEST_LOGGER_NAME).getLevel());
-
-    // Get a reference to the debugAvailable field in FastLogger
-    Field debugAvailableField = FastLogger.class.getDeclaredField("debugAvailable");
-    debugAvailableField.setAccessible(true);
-    boolean debugAvailable = (Boolean) debugAvailableField.get(FastLogger.class);
-    assertFalse(debugAvailable);
-
-    // Modify the config and verify that the debugAvailable field has changed
-    writeBaseConfigFile(configFile, "DEBUG");
-    appenderContext.reconfigure();
-    assertEquals(Level.DEBUG, LogService.getLogger(TEST_LOGGER_NAME).getLevel());
-    debugAvailable = (Boolean) debugAvailableField.get(FastLogger.class);
-    assertTrue(testLogger.isDebugEnabled());
-    assertFalse(testLogger.isTraceEnabled());
-    assertTrue(debugAvailable);
+    assertThat(fastLogger.isDelegating(), is(true));
 
-    // Modify the config and verify that the debugAvailable field has changed
-    writeBaseConfigFile(configFile, "ERROR");
-    appenderContext.reconfigure();
-    assertEquals(Level.ERROR, LogService.getLogger(TEST_LOGGER_NAME).getLevel());
-    assertFalse(testLogger.isDebugEnabled());
-    assertFalse((Boolean) debugAvailableField.get(FastLogger.class));
-
-    // Modify the config and verify that the debugAvailable field has changed
-    writeBaseConfigFile(configFile, "TRACE");
-    appenderContext.reconfigure();
-    assertEquals(Level.TRACE, LogService.getLogger(TEST_LOGGER_NAME).getLevel());
-    assertTrue(testLogger.isDebugEnabled());
-    assertTrue(testLogger.isTraceEnabled());
-    assertTrue((Boolean) debugAvailableField.get(FastLogger.class));
+    FastLogger.setDelegating(false);
     
-    // Modify the config and verify that the debugAvailable field has changed
-    writeBaseConfigFile(configFile, "INFO");
-    appenderContext.reconfigure();
-    assertEquals(Level.INFO, LogService.getLogger(TEST_LOGGER_NAME).getLevel());
-    assertFalse(testLogger.isDebugEnabled());
-    assertFalse((Boolean) debugAvailableField.get(FastLogger.class));
+    assertThat(fastLogger.isDelegating(), is(false));
+  }
+  
+  /**
+   * FastLogger should delegate getLevel
+   */
+  @Test
+  public void delegateGetLevel() {
+    FastLogger.setDelegating(true);
+    when(this.mockedLogger.getLevel()).thenReturn(Level.DEBUG);
     
-    // A reset before the next filter test
-    writeBaseConfigFile(configFile, "FATAL");
-    appenderContext.reconfigure();
-    assertFalse((Boolean) debugAvailableField.get(FastLogger.class));
+    FastLogger fastLogger = new FastLogger(this.mockedLogger);
     
-    // Modify the config and verify that the debugAvailable field has changed
-    writeLoggerFilterConfigFile(configFile);
-    appenderContext.reconfigure();
-    assertEquals(Level.ERROR, LogService.getLogger(TEST_LOGGER_NAME).getLevel());
-    assertFalse(testLogger.isDebugEnabled());
-    assertTrue((Boolean) debugAvailableField.get(FastLogger.class));
+    assertThat(fastLogger.getLevel(), is(Level.DEBUG));
+    verify(this.mockedLogger, times(1)).getLevel();
+  }
+  
+  /**
+   * FastLogger should delegate isDebugEnabled when isDelegating
+   */
+  @Test
+  public void delegateIsDebugEnabledWhenIsDelegating() {
+    FastLogger.setDelegating(true);
+    when(this.mockedLogger.getLevel()).thenReturn(Level.DEBUG);
+    when(this.mockedLogger.isEnabled(eq(Level.DEBUG), isNull(Marker.class), isNull(String.class))).thenReturn(true);
+    when(this.mockedLogger.isEnabled(eq(Level.DEBUG), eq(this.mockedMarker), isNull(Object.class), isNull(Throwable.class))).thenReturn(true);
+
+    FastLogger fastLogger = new FastLogger(this.mockedLogger);
     
-    // A reset before the next filter test
-    writeBaseConfigFile(configFile, "FATAL");
-    appenderContext.reconfigure();
-    assertFalse((Boolean) debugAvailableField.get(FastLogger.class));
+    assertThat(fastLogger.isDebugEnabled(), is(true));
+    assertThat(fastLogger.isDebugEnabled(this.mockedMarker), is(true));
+    verify(this.mockedLogger, times(1)).isEnabled(eq(Level.DEBUG), any(Marker.class), isNull(String.class));
+    verify(this.mockedLogger, times(1)).isEnabled(eq(Level.DEBUG), eq(this.mockedMarker), isNull(Object.class), isNull(Throwable.class));
+  }
+  
+  /**
+   * FastLogger should delegate isTraceEnabled when isDelegating
+   */
+  @Test
+  public void delegateIsTraceEnabledWhenIsDelegating() {
+    FastLogger.setDelegating(true);
+    when(this.mockedLogger.getLevel()).thenReturn(Level.TRACE);
+    when(this.mockedLogger.isEnabled(eq(Level.TRACE), isNull(Marker.class), isNull(Object.class), isNull(Throwable.class))).thenReturn(true);
+    when(this.mockedLogger.isEnabled(eq(Level.TRACE), eq(this.mockedMarker), isNull(Object.class), isNull(Throwable.class))).thenReturn(true);
+
+    FastLogger fastLogger = new FastLogger(this.mockedLogger);
     
-    // Modify the config and verify that the debugAvailable field has changed
-    writeContextFilterConfigFile(configFile);
-    appenderContext.reconfigure();
-    assertEquals(Level.ERROR, LogService.getLogger(TEST_LOGGER_NAME).getLevel());
-    assertFalse(testLogger.isDebugEnabled());
-    assertTrue((Boolean) debugAvailableField.get(FastLogger.class));
+    assertThat(fastLogger.isTraceEnabled(), is(true));
+    assertThat(fastLogger.isTraceEnabled(this.mockedMarker), is(true));
+    verify(this.mockedLogger, times(1)).isEnabled(eq(Level.TRACE), isNull(Marker.class), isNull(Object.class), isNull(Throwable.class));
+    verify(this.mockedLogger, times(1)).isEnabled(eq(Level.TRACE), eq(this.mockedMarker), isNull(Object.class), isNull(Throwable.class));
   }
   
   /**
-   * Verifies that default the configuration sets the FastLogger debugAvailable to false.
+   * FastLogger should not delegate isDebugEnabled when not isDelegating
    */
   @Test
-  public final void testDefaultConfig() throws Exception {
-    LogService.reconfigure();
-    assertTrue(LogService.isUsingGemFireDefaultConfig());
+  public void notDelegateIsDebugEnabledWhenNotIsDelegating() {
+    FastLogger.setDelegating(false);
+    when(this.mockedLogger.getLevel()).thenReturn(Level.INFO);
+
+    FastLogger fastLogger = new FastLogger(this.mockedLogger);
     
-    // Get a reference to the debugAvailable field in FastLogger
-    Field debugAvailableField = FastLogger.class.getDeclaredField("debugAvailable");
-    debugAvailableField.setAccessible(true);
-    boolean debugAvailable = (Boolean) debugAvailableField.get(FastLogger.class);
-    assertFalse("FastLogger debugAvailable should be false for default config", debugAvailable);
+    assertThat(fastLogger.getLevel(), is(Level.INFO));
+    assertThat(fastLogger.isDebugEnabled(), is(false));
+    assertThat(fastLogger.isDebugEnabled(this.mockedMarker), is(false));
+    verify(this.mockedLogger, times(0)).isEnabled(eq(Level.DEBUG), isNull(Marker.class), isNull(String.class));
+    verify(this.mockedLogger, times(0)).isEnabled(eq(Level.DEBUG), eq(this.mockedMarker), isNull(Object.class), isNull(Throwable.class));
   }
   
-  private static void writeBaseConfigFile(final File configFile, final String level) throws IOException {
-    final BufferedWriter writer = new BufferedWriter(new FileWriter(configFile));
-    writer.write(
-        "<?xml version=\"1.0\" encoding=\"UTF-8\"?>" +
-          "<Configuration monitorInterval=\"5\">" +
-          "<Appenders><Console name=\"STDOUT\" target=\"SYSTEM_OUT\"/></Appenders>" +
-          "<Loggers>" +
-            "<Logger name=\"" + TEST_LOGGER_NAME + "\" level=\"" + level + "\" additivity=\"false\">" +
-              "<AppenderRef ref=\"STDOUT\"/>" +
-            "</Logger>" +
-            "<Root level=\"FATAL\"/>" +
-          "</Loggers>" +
-         "</Configuration>"
-         );
-    writer.close();
+  /**
+   * FastLogger should not delegate isTraceEnabled when not isDelegating
+   */
+  @Test
+  public void notDelegateIsTraceEnabledWhenNotIsDelegating() {
+    FastLogger.setDelegating(false);
+    when(mockedLogger.getLevel()).thenReturn(Level.INFO);
+
+    FastLogger fastLogger = new FastLogger(this.mockedLogger);
+    
+    assertThat(fastLogger.getLevel(), is(Level.INFO));
+    assertThat(fastLogger.isTraceEnabled(), is(false));
+    assertThat(fastLogger.isTraceEnabled(this.mockedMarker), is(false));
+    verify(this.mockedLogger, times(0)).isEnabled(eq(Level.TRACE), isNull(Marker.class), isNull(String.class));
+    verify(this.mockedLogger, times(0)).isEnabled(eq(Level.TRACE), eq(this.mockedMarker), isNull(Object.class), isNull(Throwable.class));
   }
   
-  private static void writeLoggerFilterConfigFile(final File configFile) throws IOException {
-    final BufferedWriter writer = new BufferedWriter(new FileWriter(configFile));
-    writer.write(
-        "<?xml version=\"1.0\" encoding=\"UTF-8\"?>" +
-          "<Configuration monitorInterval=\"5\">" +
-          "<Appenders><Console name=\"STDOUT\" target=\"SYSTEM_OUT\"/></Appenders>" +
-          "<Loggers>" +
-            "<Logger name=\"" + TEST_LOGGER_NAME + "\" level=\"ERROR\" additivity=\"false\">" +
-              "<MarkerFilter marker=\"FLOW\" onMatch=\"ACCEPT\" onMismatch=\"DENY\"/>" +
-              "<AppenderRef ref=\"STDOUT\"/>" +
-            "</Logger>" +
-            "<Root level=\"FATAL\"/>" +
-          "</Loggers>" +
-         "</Configuration>"
-         );
-    writer.close();
+  /**
+   * FastLogger should wrap delegate and return from getExtendedLogger
+   */
+  @Test
+  public void wrapDelegateAndReturnFromGetExtendedLogger() {
+    FastLogger fastLogger = new FastLogger(this.mockedLogger);
+
+    assertThat(fastLogger.getExtendedLogger(), is(sameInstance(this.mockedLogger)));
   }
   
-  private static void writeContextFilterConfigFile(final File configFile) throws IOException {
-    final BufferedWriter writer = new BufferedWriter(new FileWriter(configFile));
-    writer.write(
-        "<?xml version=\"1.0\" encoding=\"UTF-8\"?>" +
-          "<Configuration monitorInterval=\"5\">" +
-          "<Appenders><Console name=\"STDOUT\" target=\"SYSTEM_OUT\"/></Appenders>" +
-          "<Loggers>" +
-            "<Logger name=\"" + TEST_LOGGER_NAME + "\" level=\"ERROR\" additivity=\"false\">" +
-              "<AppenderRef ref=\"STDOUT\"/>" +
-            "</Logger>" +
-            "<Root level=\"FATAL\"/>" +
-          "</Loggers>" +
-          "<MarkerFilter marker=\"FLOW\" onMatch=\"ACCEPT\" onMismatch=\"DENY\"/>" +
-         "</Configuration>"
-         );
-    writer.close();
+  /**
+   * FastLogger should delegate getName
+   */
+  @Test
+  public void delegateGetName() {
+    when(this.mockedLogger.getName()).thenReturn("name");
+
+    FastLogger fastLogger = new FastLogger(this.mockedLogger);
+
+    assertThat(fastLogger.getName(), is("name"));
+    verify(this.mockedLogger, times(1)).getName();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerWithDefaultConfigJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerWithDefaultConfigJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerWithDefaultConfigJUnitTest.java
new file mode 100755
index 0000000..2e08500
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/FastLoggerWithDefaultConfigJUnitTest.java
@@ -0,0 +1,74 @@
+package com.gemstone.gemfire.internal.logging.log4j;
+
+import static org.hamcrest.CoreMatchers.*;
+import static org.junit.Assert.*;
+
+import org.apache.logging.log4j.Logger;
+import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.RestoreSystemProperties;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.internal.logging.LogService;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+/**
+ * Integration tests for FastLogger when using the default log4j2 config for GemFire.
+ * 
+ * @author Kirk Lund
+ * @author David Hoots
+ */
+@Category(IntegrationTest.class)
+public class FastLoggerWithDefaultConfigJUnitTest {
+
+  private static final String TEST_LOGGER_NAME = FastLogger.class.getPackage().getName();
+  
+  @Rule
+  public final RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties();
+  
+  private Logger logger;
+  
+  @Before
+  public void setUp() throws Exception {
+    System.clearProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY);
+    LogService.reconfigure();
+  }
+  
+  /**
+   * System property "log4j.configurationFile" should be "/com/gemstone/gemfire/internal/logging/log4j/log4j2-default.xml"
+   */
+  @Test
+  public void configurationFilePropertyIsDefaultConfig() {
+    assertThat(System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY), containsString(LogService.DEFAULT_CONFIG));
+  }
+  
+  /**
+   * LogService isUsingGemFireDefaultConfig should be true
+   */
+  @Test
+  public void isUsingGemFireDefaultConfig() {
+    assertThat(LogService.isUsingGemFireDefaultConfig(), is(true));
+  }
+  
+  /**
+   * LogService getLogger should return loggers wrapped in FastLogger
+   */
+  @Test
+  public void logServiceReturnsFastLoggers() {
+    this.logger = LogService.getLogger(TEST_LOGGER_NAME);
+    
+    assertThat(this.logger, is(instanceOf(FastLogger.class)));
+  }
+  
+  /**
+   * FastLogger isDelegating should be false
+   */
+  @Test
+  public void isDelegatingShouldBeFalse() {
+    this.logger = LogService.getLogger(TEST_LOGGER_NAME);
+    
+    assertThat(((FastLogger)this.logger).isDelegating(), is(false));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4jIntegrationTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4jIntegrationTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4jIntegrationTestSuite.java
new file mode 100755
index 0000000..7cab077
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4jIntegrationTestSuite.java
@@ -0,0 +1,12 @@
+package com.gemstone.gemfire.internal.logging.log4j;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+  FastLoggerWithDefaultConfigJUnitTest.class,
+  FastLoggerIntegrationJUnitTest.class,
+})
+public class Log4jIntegrationTestSuite {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c4cd8f7/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4jUnitTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4jUnitTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4jUnitTestSuite.java
new file mode 100755
index 0000000..1441b0c
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/logging/log4j/Log4jUnitTestSuite.java
@@ -0,0 +1,16 @@
+package com.gemstone.gemfire.internal.logging.log4j;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+  AlertAppenderJUnitTest.class,
+  ConfigLocatorJUnitTest.class,
+  FastLoggerJUnitTest.class,
+  FastLoggerWithDefaultConfigJUnitTest.class,
+  LocalizedMessageJUnitTest.class,
+  LogWriterAppenderJUnitTest.class,
+})
+public class Log4jUnitTestSuite {
+}


[44/50] [abbrv] incubator-geode git commit: GEODE-195: Remove debug logging from CloseCacheAuthorization

Posted by bs...@apache.org.
GEODE-195: Remove debug logging from CloseCacheAuthorization


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

Branch: refs/heads/feature/GEODE-77
Commit: 5054a8a003d71495c77fa5a3f8c79b23c0e83530
Parents: 9de95d6
Author: Jason Huynh <jh...@pivotal.io>
Authored: Thu Aug 13 11:10:08 2015 -0700
Committer: Jason Huynh <jh...@pivotal.io>
Committed: Thu Aug 13 11:10:08 2015 -0700

----------------------------------------------------------------------
 .../gemfire/cache/query/dunit/CloseCacheAuthorization.java         | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/5054a8a0/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/CloseCacheAuthorization.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/CloseCacheAuthorization.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/CloseCacheAuthorization.java
index d269ddc..b259c60 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/CloseCacheAuthorization.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/dunit/CloseCacheAuthorization.java
@@ -36,8 +36,6 @@ public class CloseCacheAuthorization extends TestCase implements AccessControl {
 
   @Override
   public boolean authorizeOperation(String regionName, OperationContext context) {
-    this.logger.info(" JASON Authorizing request " + context.getOperationCode() + " class:" + context.getClass().getName());
-    new Exception("JASON authz stack").printStackTrace();
     if (context instanceof ExecuteCQOperationContext) {
       cache.close();
       //return false;


[12/50] [abbrv] incubator-geode git commit: GEODE-169: testEventsExpiryBug should expect "Unexpected IOException".

Posted by bs...@apache.org.
GEODE-169: testEventsExpiryBug should expect "Unexpected IOException".


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

Branch: refs/heads/feature/GEODE-77
Commit: 377beeb3a17637fb287c6b0bddef66af3050d22c
Parents: 07a5407
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Jul 29 15:31:08 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Jul 29 15:31:08 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/377beeb3/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java
index 00336e9..b22ac27 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ha/Bug36853EventsExpiryDUnitTest.java
@@ -234,6 +234,7 @@ public class Bug36853EventsExpiryDUnitTest extends CacheTestCase
    */
   public void testEventsExpiryBug() throws Exception
   {
+    addExpectedException("Unexpected IOException");
     addExpectedException("Connection reset");
     server.invoke(Bug36853EventsExpiryDUnitTest.class, "generateEvents");
     client.invoke(Bug36853EventsExpiryDUnitTest.class,


[18/50] [abbrv] incubator-geode git commit: [GEODE-176] fixes the issue

Posted by bs...@apache.org.
[GEODE-176] fixes the issue


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

Branch: refs/heads/feature/GEODE-77
Commit: 7ae27529ac36a3ea2c943346d56fe1c78019c3b0
Parents: 3393f5a
Author: eshu <es...@pivotal.io>
Authored: Thu Jul 30 11:42:28 2015 -0700
Committer: eshu <es...@pivotal.io>
Committed: Mon Aug 3 14:36:59 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java        | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ae27529/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
index edb24d1..29319e0 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/offheap/SimpleMemoryAllocatorImpl.java
@@ -348,7 +348,7 @@ public final class SimpleMemoryAllocatorImpl implements MemoryAllocator, MemoryI
           Set<BucketRegion> brs = prs.getAllLocalBucketRegions();
           if (brs != null) {
             for (BucketRegion br : brs) {
-              if (br != null) {
+              if (br != null && !br.isDestroyed()) {
                 this.basicGetRegionLiveChunks(br, result);
               }
 


[15/50] [abbrv] incubator-geode git commit: GEODE-139: Setting local-max-memory in RebalanceOperationDUnitTest

Posted by bs...@apache.org.
GEODE-139: Setting local-max-memory in RebalanceOperationDUnitTest

The "coordinator" process in dunit is launched by the gradle framework.
The rest of the processes are launched by the dunit framework. Those
processes may have different heap sizes. If local-max-memory is not set,
we read the heap size to determine how to rebalance data.

In this case we didn't move any primaries because one member had much
more heap than the remaining members.


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

Branch: refs/heads/feature/GEODE-77
Commit: 856fa0c125d847f44bbc6965cb77810346110594
Parents: b460df4
Author: Dan Smith <ds...@pivotal.io>
Authored: Wed Jul 29 17:33:06 2015 -0700
Committer: Dan Smith <ds...@pivotal.io>
Committed: Thu Jul 30 10:38:34 2015 -0700

----------------------------------------------------------------------
 .../internal/cache/control/RebalanceOperationDUnitTest.java        | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/856fa0c1/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
index 002aa36..819bf9e 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/control/RebalanceOperationDUnitTest.java
@@ -2401,6 +2401,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
         paf.setRedundantCopies(1);
         paf.setRecoveryDelay(-1);
         paf.setStartupRecoveryDelay(-1);
+        paf.setLocalMaxMemory(100);
         PartitionAttributes prAttr = paf.create();
         attr.setPartitionAttributes(prAttr);
         cache.createRegion("region1", attr.create());
@@ -2424,6 +2425,7 @@ public class RebalanceOperationDUnitTest extends CacheTestCase {
     paf.setRedundantCopies(1);
     paf.setRecoveryDelay(-1);
     paf.setStartupRecoveryDelay(-1);
+    paf.setLocalMaxMemory(100);
     PartitionAttributes prAttr = paf.create();
     attr.setPartitionAttributes(prAttr);
     final Region region = cache.createRegion("region1", attr.create());


[41/50] [abbrv] incubator-geode git commit: GEODE-116: CopyOnReadIndexDUnitTest.testPRQueryOnLocalNode fails assertion

Posted by bs...@apache.org.
GEODE-116: CopyOnReadIndexDUnitTest.testPRQueryOnLocalNode fails assertion

Use a wait criterion incase the indexes are async for some reason
Check index size instead of local pr size when comparing instance counts


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

Branch: refs/heads/feature/GEODE-77
Commit: 18bbd9b8018d2332c073651dbaa99c07e2f6dba9
Parents: 96fd856
Author: Jason Huynh <jh...@pivotal.io>
Authored: Tue Aug 11 14:01:15 2015 -0700
Committer: Jason Huynh <jh...@pivotal.io>
Committed: Tue Aug 11 14:01:15 2015 -0700

----------------------------------------------------------------------
 .../index/CopyOnReadIndexDUnitTest.java         | 68 ++++++++++++--------
 1 file changed, 41 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/18bbd9b8/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexDUnitTest.java
index 11cce7e..61bb55d 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexDUnitTest.java
@@ -24,6 +24,7 @@ import com.gemstone.gemfire.cache.CommitConflictException;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.query.Index;
 import com.gemstone.gemfire.cache.query.Query;
 import com.gemstone.gemfire.cache.query.QueryService;
 import com.gemstone.gemfire.cache.query.QueryTestUtils;
@@ -31,7 +32,6 @@ import com.gemstone.gemfire.cache.query.SelectResults;
 import com.gemstone.gemfire.cache.query.Struct;
 import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.cache.query.data.Position;
-import com.gemstone.gemfire.cache.query.internal.DefaultQueryService;
 import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.cache30.CacheSerializableRunnable;
 import com.gemstone.gemfire.cache30.CacheTestCase;
@@ -39,8 +39,8 @@ import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.AvailablePortHelper;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
-import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerTestUtil;
 
+import dunit.DistributedTestCase;
 import dunit.Host;
 import dunit.SerializableCallable;
 import dunit.SerializableRunnable;
@@ -157,13 +157,14 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
           //operations we have done on this vm consist of:
           //numPortfoliosPerVM instances of Portfolio created for put operation
           //Due to index, we have deserialized all of the entries this vm currently host
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(),((PartitionedRegion)region).getLocalSize() + numPortfoliosPerVM , Portfolio.instanceCount.get());
+          Index index = getCache().getQueryService().getIndex(region, "idIndex");
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount((int)index.getStatistics().getNumberOfValues() + numPortfoliosPerVM), 5000, 200, true);
         }
         else {
           //operations we have done on this vm consist of:
           //numPortfoliosPerVM instances of Portfolio created for put operation
           //We do not have an index, so we have not deserialized any values
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), numPortfoliosPerVM , Portfolio.instanceCount.get());
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount(numPortfoliosPerVM), 5000, 200, true);
         }
         return null;
       }
@@ -184,13 +185,14 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
           //operations we have done on this vm consist of:
           //numPortfoliosPerVM instances of Portfolio created for put operation
           //Due to index, we have deserialized all of the entries this vm currently host
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), ((PartitionedRegion)region).getLocalSize() + numPortfoliosPerVM, Portfolio.instanceCount.get());
+          Index index = getCache().getQueryService().getIndex(region, "idIndex");
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount((int)index.getStatistics().getNumberOfValues() + numPortfoliosPerVM), 5000, 200, true);
         }
         else {
           //operations we have done on this vm consist of:
           //numPortfoliosPerVM instances of Portfolio created for put operation
           //We do not have an index, so we have not deserialized any values
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), numPortfoliosPerVM, Portfolio.instanceCount.get());
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount(numPortfoliosPerVM), 5000, 200, true);
         }
         return null;
       }
@@ -219,14 +221,15 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
           //50 instances of Portfolio created for put operation
           //Due to index, we have deserialized all of the entries this vm currently host
           //Since we have deserialized and cached these values, we just need to add the number of results we did a copy of due to copy on read
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), ((PartitionedRegion)region).getLocalSize() + numPortfoliosPerVM + numExpectedResults, Portfolio.instanceCount.get());
+          Index index = getCache().getQueryService().getIndex(region, "idIndex");
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount((int)index.getStatistics().getNumberOfValues() + numPortfoliosPerVM + numExpectedResults), 5000, 200, true);
         }
         else {
           //operations we have done on this vm consist of:
           //50 instances of Portfolio created for put operation
           //Due to the query we deserialized the number of entries this vm currently hosts
           //We had to deserialized the results from the other data nodes when we iterated through the results as well as our own
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(),((PartitionedRegion)region).getLocalSize() + numExpectedResults + numPortfoliosPerVM , Portfolio.instanceCount.get());
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount((int)((PartitionedRegion)region).getLocalSize() + numExpectedResults + numPortfoliosPerVM), 5000, 200, true);
         }
         return null;
       }
@@ -238,11 +241,11 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
         if (hasIndex) {
           //After vm0 executed the query, we already had the values deserialized in our cache
           //So it's the same total as before
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), ((PartitionedRegion)region).getLocalSize() + numPortfoliosPerVM, Portfolio.instanceCount.get());
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount((int)((PartitionedRegion)region).getLocalSize() + numPortfoliosPerVM), 5000, 200, true);
         }
         else {
           //After vm0 executed the query, we had to deserialize the values in our vm
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), ((PartitionedRegion)region).getLocalSize() + numPortfoliosPerVM, Portfolio.instanceCount.get());
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount((int)((PartitionedRegion)region).getLocalSize() + numPortfoliosPerVM), 5000, 200, true);
         }
         return null;
       }
@@ -271,7 +274,8 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
           //50 instances of Portfolio created for put operation
           //Due to index, we have deserialized all of the entries this vm currently host
           //This is the second query, because we have deserialized and cached these values, we just need to add the number of results a second time
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), ((PartitionedRegion)region).getLocalSize() + numExpectedResults + numExpectedResults + numPortfoliosPerVM, Portfolio.instanceCount.get());
+          Index index = getCache().getQueryService().getIndex(region, "idIndex");
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount((int)index.getStatistics().getNumberOfValues() + numExpectedResults + numExpectedResults + numPortfoliosPerVM), 5000, 200, true);
         }
         else {
         //operations we have done on this vm consist of:
@@ -279,7 +283,7 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
           //Due to index, we have deserialized all of the entries this vm currently host
           //This is the second query, because we have deserialized and cached these values, we just need to add the number of results a second time
           //Because we have no index, we have to again deserialize all the values that this vm is hosting
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(),((PartitionedRegion)region).getLocalSize() + ((PartitionedRegion)region).getLocalSize() + numExpectedResults + numExpectedResults + numPortfoliosPerVM, Portfolio.instanceCount.get());
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount((int)(((PartitionedRegion)region).getLocalSize() + ((PartitionedRegion)region).getLocalSize() + numExpectedResults + numExpectedResults + numPortfoliosPerVM)), 5000, 200, true);
         }
         return null;
       }
@@ -333,7 +337,7 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
         }
         
         //We should have the same number of portfolio objects that we created for the put
-        assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), numPortfolios , Portfolio.instanceCount.get());
+        DistributedTestCase.waitForCriterion(verifyPortfolioCount(numPortfolios), 5000, 200, true);
         return null;
       }
     });
@@ -342,7 +346,7 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
       public Object call() throws Exception {
         //At this point, we should only have serialized values in this vm
         Region region = getCache().getRegion("/portfolios");
-        assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), 0, Portfolio.instanceCount.get());
+        DistributedTestCase.waitForCriterion(verifyPortfolioCount(0), 0, 200, true);
         return null;
       }
     });
@@ -352,10 +356,10 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
         //There is an index for vm2, so we should have deserialized values at this point,
         Region region = getCache().getRegion("/portfolios");
         if (hasIndex) {
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), numPortfolios, Portfolio.instanceCount.get());
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount(numPortfolios), 0, 200, true);
         }
         else {
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), 0, Portfolio.instanceCount.get());
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount(0), 0, 200, true);
         }
         return null;
       }
@@ -397,7 +401,7 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
         
         //We have created puts from our previous callable
         //Now we have copied the results from the query 
-        assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), numExpectedResults + numPortfolios, Portfolio.instanceCount.get());
+        DistributedTestCase.waitForCriterion(verifyPortfolioCount(numExpectedResults + numPortfolios), 0, 200, true);
         return null;
       }
     });
@@ -425,8 +429,8 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
         }
         //first it must deserialize the portfolios in the replicated region
         //then we do a copy on read of these deserialized objects for the final result set
-        assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), numPortfolios + numExpectedResults, Portfolio.instanceCount.get());
-        
+        DistributedTestCase.waitForCriterion(verifyPortfolioCount(numExpectedResults + numPortfolios), 0, 200, true);
+
         results = (SelectResults) query.execute();
         assertEquals(numExpectedResults, results.size());
         for (Object o: results) {
@@ -443,8 +447,7 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
       
         //we never created index on vm1
         //so in this case, we always have to deserialize the value from the region
-        assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), numPortfolios * 2 + numExpectedResults * 2, Portfolio.instanceCount.get());
-
+        DistributedTestCase.waitForCriterion(verifyPortfolioCount(numPortfolios * 2 + numExpectedResults * 2), 0, 200, true);
         return null;
       }
     });
@@ -471,8 +474,7 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
           }
         }
         //with or without index, the values had to have been deserialized at one point
-        assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), numPortfolios + numExpectedResults, Portfolio.instanceCount.get());
-        
+        DistributedTestCase.waitForCriterion(verifyPortfolioCount(numPortfolios + numExpectedResults), 0, 200, true);        
         results = (SelectResults) query.execute();
         assertEquals(numExpectedResults, results.size());
         for (Object o: results) {
@@ -492,12 +494,12 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
           //we have an index, so the values are already deserialized
           //total is now our original deserialization amount : numPortfolios
           //two query results copied.
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), numPortfolios + numExpectedResults * 2, Portfolio.instanceCount.get());
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount(numPortfolios + numExpectedResults * 2), 0, 200, true);        
         }
         else {
           //we never created index on vm1
           //so in this case, we always have to deserialize the value from the region
-          assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(), numPortfolios * 2 + numExpectedResults * 2, Portfolio.instanceCount.get());
+          DistributedTestCase.waitForCriterion(verifyPortfolioCount(numPortfolios * 2 + numExpectedResults * 2), 0, 200, true);        
         }
         return null;
       }
@@ -524,7 +526,7 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
         }
         
         //with or without index, the values we put in the region were already deserialized values
-        assertEquals("Incorrect number of portfolio instances"+ Portfolio.instanceCount.get(),numExpectedResults * 2 + numPortfolios, Portfolio.instanceCount.get());
+        DistributedTestCase.waitForCriterion(verifyPortfolioCount(numExpectedResults * 2 + numPortfolios), 0, 200, true);        
         return null;
       }
     });
@@ -605,7 +607,19 @@ public class CopyOnReadIndexDUnitTest extends CacheTestCase {
     p.setProperty(DistributionConfig.LOCATORS_NAME, "");
     return p;
   }
-  
+
+  private WaitCriterion verifyPortfolioCount(final int expected) {
+    return new WaitCriterion() {
+      private int expectedCount = expected;
+      public boolean done() {
+        return expectedCount == Portfolio.instanceCount.get();
+      }
+      
+      public String description() {
+        return "verifying number of object instances created";
+      }
+    };
+  }
   
 
 }


[37/50] [abbrv] incubator-geode git commit: GEODE-190: Fix ReflectionBasedAutoSerializer leak

Posted by bs...@apache.org.
GEODE-190: Fix ReflectionBasedAutoSerializer leak

A weak map value references its key keeping the key from ever being garbage.
The only reason the weak map was added was to allow internal code to get
the AutoSerializableManager given a ReflectionBasedAutoSerializer.
But the map was not really needed since the ReflectionBasedAutoSerializer
has a final field named "manager".
The only problem is that ReflectionBasedAutoSerializer is a public
api and AutoSerializableManager is internal.
This fix adds a getManager method on ReflectionBasedAutoSerializer that returns object
and is javadoc'd for "internal use only". The internal code casts the result to
AutoSerializableManager. This allows the weak map that was causing the memory leak
to be removed.


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

Branch: refs/heads/feature/GEODE-77
Commit: 1c7cbb2d1c56f66f76370ebd82a44a90d3c4e7e1
Parents: b1e7466
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Mon Aug 3 14:04:23 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Mon Aug 10 14:01:13 2015 -0700

----------------------------------------------------------------------
 .../com/gemstone/gemfire/internal/cache/CacheConfig.java    | 4 ++--
 .../gemstone/gemfire/internal/cache/GemFireCacheImpl.java   | 2 +-
 .../gemstone/gemfire/pdx/ReflectionBasedAutoSerializer.java | 9 +++++++++
 .../gemfire/pdx/internal/AutoSerializableManager.java       | 8 --------
 .../com/gemstone/gemfire/pdx/internal/TypeRegistry.java     | 4 ++--
 .../com/gemstone/gemfire/pdx/AutoSerializableJUnitTest.java | 2 +-
 6 files changed, 15 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c7cbb2d/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheConfig.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheConfig.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheConfig.java
index 60b765c..7aaa241 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheConfig.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheConfig.java
@@ -175,8 +175,8 @@ public class CacheConfig {
     Object o2 = s2;
     if (s1 instanceof ReflectionBasedAutoSerializer && s2 instanceof ReflectionBasedAutoSerializer) {
       // Fix for bug 44907.
-      o1 = AutoSerializableManager.getInstance((ReflectionBasedAutoSerializer) s1);
-      o2 = AutoSerializableManager.getInstance((ReflectionBasedAutoSerializer) s2);
+      o1 = ((ReflectionBasedAutoSerializer) s1).getManager();
+      o2 = ((ReflectionBasedAutoSerializer) s2).getManager();
     }
     return equals(o1, o2);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c7cbb2d/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index f5be144..79bcbc2 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -5258,7 +5258,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
   private void basicSetPdxSerializer(PdxSerializer v) {
     TypeRegistry.setPdxSerializer(v);
     if (v instanceof ReflectionBasedAutoSerializer) {
-      AutoSerializableManager asm = AutoSerializableManager.getInstance((ReflectionBasedAutoSerializer) v);
+      AutoSerializableManager asm = (AutoSerializableManager) ((ReflectionBasedAutoSerializer) v).getManager();
       if (asm != null) {
         asm.setRegionService(this);
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c7cbb2d/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/ReflectionBasedAutoSerializer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/ReflectionBasedAutoSerializer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/ReflectionBasedAutoSerializer.java
index 7065cea..ef89bcb 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/ReflectionBasedAutoSerializer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/ReflectionBasedAutoSerializer.java
@@ -540,4 +540,13 @@ public class ReflectionBasedAutoSerializer implements PdxSerializer, Declarable
   public final RegionService getRegionService() {
     return this.manager.getRegionService();
   }
+  /**
+   * For internal use only.
+   * @since 8.2
+   */
+  public final Object getManager() {
+    // The result is not AutoSerializableManager because
+    // that class is not part of our public APIs.
+    return this.manager;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c7cbb2d/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/AutoSerializableManager.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/AutoSerializableManager.java b/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/AutoSerializableManager.java
index e7a7ab7..5c560df 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/AutoSerializableManager.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/AutoSerializableManager.java
@@ -133,8 +133,6 @@ public class AutoSerializableManager {
       new CopyOnWriteHashSet<String>();
 
 
-  private static final Map<ReflectionBasedAutoSerializer, AutoSerializableManager> instances = new CopyOnWriteWeakHashMap<ReflectionBasedAutoSerializer, AutoSerializableManager>();
-
   private final ReflectionBasedAutoSerializer owner;
   
   public ReflectionBasedAutoSerializer getOwner() {
@@ -144,18 +142,12 @@ public class AutoSerializableManager {
   public static AutoSerializableManager create(ReflectionBasedAutoSerializer owner, boolean checkPortability, String... patterns) {
     AutoSerializableManager result = new AutoSerializableManager(owner);
     result.reconfigure(checkPortability, patterns);
-    instances.put(owner, result);
     return result;
   }
   private AutoSerializableManager(ReflectionBasedAutoSerializer owner) {
     this.owner = owner;
   }
 
-  public static AutoSerializableManager getInstance(ReflectionBasedAutoSerializer owner) {
-    return instances.get(owner);
-  }
-
-
   public Map<Class<?>, AutoClassInfo> getClassMap() {
     return classMap;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c7cbb2d/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/TypeRegistry.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/TypeRegistry.java b/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/TypeRegistry.java
index 4368e84..4ca1a90 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/TypeRegistry.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/pdx/internal/TypeRegistry.java
@@ -321,13 +321,13 @@ public class TypeRegistry {
     if (v == null) {
       PdxSerializer oldValue = pdxSerializer.getAndSet(null);
       if (oldValue instanceof ReflectionBasedAutoSerializer) {
-        asm.compareAndSet(AutoSerializableManager.getInstance((ReflectionBasedAutoSerializer) oldValue), null);
+        asm.compareAndSet((AutoSerializableManager) ((ReflectionBasedAutoSerializer) oldValue).getManager(), null);
       }
     } else {
       pdxSerializerWasSet = true;
       pdxSerializer.set(v);
       if (v instanceof ReflectionBasedAutoSerializer) {
-        asm.set(AutoSerializableManager.getInstance((ReflectionBasedAutoSerializer) v));
+        asm.set((AutoSerializableManager) ((ReflectionBasedAutoSerializer) v).getManager());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1c7cbb2d/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/AutoSerializableJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/AutoSerializableJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/AutoSerializableJUnitTest.java
index e3ab9ca..cb9d0f3 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/AutoSerializableJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/pdx/AutoSerializableJUnitTest.java
@@ -103,7 +103,7 @@ public class AutoSerializableJUnitTest {
   
   private void setupSerializer(ReflectionBasedAutoSerializer s, boolean readSerialized) {
     this.serializer = s;
-    this.manager = AutoSerializableManager.getInstance(s);
+    this.manager = (AutoSerializableManager) s.getManager();
     this.c = (GemFireCacheImpl) new CacheFactory().
     set("mcast-port", "0").
     setPdxReadSerialized(readSerialized).


[19/50] [abbrv] incubator-geode git commit: Fix for GEODE-110

Posted by bs...@apache.org.
Fix for GEODE-110

In the context of distributed transactions, DistributedPutAllOperation.createPRMessages calls putAllData[i].setFakeEventID() to set event ID for each entry in put all op (called from DistTXStateProxyImplOnCoordinator.postPutAll->DistPeerTXStateStub.postPutAll->TXStateStub.postPutAll -> PartitionedTXRegionStub.postPutAll). However when the data node itself is a transaction coordinator, createPRMessages is not called and the data is locally put by calling DistTXStateProxyImplOnCoordinator.postPutAll->DistTXStateOnCoordinator.postPutAll(). This call sequence does not call putAllData[i].setFakeEventID(). At the commit time when the coordinator sends TX events to secondaries, some of the events from different buckets had same event ID. This caused TXState.txPutEntry() to skip those events (assuming that the entry has been processed already) and therefore the test failed. The fix is to call putallOp.putAllData[i].setFakeEventID() in DistTXStateProxyImplOnCoordinator.postPutAll so that even
 tID is set for all entries. Similar fix is done for remove all.

Also enabled the assertion that was surrounded by logger.isDebugEnabled() call due to this issue


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

Branch: refs/heads/feature/GEODE-77
Commit: bc4508c13bdf23c5c664ecc08d03c706d640403f
Parents: 7ae2752
Author: shirishd <sd...@pivotal.io>
Authored: Tue Aug 4 12:02:39 2015 +0530
Committer: shirishd <sd...@pivotal.io>
Committed: Tue Aug 4 12:02:39 2015 +0530

----------------------------------------------------------------------
 .../cache/DistTXStateProxyImplOnCoordinator.java   |  2 ++
 .../gemfire/internal/cache/TXRegionState.java      | 17 +++++------------
 2 files changed, 7 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bc4508c1/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXStateProxyImplOnCoordinator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXStateProxyImplOnCoordinator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXStateProxyImplOnCoordinator.java
index 2f79605..b44713f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXStateProxyImplOnCoordinator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/DistTXStateProxyImplOnCoordinator.java
@@ -888,6 +888,7 @@ public class DistTXStateProxyImplOnCoordinator extends DistTXStateProxyImpl {
               event, putallOp.putAllDataSize, putallOp.isBridgeOp);
           bucketToPutallMap.put(bucketId, putAllForBucket);
         } 
+        putallOp.putAllData[i].setFakeEventID();
         putAllForBucket.addEntry(putallOp.putAllData[i]);
 
         KeyInfo ki = new KeyInfo(key, null, null);
@@ -965,6 +966,7 @@ public class DistTXStateProxyImplOnCoordinator extends DistTXStateProxyImpl {
               event, op.removeAllDataSize, op.isBridgeOp);
           bucketToRemoveAllMap.put(bucketId, removeAllForBucket);
         } 
+        op.removeAllData[i].setFakeEventID();
         removeAllForBucket.addEntry(op.removeAllData[i]);
 
         KeyInfo ki = new KeyInfo(key, null, null);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bc4508c1/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
index 67cb8c5..bb6ae5f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
@@ -596,18 +596,11 @@ public class TXRegionState {
     String regionFullPath = this.getRegion().getFullPath();
     int entryModsSize = this.entryMods.size();
     int entryEventListSize = entryEventList.size();
-    /*
-     * [DISTTX] TODO
-     * This assertion is not working for PutAll and RemoveAll operations 
-     * and thus guarding within Debug flags. May be enabled at later stage.
-     */
-    if (logger.isDebugEnabled()) {
-      if (entryModsSize != entryEventListSize) {
-        throw new UnsupportedOperationInTransactionException(
-            LocalizedStrings.DISTTX_TX_EXPECTED
-                .toLocalizedString("entry size of " + entryModsSize
-                    + " for region " + regionFullPath, entryEventListSize));
-      }
+    if (entryModsSize != entryEventListSize) {
+      throw new UnsupportedOperationInTransactionException(
+          LocalizedStrings.DISTTX_TX_EXPECTED.toLocalizedString(
+              "entry size of " + entryModsSize + " for region "
+                  + regionFullPath, entryEventListSize));
     }
 
     int index = 0;


[34/50] [abbrv] incubator-geode git commit: Fix for GEODE-109

Posted by bs...@apache.org.
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/1a6a0ef5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/1a6a0ef5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/1a6a0ef5

Branch: refs/heads/feature/GEODE-77
Commit: 1a6a0ef53acfdc3931f48881b4b90907c16dc972
Parents: d0d4858
Author: Vito Gavrilov <vg...@pivotal.io>
Authored: Tue Jul 28 09:40:42 2015 -0700
Committer: Swapnil Bawaskar <sb...@pivotal.io>
Committed: Thu Aug 6 11:25:54 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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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/1a6a0ef5/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 {



[48/50] [abbrv] incubator-geode git commit: GEODE-209: Change subTearDown to destroy process

Posted by bs...@apache.org.
GEODE-209: Change subTearDown to destroy process


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

Branch: refs/heads/feature/GEODE-77
Commit: 29145676960c7d75f1da04cf618f71f6a399eda7
Parents: 41d0492
Author: Kirk Lund <kl...@pivotal.io>
Authored: Thu Aug 13 13:22:02 2015 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Fri Aug 14 09:40:55 2015 -0700

----------------------------------------------------------------------
 .../golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java    | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/29145676/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java
index 1b8a312..bd55e6b 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java
@@ -16,8 +16,7 @@ public class FailWithTimeoutOfWaitForOutputToMatchJUnitTest extends FailOutputTe
   private ProcessWrapper process;
   
   public void subTearDown() throws Exception {
-    this.process.waitFor();
-    assertFalse(this.process.isAlive());
+    this.process.destroy();
   }
   
   @Override


[24/50] [abbrv] incubator-geode git commit: GEODE-187:AsyncEventQueue with Event Filter test log shows "IllegalStateException" -AbstractGatewaySenderEventProcessor#processqueue iterates events to get filtered. While iterating 2 consditions are checked a

Posted by bs...@apache.org.
GEODE-187:AsyncEventQueue with Event Filter test log shows "IllegalStateException"
 -AbstractGatewaySenderEventProcessor#processqueue iterates events to get filtered. While iterating 2 consditions are checked and event is removed in thses both condition. for single itr.next() when itr.remove is called second time, it causes IllegalStateException.
 -Added 'continue' after first itr.remove is called


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

Branch: refs/heads/feature/GEODE-77
Commit: 01145b8e9eeec8d7fdf8e6fed500ff54021ab609
Parents: 8e4c704
Author: kbachhav <kbachhav.pivotal.io>
Authored: Wed Aug 5 12:35:10 2015 +0530
Committer: kbachhav <kbachhav.pivotal.io>
Committed: Wed Aug 5 12:35:10 2015 +0530

----------------------------------------------------------------------
 .../internal/cache/wan/AbstractGatewaySenderEventProcessor.java     | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/01145b8e/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySenderEventProcessor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySenderEventProcessor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySenderEventProcessor.java
index 67b3894..6a0392a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySenderEventProcessor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/AbstractGatewaySenderEventProcessor.java
@@ -499,6 +499,7 @@ public abstract class AbstractGatewaySenderEventProcessor extends Thread {
                 
                 itr.remove();
                 statistics.incEventsNotQueued();
+                continue;
               }
               
               boolean transmit = filter.beforeTransmit(event);


[05/50] [abbrv] incubator-geode git commit: GEODE-127: Improve test reliability and execution speed.

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapper.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapper.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapper.java
index 3931b22..5bf2408 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapper.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapper.java
@@ -1,5 +1,7 @@
 package com.gemstone.gemfire.test.process;
 
+import static org.junit.Assert.*;
+
 import java.io.File;
 import java.io.PrintStream;
 import java.util.ArrayList;
@@ -12,6 +14,7 @@ import java.util.Properties;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.regex.Pattern;
 
@@ -19,28 +22,26 @@ import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.internal.logging.LogService;
 
-import com.gemstone.gemfire.test.process.*;
-
-import junit.framework.Assert;
-
 /**
- * Wraps spawned process to capture output and provide hooks to the
- * {@link java.lang.Process} object.
+ * Wraps spawned {@link java.lang.Process} to capture output and provide interaction with the process.
  *
  * @author Kirk Lund
  * @since 4.1.1
  */
-public class ProcessWrapper extends Assert {
+public class ProcessWrapper {
   private static final Logger logger = LogService.getLogger();
 
-  protected static final String TIMEOUT_MILLIS_PROPERTY = "quickstart.test.TIMEOUT_MILLIS";
-  private static final long TIMEOUT_MILLIS = Long.getLong(TIMEOUT_MILLIS_PROPERTY, 5 * 60 * 1000L); 
-  private static final boolean JAVA_AWT_HEADLESS = true;
+  protected static final String TIMEOUT_MILLIS_PROPERTY = "process.test.timeoutMillis";
+  protected static final long TIMEOUT_MILLIS_DEFAULT = 5 * 60 * 1000;
+  private static final long DELAY = 10;
 
-  private final String[] jvmArgs;  
+  private final boolean headless;
+  private final long timeoutMillis; 
+  
+  private final String[] jvmArguments;  
   
   private final Class<?> mainClass;
-  private final String[] mainArgs;
+  private final String[] mainArguments;
 
   private volatile Process process;
   private volatile Throwable processException;
@@ -60,54 +61,45 @@ public class ProcessWrapper extends Assert {
   private ProcessStreamReader stdout;
   private ProcessStreamReader stderr;
   
-  public ProcessWrapper(Class<?> main, String[] mainArgs) {
-    this(main, mainArgs, true);
-  }
-
-  public ProcessWrapper(Class<?> main, String[] mainArgs, boolean useMainLauncher) {
-    this(null, main, mainArgs, true);
-  }
-  
-  public ProcessWrapper(String[] jvmArgs, Class<?> main, String[] mainArgs, boolean useMainLauncher) {
-    this.jvmArgs = jvmArgs;
+  private ProcessWrapper(final String[] jvmArguments, final Class<?> mainClass, final String[] mainArguments, final boolean useMainLauncher, final boolean headless, final long timeoutMillis) {
+    this.jvmArguments = jvmArguments;
+    this.mainClass = mainClass;
+    this.mainArguments = mainArguments;
+    this.useMainLauncher = useMainLauncher;
+    this.headless = headless;
+    this.timeoutMillis = timeoutMillis;
     
-    this.mainClass = main;
-    this.mainArgs = mainArgs;
-
     this.lineBuffer = new LinkedBlockingQueue<String>();
     this.allLines = Collections.synchronizedList(new ArrayList<String>());
-    
-    this.useMainLauncher = useMainLauncher;
   }
   
-  public ProcessStreamReader getStandardOutReader() { // TODO:protected
+  public ProcessStreamReader getStandardOutReader() {
     synchronized (this.exitValue) {
       return stdout;
     }
   }
   
-  public ProcessStreamReader getStandardErrorReader() { // TODO:protected
+  public ProcessStreamReader getStandardErrorReader() {
     synchronized (this.exitValue) {
       return stderr;
     }
   }
   
-  private void waitForProcessStart() throws InterruptedException {
-    long start = System.currentTimeMillis();
+  private void waitForProcessStart() throws InterruptedException, TimeoutException {
+    final long start = System.currentTimeMillis();
     boolean done = false;
     while (!done) {
       synchronized (this.exitValue) {
-        done = (this.process != null || this.processException != null) && 
-            (this.started || this.exitValue.get() > -1 || this.interrupted);
+        done = (this.process != null || this.processException != null) && (this.started || this.exitValue.get() > -1 || this.interrupted);
       }
-      if (!done && System.currentTimeMillis() > start + TIMEOUT_MILLIS) {
-        fail("Timed out launching process");
+      if (!done && System.currentTimeMillis() > start + timeoutMillis) {
+        throw new TimeoutException("Timed out launching process");
       }
-      Thread.sleep(100);
+      Thread.sleep(DELAY);
     }
   }
   
-  public boolean isAlive() throws InterruptedException {
+  public boolean isAlive() throws InterruptedException, TimeoutException {
     checkStarting();
     waitForProcessStart();
     
@@ -126,9 +118,9 @@ public class ProcessWrapper extends Assert {
     return this;
   }
 
-  public int waitFor(long timeout, boolean throwOnTimeout) throws InterruptedException {
+  public int waitFor(final long timeout, final boolean throwOnTimeout) throws InterruptedException {
     checkStarting();
-    Thread thread = getThread();
+    final Thread thread = getThread();
     thread.join(timeout);
     synchronized (this.exitValue) {
       if (throwOnTimeout) {
@@ -138,29 +130,29 @@ public class ProcessWrapper extends Assert {
     }
   }
   
-  public int waitFor(long timeout) throws InterruptedException {
+  public int waitFor(final long timeout) throws InterruptedException {
     return waitFor(timeout, false);
   }
   
-  public int waitFor(boolean throwOnTimeout) throws InterruptedException {
-    return waitFor(TIMEOUT_MILLIS, throwOnTimeout);
+  public int waitFor(final boolean throwOnTimeout) throws InterruptedException {
+    return waitFor(timeoutMillis, throwOnTimeout);
   }
   
   public int waitFor() throws InterruptedException {
-    return waitFor(TIMEOUT_MILLIS, false);
+    return waitFor(timeoutMillis, false);
   }
   
   public String getOutput() { 
     return getOutput(false);
   }
 
-  public String getOutput(boolean ignoreStopped) { 
+  public String getOutput(final boolean ignoreStopped) { 
     checkStarting();
     if (!ignoreStopped) {
       checkStopped();
     }
-    StringBuffer sb = new StringBuffer();
-    Iterator<String> iterator = allLines.iterator();
+    final StringBuffer sb = new StringBuffer();
+    final Iterator<String> iterator = this.allLines.iterator();
     while (iterator.hasNext()) {
       sb.append(iterator.next() + "\n");
     }
@@ -173,25 +165,24 @@ public class ProcessWrapper extends Assert {
     return this;
   }
 
-  public ProcessWrapper sendInput(String input) {
+  public ProcessWrapper sendInput(final String input) {
     checkStarting();
-    PrintStream ps = new PrintStream(this.process.getOutputStream());
+    final PrintStream ps = new PrintStream(this.process.getOutputStream());
     ps.println(input);
     ps.flush();
     return this;
   }
 
-  public ProcessWrapper failIfOutputMatches(String patternString, long timeoutMillis) throws InterruptedException {
+  public ProcessWrapper failIfOutputMatches(final String patternString, final long timeoutMillis) throws InterruptedException {
     checkStarting();
     checkOk();
-
-    Pattern pattern = Pattern.compile(patternString);
     
+    final Pattern pattern = Pattern.compile(patternString);
     logger.debug("failIfOutputMatches waiting for \"{}\"...", patternString);
-    long start = System.currentTimeMillis();
+    final long start = System.currentTimeMillis();
+    
     while(System.currentTimeMillis() <= start+timeoutMillis) {
-      String line = lineBuffer.poll(timeoutMillis, TimeUnit.MILLISECONDS);
-
+      final String line = lineBuffer.poll(timeoutMillis, TimeUnit.MILLISECONDS);
       if (line != null && pattern.matcher(line).matches()) {
         fail("failIfOutputMatches Matched pattern \"" + patternString + "\" against output \"" + line + "\". Output: " + this.allLines);
       }
@@ -203,18 +194,17 @@ public class ProcessWrapper extends Assert {
    * Waits for the process stdout or stderr stream to contain the specified 
    * text. Uses the specified timeout for debugging purposes.
    */
-  public ProcessWrapper waitForOutputToMatch(String patternString, long timeoutMillis) throws InterruptedException {
+  public ProcessWrapper waitForOutputToMatch(final String patternString, final long timeoutMillis) throws InterruptedException {
     checkStarting();
     checkOk();
 
-    Pattern pattern = Pattern.compile(patternString);
-    
+    final Pattern pattern = Pattern.compile(patternString);
     logger.debug("ProcessWrapper:waitForOutputToMatch waiting for \"{}\"...", patternString);
+    
     while(true) {
-      String line = this.lineBuffer.poll(timeoutMillis, TimeUnit.MILLISECONDS);
-
+      final String line = this.lineBuffer.poll(timeoutMillis, TimeUnit.MILLISECONDS);
       if (line == null) {
-        fail("Timed out waiting for output \"" + patternString + "\" after " + TIMEOUT_MILLIS + " ms. Output: " + new OutputFormatter(this.allLines));
+        fail("Timed out waiting for output \"" + patternString + "\" after " + timeoutMillis + " ms. Output: " + new OutputFormatter(this.allLines));
       }
       
       if (pattern.matcher(line).matches()) {
@@ -231,19 +221,19 @@ public class ProcessWrapper extends Assert {
    * Waits for the process stdout or stderr stream to contain the specified 
    * text. Uses the default timeout.
    */
-  public ProcessWrapper waitForOutputToMatch(String patternString) throws InterruptedException {
-    return waitForOutputToMatch(patternString, TIMEOUT_MILLIS);
+  public ProcessWrapper waitForOutputToMatch(final String patternString) throws InterruptedException {
+    return waitForOutputToMatch(patternString, timeoutMillis);
   }
 
-  public ProcessWrapper execute() throws InterruptedException {
+  public ProcessWrapper execute() throws InterruptedException, TimeoutException {
     return execute(null, new File(System.getProperty("user.dir")));
   }
 
-  public ProcessWrapper execute(Properties props) throws InterruptedException {
-    return execute(props, new File(System.getProperty("user.dir")));
+  public ProcessWrapper execute(final Properties properties) throws InterruptedException, TimeoutException {
+    return execute(properties, new File(System.getProperty("user.dir")));
   }
   
-  public ProcessWrapper execute(final Properties props, final File workingDirectory) throws InterruptedException {
+  public ProcessWrapper execute(final Properties properties, final File workingDirectory) throws InterruptedException, TimeoutException {
     synchronized (this.exitValue) {
       if (this.starting) {
         throw new IllegalStateException("ProcessWrapper can only be executed once");
@@ -251,7 +241,7 @@ public class ProcessWrapper extends Assert {
       this.starting = true;
       this.processThread = new Thread(new Runnable() {
         public void run() {
-          exec(props, workingDirectory);
+          start(properties, workingDirectory);
         }
       }, "ProcessWrapper Process Thread");
     }
@@ -261,7 +251,7 @@ public class ProcessWrapper extends Assert {
 
     synchronized (this.exitValue) {
       if (this.processException != null) {
-        System.out.println("ProcessWrapper:execute failed with " + this.processException);
+        logger.error("ProcessWrapper:execute failed with " + this.processException);
         this.processException.printStackTrace();
       }
     }
@@ -272,36 +262,35 @@ public class ProcessWrapper extends Assert {
     return this;
   }
 
-  private void exec(Properties dsProps, final File workingDirectory) {
-    List<String> vmArgList = new ArrayList<String>();
+  private void start(final Properties properties, final File workingDirectory) {
+    final List<String> jvmArgumentsList = new ArrayList<String>();
 
-    if (dsProps != null) {
-      for (Map.Entry<Object, Object> entry : dsProps.entrySet()) {
+    if (properties != null) {
+      for (Map.Entry<Object, Object> entry : properties.entrySet()) {
         if (!entry.getKey().equals("log-file")) {
-          vmArgList.add("-D" + entry.getKey() + "=" + entry.getValue());
+          jvmArgumentsList.add("-D" + entry.getKey() + "=" + entry.getValue());
         }
       }
     }
 
-    if (JAVA_AWT_HEADLESS) {
-      vmArgList.add("-Djava.awt.headless=true");
+    if (this.headless) {
+      jvmArgumentsList.add("-Djava.awt.headless=true");
     }
     
-    if (this.jvmArgs != null) {
-      for (String vmArg: this.jvmArgs) {
-        vmArgList.add(vmArg);
+    if (this.jvmArguments != null) {
+      for (String jvmArgument: this.jvmArguments) {
+        jvmArgumentsList.add(jvmArgument);
       }
     }
     
-    String[] vmArgs = vmArgList.toArray(new String[vmArgList.size()]);
-
     try {
       synchronized (this.exitValue) {
-        String[] command = defineCommand(vmArgs);
+        final String[] command = defineCommand(jvmArgumentsList.toArray(new String[jvmArgumentsList.size()]));
         this.process = new ProcessBuilder(command).directory(workingDirectory).start();
         
-        StringBuilder processCommand = new StringBuilder();
+        final StringBuilder processCommand = new StringBuilder();
         boolean addSpace = false;
+        
         for (String string : command) {
           if (addSpace) {
             processCommand.append(" ");
@@ -309,16 +298,16 @@ public class ProcessWrapper extends Assert {
           processCommand.append(string);
           addSpace = true;
         }
-        String commandString = processCommand.toString();
-        System.out.println("Executing " + commandString);
+        
+        final String commandString = processCommand.toString();
+        logger.debug("Starting " + commandString);
+        
         final ProcessStreamReader stdOut = new ProcessStreamReader(commandString, this.process.getInputStream(), this.lineBuffer, this.allLines);
         final ProcessStreamReader stdErr = new ProcessStreamReader(commandString, this.process.getErrorStream(), this.lineBuffer, this.allLines);
   
         this.stdout = stdOut;
         this.stderr = stdErr;
-  
         this.outputReader = new ProcessOutputReader(this.process, stdOut, stdErr, this.allLines);
-      
         this.started = true;
       }
       
@@ -342,51 +331,40 @@ public class ProcessWrapper extends Assert {
     }
   }
   
-  private String[] defineCommand(String[] vmArgs) {
-    File javabindir = new File(System.getProperty("java.home"), "bin");
-    File javaexe = new File(javabindir, "java");
+  private String[] defineCommand(final String[] jvmArguments) {
+    final File javaBinDir = new File(System.getProperty("java.home"), "bin");
+    final File javaExe = new File(javaBinDir, "java");
 
-    List<String> argList = new ArrayList<String>();
-    argList.add(javaexe.getPath());
-    argList.add("-classpath");
-    argList.add(System.getProperty("java.class.path"));
+    final List<String> argumentList = new ArrayList<String>();
+    argumentList.add(javaExe.getPath());
+    argumentList.add("-classpath");
+    argumentList.add(System.getProperty("java.class.path"));
 
     // -d64 is not a valid option for windows and results in failure
-    int bits = Integer.getInteger("sun.arch.data.model", 0).intValue();
+    final int bits = Integer.getInteger("sun.arch.data.model", 0).intValue();
     if (bits == 64 && !(System.getProperty("os.name").toLowerCase().contains("windows"))) {
-      argList.add("-d64");
+      argumentList.add("-d64");
     }
 
-    argList.add("-Djava.library.path=" + System.getProperty("java.library.path"));
+    argumentList.add("-Djava.library.path=" + System.getProperty("java.library.path"));
 
-    if (vmArgs != null) {
-      argList.addAll(Arrays.asList(vmArgs));
+    if (jvmArguments != null) {
+      argumentList.addAll(Arrays.asList(jvmArguments));
     }
 
     if (this.useMainLauncher) {
-      argList.add(MainLauncher.class.getName());
+      argumentList.add(MainLauncher.class.getName());
     }
-    argList.add(mainClass.getName());
+    argumentList.add(mainClass.getName());
     
-    if (mainArgs != null) {
-      argList.addAll(Arrays.asList(mainArgs));
+    if (mainArguments != null) {
+      argumentList.addAll(Arrays.asList(mainArguments));
     }
 
-    String[] cmd = argList.toArray(new String[argList.size()]);
-    return cmd;
+    final String[] command = argumentList.toArray(new String[argumentList.size()]);
+    return command;
   }
   
-  private String toString(String[] strings) {
-    if (strings == null || strings.length < 1) {
-      return null;
-    }
-    StringBuilder sb = new StringBuilder();
-    for (String string : strings) {
-      sb.append(string).append("\n");
-    }
-    return sb.toString();
-  }
-
   private void checkStarting() throws IllegalStateException {
     synchronized (this.exitValue) {
       if (!this.starting) {
@@ -405,8 +383,7 @@ public class ProcessWrapper extends Assert {
   
   private void checkOk() throws RuntimeException {
     if (this.processException != null) {
-      RuntimeException rt = new RuntimeException("Failed to launch process", this.processException);
-      throw rt;
+      throw new RuntimeException("Failed to launch process", this.processException);
     }
   }
 
@@ -416,8 +393,9 @@ public class ProcessWrapper extends Assert {
     }
   }
   
+  @Override
   public String toString() {
-    StringBuilder sb = new StringBuilder(getClass().getSimpleName());
+    final StringBuilder sb = new StringBuilder(getClass().getSimpleName());
     sb.append("@").append(System.identityHashCode(this)).append("{");
     sb.append(this.mainClass);
     sb.append("}");
@@ -425,31 +403,46 @@ public class ProcessWrapper extends Assert {
   }
 
   public static class Builder {
-    private String[] jvmArgs = null;
-    private Class<?> main;
-    private String[] mainArgs = null;
+    private String[] jvmArguments = null;
+    private Class<?> mainClass;
+    private String[] mainArguments = null;
     private boolean useMainLauncher = true;
+    private boolean headless = true;
+    private long timeoutMillis = TIMEOUT_MILLIS_DEFAULT;
+    private boolean inline = false;
     public Builder() {
       //nothing
     }
-    public Builder jvmArgs(String[] jvmArgs) {
-      this.jvmArgs = jvmArgs;
+    public Builder jvmArguments(final String[] jvmArguments) {
+      this.jvmArguments = jvmArguments;
       return this;
     }
-    public Builder main(Class<?> main) { 
-      this.main = main;
+    public Builder mainClass(final Class<?> mainClass) { 
+      this.mainClass = mainClass;
       return this;
     }
-    public Builder mainArgs(String[] mainArgs) {
-      this.mainArgs = mainArgs;
+    public Builder mainArguments(final String[] mainArguments) {
+      this.mainArguments = mainArguments;
       return this;
     }
-    public Builder useMainLauncher(boolean useMainLauncher) {
+    public Builder useMainLauncher(final boolean useMainLauncher) {
       this.useMainLauncher = useMainLauncher;
       return this;
     }
+    public Builder headless(final boolean headless) {
+      this.headless = headless;
+      return this;
+    }
+    public Builder timeoutMillis(final long timeoutMillis) {
+      this.timeoutMillis = timeoutMillis;
+      return this;
+    }
+    public Builder inline(final boolean inline) {
+      this.inline = inline;
+      return this;
+    }
     public ProcessWrapper build() {
-      return new ProcessWrapper(jvmArgs, main, mainArgs, useMainLauncher);
+      return new ProcessWrapper(jvmArguments, mainClass, mainArguments, useMainLauncher, headless, timeoutMillis);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapperJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapperJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapperJUnitTest.java
index 7c08fcc..a548e75 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapperJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessWrapperJUnitTest.java
@@ -6,18 +6,14 @@ import org.junit.After;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
- * Unit test basic functionality in ProcessWrapper.
+ * Integration tests for ProcessWrapper.
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class ProcessWrapperJUnitTest {
 
   private static final String OUTPUT_OF_MAIN = "Executing ProcessWrapperJUnitTest main";
@@ -33,23 +29,20 @@ public class ProcessWrapperJUnitTest {
   @Test
   public void testClassPath() throws Exception {
     final String classPath = System.getProperty("java.class.path");
-    System.out.println("Classpath: " + classPath);
     assertTrue("Classpath is missing log4j-api: " + classPath, classPath.toLowerCase().contains("log4j-api"));
     assertTrue("Classpath is missing log4j-core: " + classPath, classPath.toLowerCase().contains("log4j-core"));
     assertTrue("Classpath is missing fastutil: " + classPath, classPath.toLowerCase().contains("fastutil"));
   
-    System.out.println(String.valueOf(Integer.MAX_VALUE));
-    
-    this.process = new ProcessWrapper.Builder().main(getClass()).build();
+    this.process = new ProcessWrapper.Builder().mainClass(getClass()).build();
     this.process.execute();
     this.process.waitFor();
-    System.out.println("Output: " + process.getOutput());
+    
     assertTrue("Output is wrong: " + process.getOutput(), process.getOutput().contains(OUTPUT_OF_MAIN));
   }
   
   @Test
   public void testInvokeWithNullArgs() throws Exception {
-    this.process = new ProcessWrapper.Builder().main(getClass()).build();
+    this.process = new ProcessWrapper.Builder().mainClass(getClass()).build();
     this.process.execute();
     this.process.waitFor();
     assertTrue(process.getOutput().contains(OUTPUT_OF_MAIN));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/resources/com/gemstone/gemfire/test/golden/log4j2-test.xml
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/resources/com/gemstone/gemfire/test/golden/log4j2-test.xml b/gemfire-core/src/test/resources/com/gemstone/gemfire/test/golden/log4j2-test.xml
new file mode 100755
index 0000000..6a885ef
--- /dev/null
+++ b/gemfire-core/src/test/resources/com/gemstone/gemfire/test/golden/log4j2-test.xml
@@ -0,0 +1,18 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<Configuration status="ERROR" shutdownHook="disable" packages="com.gemstone.org.apache.logging.log4j.core.config.xml,com.gemstone.gemfire.internal.logging.log4j">
+  <Properties>
+    <Property name="gemfire-pattern">[%level{lowerCase=true} %date{yyyy/MM/dd HH:mm:ss.SSS z} &lt;%thread&gt; tid=%tid] %message%n%throwable%n</Property>
+  </Properties>
+  <Appenders>
+    <Console name="STDOUT" target="SYSTEM_OUT">
+      <PatternLayout pattern="${gemfire-pattern}"/>
+    </Console>
+  </Appenders>
+  <Loggers>
+    <Logger name="com.gemstone" level="WARN" additivity="true"/>
+    <Logger name="quickstart" level="WARN" additivity="true"/>
+    <Root level="ERROR">
+      <AppenderRef ref="STDOUT"/>
+    </Root>
+  </Loggers>
+</Configuration>


[36/50] [abbrv] incubator-geode git commit: GEODE-191 fix for transactional region.size()

Posted by bs...@apache.org.
GEODE-191 fix for transactional region.size()

In the merge from sga2 branch, entryCount() no longer consulted the dataView() which led to getting wrong size when a transaction is hosted on a remote member.


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

Branch: refs/heads/feature/GEODE-77
Commit: b1e7466cb64fd6413c4658026498e5e6f4b8a7c8
Parents: fcb42ad
Author: Swapnil Bawaskar <sb...@pivotal.io>
Authored: Fri Aug 7 13:48:22 2015 -0700
Committer: Swapnil Bawaskar <sb...@pivotal.io>
Committed: Fri Aug 7 13:48:22 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/LocalRegion.java     |  2 +-
 .../cache/ClientServerTransactionDUnitTest.java |  6 ++-
 .../cache/RemoteTransactionDUnitTest.java       | 41 ++++++++++++++++----
 3 files changed, 39 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b1e7466c/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
index 495e992..617a7ec 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
@@ -2234,7 +2234,7 @@ public class LocalRegion extends AbstractRegion
    * author David Whitlock
    */
   public final int entryCount() {
-    return entryCount(null);
+    return getDataView().entryCount(this);
   }
 
   public int entryCount(Set<Integer> buckets) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b1e7466c/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
index d80f6bb..51a8dea 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
@@ -2130,7 +2130,11 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
   }
 
   @Override
-  public void testSize() {
+  public void testSizeForTXHostedOnRemoteNode() {
+  }
+
+  @Override
+  public void testSizeOnAccessor() {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b1e7466c/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
index a78fab4..5834622 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
@@ -1721,14 +1721,24 @@ public class RemoteTransactionDUnitTest extends CacheTestCase {
     });
   }
 
-  public void testSize() {
+  public void testSizeForTXHostedOnRemoteNode() {
+    doSizeTest(false);
+  }
+
+  public void testSizeOnAccessor() {
+    doSizeTest(true);
+  }
+
+  private void doSizeTest(final boolean isAccessor) {
     Host host = Host.getHost(0);
     VM accessor = host.getVM(0);
     VM datastore1 = host.getVM(1);
     VM datastore2 = host.getVM(2);
     initAccessorAndDataStore(accessor, datastore1, datastore2, 0);
 
-    accessor.invoke(new SerializableCallable() {
+    VM taskVM = isAccessor ? accessor : datastore1;
+
+    taskVM.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         Region custRegion = getCache().getRegion(CUSTOMER);
         TXManagerImpl mgr = getGemfireCache().getTxManager();
@@ -1741,16 +1751,31 @@ public class RemoteTransactionDUnitTest extends CacheTestCase {
     datastore1.invoke(verifyNoTxState);
     datastore2.invoke(verifyNoTxState);
     
-    accessor.invoke(new SerializableCallable() {
+    taskVM.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         Region custRegion = getCache().getRegion(CUSTOMER);
         Region orderRegion = getCache().getRegion(ORDER);
         TXManagerImpl mgr = getGemfireCache().getTxManager();
+        TransactionId txId = mgr.suspend();
+        PartitionedRegion custPR = (PartitionedRegion)custRegion;
+        int remoteKey = -1;
+        for (int i=100; i<200; i++) {
+          DistributedMember myId = custPR.getMyId();
+          if (!myId.equals(custPR.getOwnerForKey(custPR.getKeyInfo(new CustId(i))))) {
+            remoteKey = i;
+            break;
+          }
+        }
+        if (remoteKey == -1) {
+          throw new IllegalStateException("expected non-negative key");
+        }
+        mgr.resume(txId);
         assertNotNull(mgr.getTXState());
-        CustId custId = new CustId(5);
-        OrderId orderId = new OrderId(5, custId);
-        custRegion.put(custId, new Customer("customer5", "address5"));
-        orderRegion.put(orderId, new Order("order5"));
+        CustId custId = new CustId(remoteKey);
+        OrderId orderId = new OrderId(remoteKey, custId);
+        custRegion.put(custId, new Customer("customer"+remoteKey, "address"+remoteKey));
+        getCache().getLogger().info("Putting "+custId+", keyInfo:"+custPR.getKeyInfo(new CustId(remoteKey)));
+        orderRegion.put(orderId, new Order("order"+remoteKey));
         assertEquals(6, custRegion.size());
         return mgr.getTransactionId();
       }
@@ -1759,7 +1784,7 @@ public class RemoteTransactionDUnitTest extends CacheTestCase {
     final Integer txOnDatastore2 = (Integer)datastore2.invoke(getNumberOfTXInProgress);
     assertEquals(1, txOnDatastore1+txOnDatastore2);
 
-    accessor.invoke(new SerializableCallable() {
+    taskVM.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         CacheTransactionManager mgr = getGemfireCache().getTxManager();
         mgr.commit();


[45/50] [abbrv] incubator-geode git commit: GEODE-196: Remove unneeded system out in RuntimeIterator

Posted by bs...@apache.org.
GEODE-196: Remove unneeded system out in RuntimeIterator


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

Branch: refs/heads/feature/GEODE-77
Commit: 1eccf917198b0d2ce1e3f350fbf7317d322f2eed
Parents: 5054a8a
Author: Jason Huynh <jh...@pivotal.io>
Authored: Thu Aug 13 11:13:51 2015 -0700
Committer: Jason Huynh <jh...@pivotal.io>
Committed: Thu Aug 13 11:13:51 2015 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/cache/query/internal/RuntimeIterator.java    | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1eccf917/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/RuntimeIterator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/RuntimeIterator.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/RuntimeIterator.java
index 5b3944a..6bf0da2 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/RuntimeIterator.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/RuntimeIterator.java
@@ -137,9 +137,6 @@ public class RuntimeIterator extends AbstractCompiledValue  {
   }
 
   public Object evaluate(ExecutionContext context) {
-    if(current == UNINITIALIZED) {
-      System.out.println("asif");
-    }
     Support.Assert(current != UNINITIALIZED,
         "error to evaluate RuntimeIterator without setting current first");
     return this.current;


[47/50] [abbrv] incubator-geode git commit: GEODE-216: Handle any exception that might be thrown

Posted by bs...@apache.org.
GEODE-216: Handle any exception that might be thrown

The old code was trying to prevent a deadlock under some certain
conditions. But if it can not get a PR because of an exception
then a deadlock is not possible. So the new code just catches
and ignores exceptions and falls through to the code that will
process the message the old way. This code will encounter the
same exception and has the proper logic to handle it.


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

Branch: refs/heads/feature/GEODE-77
Commit: 41d04928be3a455b02e58432713ed694f5f2b238
Parents: 587f589
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Thu Aug 13 16:16:43 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Aug 13 16:16:43 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/partitioned/GetMessage.java         | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/41d04928/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
index c8d990d..64cbccd 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/GetMessage.java
@@ -129,6 +129,12 @@ public final class GetMessage extends PartitionMessageWithDirectReply
           return DistributionManager.PARTITIONED_REGION_EXECUTOR;
         }
       } catch (PRLocallyDestroyedException ignore) {
+      } catch (RuntimeException ignore) {
+        // fix for GEODE-216
+        // Most likely here would be RegionDestroyedException or CacheClosedException
+        // but the cancel criteria code can throw any RuntimeException.
+        // In all these cases it is ok to just fall through and return the
+        // old executor type.
       }
     }
     if (forceUseOfPRExecutor) {


[04/50] [abbrv] incubator-geode git commit: GEODE-124: Add min size check and optimize calls

Posted by bs...@apache.org.
GEODE-124: Add min size check and optimize calls

* Member stats are needed for computing member load skew and total data size.
  The gathering step is common and need not be executed twice
* If the total transfer size during rebalance is low, then avoid rebalance. This
  could happen when a new cluster is created is data is being loaded


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

Branch: refs/heads/feature/GEODE-77
Commit: 1683361bc87368e48d8cfefebf6dbcda932ed90c
Parents: d307eda
Author: Ashvin Agrawal <as...@apache.org>
Authored: Mon Jul 27 22:11:27 2015 -0700
Committer: Ashvin Agrawal <as...@apache.org>
Committed: Tue Jul 28 16:23:46 2015 -0700

----------------------------------------------------------------------
 .../gemfire/cache/util/AutoBalancer.java        |  80 +++++++--
 .../cache/util/AutoBalancerJUnitTest.java       | 166 +++++++++++++++----
 2 files changed, 200 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1683361b/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java b/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
index 1de7031..ef795b0 100644
--- a/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
+++ b/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
@@ -1,6 +1,8 @@
 package com.gemstone.gemfire.cache.util;
 
 import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.CancellationException;
@@ -74,9 +76,9 @@ public class AutoBalancer implements Declarable {
   public static final String SCHEDULE = "schedule";
 
   /**
-   * Use this configuration to manage re-balance threshold. Rebalance operation
+   * Use this configuration to manage re-balance invocation. Rebalance operation
    * will be triggered if the total number of bytes rebalance operation may move
-   * is more than this threshold, percentage of the total data size.
+   * is more than this threshold, in percentage of the total data size.
    * <P>
    * Default {@value AutoBalancer#DEFAULT_SIZE_THRESHOLD_PERCENT}
    */
@@ -89,6 +91,26 @@ public class AutoBalancer implements Declarable {
   public static final int DEFAULT_SIZE_THRESHOLD_PERCENT = 10;
 
   /**
+   * In the initial data load phases,
+   * {@link AutoBalancer#SIZE_THRESHOLD_PERCENT} based rebalance invocation may
+   * be unnecessary. Rebalance should not be triggered if the total data size
+   * managed by cluster is too small. Rebalance operation will be triggered if
+   * the total number of bytes rebalance operation may move is more than this
+   * number of bytes.
+   * <P>
+   * Default {@value AutoBalancer#DEFAULT_SIZE_MINIMUM}
+   */
+  public static final String SIZE_MINIMUM = "size-minimum";
+
+  /**
+   * Default value of {@link AutoBalancer#SIZE_MINIMUM}. In the initial data
+   * load phases, {@link AutoBalancer#SIZE_THRESHOLD_PERCENT} based rebalance
+   * invocation may be unnecessary. Do not rebalance if the data to be moved is
+   * less than 100MB
+   */
+  public static final int DEFAULT_SIZE_MINIMUM = 100 * 1024 * 1024;
+
+  /**
    * Name of the DistributedLockService that {@link AutoBalancer} will use to
    * guard against concurrent maintenance activity
    */
@@ -193,6 +215,7 @@ public class AutoBalancer implements Declarable {
    */
   class SizeBasedOOBAuditor implements OOBAuditor {
     private int sizeThreshold = DEFAULT_SIZE_THRESHOLD_PERCENT;
+    private int sizeMinimum = DEFAULT_SIZE_MINIMUM;
 
     @Override
     public void init(Properties props) {
@@ -207,6 +230,12 @@ public class AutoBalancer implements Declarable {
             throw new GemFireConfigException(SIZE_THRESHOLD_PERCENT + " should be integer, 1 to 99");
           }
         }
+        if (props.getProperty(SIZE_MINIMUM) != null) {
+          sizeMinimum = Integer.valueOf(props.getProperty(SIZE_MINIMUM));
+          if (sizeMinimum <= 0) {
+            throw new GemFireConfigException(SIZE_MINIMUM + " should be greater than 0");
+          }
+        }
       }
     }
 
@@ -250,7 +279,12 @@ public class AutoBalancer implements Declarable {
     boolean needsRebalancing() {
       // test cluster level status
       long transferSize = cacheFacade.getTotalTransferSize();
-      long totalSize = cacheFacade.getTotalDataSize();
+      if (transferSize <= sizeMinimum) {
+        return false;
+      }
+
+      Map<PartitionedRegion, InternalPRInfo> details = cacheFacade.getRegionMemberDetails();
+      long totalSize = cacheFacade.getTotalDataSize(details);
 
       if (totalSize > 0) {
         int transferPercent = (int) ((100.0 * transferSize) / totalSize);
@@ -264,9 +298,13 @@ public class AutoBalancer implements Declarable {
       return false;
     }
 
-    public int getSizeThreshold() {
+    int getSizeThreshold() {
       return sizeThreshold;
     }
+
+    public long getSizeMinimum() {
+      return sizeMinimum;
+    }
   }
 
   /**
@@ -275,18 +313,30 @@ public class AutoBalancer implements Declarable {
    */
   static class GeodeCacheFacade implements CacheOperationFacade {
     @Override
-    public long getTotalDataSize() {
-      long totalSize = 0;
+    public Map<PartitionedRegion, InternalPRInfo> getRegionMemberDetails() {
       GemFireCacheImpl cache = getCache();
+      Map<PartitionedRegion, InternalPRInfo> detailsMap = new HashMap<>();
       for (PartitionedRegion region : cache.getPartitionedRegions()) {
         LoadProbe probe = cache.getResourceManager().getLoadProbe();
         InternalPRInfo info = region.getRedundancyProvider().buildPartitionedRegionInfo(true, probe);
-        Set<PartitionMemberInfo> membersInfo = info.getPartitionMemberInfo();
-        for (PartitionMemberInfo member : membersInfo) {
-          if (logger.isDebugEnabled()) {
-            logger.debug("Region:{}, Member: {}, Size: {}", region.getFullPath(), member, member.getSize());
+        detailsMap.put(region, info);
+      }
+      return detailsMap;
+    }
+
+    @Override
+    public long getTotalDataSize(Map<PartitionedRegion, InternalPRInfo> details) {
+      long totalSize = 0;
+      if (details != null) {
+        for (PartitionedRegion region : details.keySet()) {
+          InternalPRInfo info = details.get(region);
+          Set<PartitionMemberInfo> membersInfo = info.getPartitionMemberInfo();
+          for (PartitionMemberInfo member : membersInfo) {
+            if (logger.isDebugEnabled()) {
+              logger.debug("Region:{}, Member: {}, Size: {}", region.getFullPath(), member, member.getSize());
+            }
+            totalSize += member.getSize();
           }
-          totalSize += member.getSize();
         }
       }
       return totalSize;
@@ -417,7 +467,9 @@ public class AutoBalancer implements Declarable {
 
     void incrementAttemptCounter();
 
-    long getTotalDataSize();
+    Map<PartitionedRegion, InternalPRInfo> getRegionMemberDetails();
+
+    long getTotalDataSize(Map<PartitionedRegion, InternalPRInfo> details);
 
     long getTotalTransferSize();
   }
@@ -456,4 +508,8 @@ public class AutoBalancer implements Declarable {
   public void setCacheOperationFacade(CacheOperationFacade facade) {
     this.cacheFacade = facade;
   }
+
+  public CacheOperationFacade getCacheOperationFacade() {
+    return this.cacheFacade;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1683361b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
index 888ea20..db225cb 100644
--- a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
+++ b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
@@ -2,11 +2,14 @@ package com.gemstone.gemfire.cache.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.ByteArrayInputStream;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -47,10 +50,6 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class AutoBalancerJUnitTest {
-
-  // OOB > threshold && size < min
-  // OOB > threshold && size < min
-  // OOB critical nodes
   GemFireCacheImpl cache;
   Mockery mockContext;
 
@@ -355,19 +354,20 @@ public class AutoBalancerJUnitTest {
   public void testOOBWhenBelowSizeThreshold() {
     final long totalSize = 1000L;
 
+    final Map<PartitionedRegion, InternalPRInfo> details = new HashMap<>();
     final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
     mockContext.checking(new Expectations() {
       {
+        allowing(mockCacheFacade).getRegionMemberDetails();
+        will(returnValue(details));
         // first run
-        oneOf(mockCacheFacade).getTotalDataSize();
+        oneOf(mockCacheFacade).getTotalDataSize(details);
         will(returnValue(totalSize));
         oneOf(mockCacheFacade).getTotalTransferSize();
         // half of threshold limit
         will(returnValue((AutoBalancer.DEFAULT_SIZE_THRESHOLD_PERCENT * totalSize / 100) / 2));
 
         // second run
-        oneOf(mockCacheFacade).getTotalDataSize();
-        will(returnValue(totalSize));
         oneOf(mockCacheFacade).getTotalTransferSize();
         // nothing to transfer
         will(returnValue(0L));
@@ -376,7 +376,9 @@ public class AutoBalancerJUnitTest {
 
     AutoBalancer balancer = new AutoBalancer();
     balancer.setCacheOperationFacade(mockCacheFacade);
-    balancer.init(getBasicConfig());
+    Properties config = getBasicConfig();
+    config.put(AutoBalancer.SIZE_MINIMUM, "10");
+    balancer.init(config);
     SizeBasedOOBAuditor auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
 
     // first run
@@ -387,22 +389,18 @@ public class AutoBalancerJUnitTest {
   }
 
   @Test
-  public void testOOBWhenBelowAboveThreshold() {
+  public void testOOBWhenAboveThresholdButBelowMin() {
     final long totalSize = 1000L;
 
     final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
     mockContext.checking(new Expectations() {
       {
         // first run
-        oneOf(mockCacheFacade).getTotalDataSize();
-        will(returnValue(totalSize));
         oneOf(mockCacheFacade).getTotalTransferSize();
         // twice threshold
         will(returnValue((AutoBalancer.DEFAULT_SIZE_THRESHOLD_PERCENT * totalSize / 100) * 2));
 
         // second run
-        oneOf(mockCacheFacade).getTotalDataSize();
-        will(returnValue(totalSize));
         oneOf(mockCacheFacade).getTotalTransferSize();
         // more than total size
         will(returnValue(2 * totalSize));
@@ -411,17 +409,60 @@ public class AutoBalancerJUnitTest {
 
     AutoBalancer balancer = new AutoBalancer();
     balancer.setCacheOperationFacade(mockCacheFacade);
-    balancer.init(getBasicConfig());
+    Properties config = getBasicConfig();
+    config.put(AutoBalancer.SIZE_MINIMUM, "" + (totalSize * 5));
+    balancer.init(config);
     SizeBasedOOBAuditor auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
 
     // first run
-    assertTrue(auditor.needsRebalancing());
+    assertFalse(auditor.needsRebalancing());
 
     // second run
-    assertTrue(auditor.needsRebalancing());
+    assertFalse(auditor.needsRebalancing());
   }
 
   @Test
+  public void testOOBWhenAboveThresholdAndMin() {
+    final long totalSize = 1000L;
+    
+    final Map<PartitionedRegion, InternalPRInfo> details = new HashMap<>();
+    final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
+    mockContext.checking(new Expectations() {
+      {
+        allowing(mockCacheFacade).getRegionMemberDetails();
+        will(returnValue(details));
+        
+        // first run
+        oneOf(mockCacheFacade).getTotalDataSize(details);
+        will(returnValue(totalSize));
+        oneOf(mockCacheFacade).getTotalTransferSize();
+        // twice threshold
+        will(returnValue((AutoBalancer.DEFAULT_SIZE_THRESHOLD_PERCENT * totalSize / 100) * 2));
+        
+        // second run
+        oneOf(mockCacheFacade).getTotalDataSize(details);
+        will(returnValue(totalSize));
+        oneOf(mockCacheFacade).getTotalTransferSize();
+        // more than total size
+        will(returnValue(2 * totalSize));
+      }
+    });
+    
+    AutoBalancer balancer = new AutoBalancer();
+    balancer.setCacheOperationFacade(mockCacheFacade);
+    Properties config = getBasicConfig();
+    config.put(AutoBalancer.SIZE_MINIMUM, "10");
+    balancer.init(config);
+    SizeBasedOOBAuditor auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
+    
+    // first run
+    assertTrue(auditor.needsRebalancing());
+    
+    // second run
+    assertTrue(auditor.needsRebalancing());
+  }
+  
+  @Test
   public void testInitializerCacheXML() {
     String configStr = "<cache xmlns=\"http://schema.pivotal.io/gemfire/cache\"                          "
         + " xmlns:xsi=\"http://www.w3.org/2001/XMLSchema-instance\"                                      "
@@ -502,17 +543,20 @@ public class AutoBalancerJUnitTest {
     balancer.init(getBasicConfig());
     SizeBasedOOBAuditor auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
     assertEquals(AutoBalancer.DEFAULT_SIZE_THRESHOLD_PERCENT, auditor.getSizeThreshold());
+    assertEquals(AutoBalancer.DEFAULT_SIZE_MINIMUM, auditor.getSizeMinimum());
 
     Properties props = getBasicConfig();
     props.put(AutoBalancer.SIZE_THRESHOLD_PERCENT, "17");
+    props.put(AutoBalancer.SIZE_MINIMUM, "10");
     balancer = new AutoBalancer();
     balancer.init(props);
     auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
     assertEquals(17, auditor.getSizeThreshold());
+    assertEquals(10, auditor.getSizeMinimum());
   }
 
   @Test(expected = GemFireConfigException.class)
-  public void testSizeThresholdNegative() {
+  public void testConfigTransferThresholdNegative() {
     AutoBalancer balancer = new AutoBalancer();
     Properties props = getBasicConfig();
     props.put(AutoBalancer.SIZE_THRESHOLD_PERCENT, "-1");
@@ -520,7 +564,15 @@ public class AutoBalancerJUnitTest {
   }
 
   @Test(expected = GemFireConfigException.class)
-  public void testSizeThresholdZero() {
+  public void testConfigSizeMinNegative() {
+    AutoBalancer balancer = new AutoBalancer();
+    Properties props = getBasicConfig();
+    props.put(AutoBalancer.SIZE_MINIMUM, "-1");
+    balancer.init(props);
+  }
+
+  @Test(expected = GemFireConfigException.class)
+  public void testConfigTransferThresholdZero() {
     AutoBalancer balancer = new AutoBalancer();
     Properties props = getBasicConfig();
     props.put(AutoBalancer.SIZE_THRESHOLD_PERCENT, "0");
@@ -528,7 +580,7 @@ public class AutoBalancerJUnitTest {
   }
 
   @Test(expected = GemFireConfigException.class)
-  public void testSizeThresholdToohigh() {
+  public void testConfigTransferThresholdTooHigh() {
     AutoBalancer balancer = new AutoBalancer();
     Properties props = getBasicConfig();
     props.put(AutoBalancer.SIZE_THRESHOLD_PERCENT, "100");
@@ -623,6 +675,13 @@ public class AutoBalancerJUnitTest {
 
   @Test
   public void testFacadeTotalBytesNoRegion() {
+    CacheOperationFacade facade = new AutoBalancer().getCacheOperationFacade();
+
+    assertEquals(0, facade.getTotalDataSize(new HashMap<PartitionedRegion, InternalPRInfo>()));
+  }
+
+  @Test
+  public void testFacadeCollectMemberDetailsNoRegion() {
     final GemFireCacheImpl mockCache = mockContext.mock(GemFireCacheImpl.class);
     mockContext.checking(new Expectations() {
       {
@@ -638,11 +697,11 @@ public class AutoBalancerJUnitTest {
       }
     };
 
-    assertEquals(0, facade.getTotalDataSize());
+    assertEquals(0, facade.getRegionMemberDetails().size());
   }
 
   @Test
-  public void testFacadeTotalBytes2Regions() {
+  public void testFacadeCollectMemberDetails2Regions() {
     cache = createBasicCache();
 
     final GemFireCacheImpl mockCache = mockContext.mock(GemFireCacheImpl.class);
@@ -655,17 +714,9 @@ public class AutoBalancerJUnitTest {
 
     final PRHARedundancyProvider mockRedundancyProviderR1 = mockContext.mock(PRHARedundancyProvider.class, "prhaR1");
     final InternalPRInfo mockR1PRInfo = mockContext.mock(InternalPRInfo.class, "prInforR1");
-    final PartitionMemberInfo mockR1M1Info = mockContext.mock(PartitionMemberInfo.class, "r1M1");
-    final PartitionMemberInfo mockR1M2Info = mockContext.mock(PartitionMemberInfo.class, "r1M2");
-    final HashSet<PartitionMemberInfo> r1Members = new HashSet<>();
-    r1Members.add(mockR1M1Info);
-    r1Members.add(mockR1M2Info);
 
     final PRHARedundancyProvider mockRedundancyProviderR2 = mockContext.mock(PRHARedundancyProvider.class, "prhaR2");
     final InternalPRInfo mockR2PRInfo = mockContext.mock(InternalPRInfo.class, "prInforR2");
-    final PartitionMemberInfo mockR2M1Info = mockContext.mock(PartitionMemberInfo.class, "r2M1");
-    final HashSet<PartitionMemberInfo> r2Members = new HashSet<>();
-    r2Members.add(mockR2M1Info);
 
     mockContext.checking(new Expectations() {
       {
@@ -682,6 +733,55 @@ public class AutoBalancerJUnitTest {
 
         oneOf(mockRedundancyProviderR1).buildPartitionedRegionInfo(with(true), with(any(LoadProbe.class)));
         will(returnValue(mockR1PRInfo));
+
+        oneOf(mockRedundancyProviderR2).buildPartitionedRegionInfo(with(true), with(any(LoadProbe.class)));
+        will(returnValue(mockR2PRInfo));
+      }
+    });
+
+    GeodeCacheFacade facade = new GeodeCacheFacade() {
+      @Override
+      GemFireCacheImpl getCache() {
+        return mockCache;
+      }
+    };
+
+    Map<PartitionedRegion, InternalPRInfo> map = facade.getRegionMemberDetails();
+    assertNotNull(map);
+    assertEquals(2, map.size());
+    assertEquals(map.get(mockR1), mockR1PRInfo);
+    assertEquals(map.get(mockR2), mockR2PRInfo);
+  }
+
+  @Test
+  public void testFacadeTotalBytes2Regions() {
+    final PartitionedRegion mockR1 = mockContext.mock(PartitionedRegion.class, "r1");
+    final PartitionedRegion mockR2 = mockContext.mock(PartitionedRegion.class, "r2");
+    final HashSet<PartitionedRegion> regions = new HashSet<>();
+    regions.add(mockR1);
+    regions.add(mockR2);
+
+    final InternalPRInfo mockR1PRInfo = mockContext.mock(InternalPRInfo.class, "prInforR1");
+    final PartitionMemberInfo mockR1M1Info = mockContext.mock(PartitionMemberInfo.class, "r1M1");
+    final PartitionMemberInfo mockR1M2Info = mockContext.mock(PartitionMemberInfo.class, "r1M2");
+    final HashSet<PartitionMemberInfo> r1Members = new HashSet<>();
+    r1Members.add(mockR1M1Info);
+    r1Members.add(mockR1M2Info);
+
+    final InternalPRInfo mockR2PRInfo = mockContext.mock(InternalPRInfo.class, "prInforR2");
+    final PartitionMemberInfo mockR2M1Info = mockContext.mock(PartitionMemberInfo.class, "r2M1");
+    final HashSet<PartitionMemberInfo> r2Members = new HashSet<>();
+    r2Members.add(mockR2M1Info);
+
+    final Map<PartitionedRegion, InternalPRInfo> details = new HashMap<>();
+    details.put(mockR1, mockR1PRInfo);
+    details.put(mockR2, mockR2PRInfo);
+
+    mockContext.checking(new Expectations() {
+      {
+        allowing(mockR1).getFullPath();
+        allowing(mockR2).getFullPath();
+
         oneOf(mockR1PRInfo).getPartitionMemberInfo();
         will(returnValue(r1Members));
         atLeast(1).of(mockR1M1Info).getSize();
@@ -689,8 +789,6 @@ public class AutoBalancerJUnitTest {
         atLeast(1).of(mockR1M2Info).getSize();
         will(returnValue(74L));
 
-        oneOf(mockRedundancyProviderR2).buildPartitionedRegionInfo(with(true), with(any(LoadProbe.class)));
-        will(returnValue(mockR2PRInfo));
         oneOf(mockR2PRInfo).getPartitionMemberInfo();
         will(returnValue(r2Members));
         atLeast(1).of(mockR2M1Info).getSize();
@@ -700,12 +798,12 @@ public class AutoBalancerJUnitTest {
 
     GeodeCacheFacade facade = new GeodeCacheFacade() {
       @Override
-      GemFireCacheImpl getCache() {
-        return mockCache;
+      public Map<PartitionedRegion, InternalPRInfo> getRegionMemberDetails() {
+        return details;
       }
     };
 
-    assertEquals(123 + 74 + 3475, facade.getTotalDataSize());
+    assertEquals(123 + 74 + 3475, facade.getTotalDataSize(details));
   }
 
   private Properties getBasicConfig() {


[31/50] [abbrv] incubator-geode git commit: Fix for GEODE-109

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/test/java/com/gemstone/gemfire/redis/SortedSetsJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/redis/SortedSetsJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/redis/SortedSetsJUnitTest.java
new file mode 100755
index 0000000..4adfe07
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/redis/SortedSetsJUnitTest.java
@@ -0,0 +1,414 @@
+package com.gemstone.gemfire.redis;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+import java.util.Set;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import redis.clients.jedis.Jedis;
+import redis.clients.jedis.Tuple;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class SortedSetsJUnitTest {
+  private static Jedis jedis;
+  private static GemFireRedisServer server;
+  private static GemFireCache cache;
+  private static Random rand;
+  private static int port = 6379;
+
+  @BeforeClass
+  public static void setUp() throws IOException {
+    rand = new Random();
+    CacheFactory cf = new CacheFactory();
+    //cf.set("log-file", "redis.log");
+    cf.set("log-level", "error");
+    cf.set("mcast-port", "0");
+    cf.set("locators", "");
+    cache = cf.create();
+    port = AvailablePortHelper.getRandomAvailableTCPPort();
+    server = new GemFireRedisServer("localhost", port);
+
+    server.start();
+    jedis = new Jedis("localhost", port, 10000000);
+  }
+
+  @Test
+  public void testZAddZRange() {
+    int numMembers = 10;
+    String key = randString();
+    Map<String, Double> scoreMembers = new HashMap<String, Double>();
+
+    for (int i = 0; i < numMembers; i++)
+      scoreMembers.put(randString(), rand.nextDouble());
+
+    jedis.zadd(key, scoreMembers);
+    int k = 0;
+    for (String entry: scoreMembers.keySet())
+      assertNotNull(jedis.zscore(key, entry));
+
+    Set<Tuple> results = jedis.zrangeWithScores(key, 0, -1);
+    Map<String, Double> resultMap = new HashMap<String, Double>();
+    for (Tuple t: results) {
+      resultMap.put(t.getElement(), t.getScore());
+    }
+
+    assertEquals(scoreMembers, resultMap);
+
+    for (int i = 0; i < 10; i++) {
+      int start;
+      int stop;
+      do {
+        start = rand.nextInt(numMembers);
+        stop = rand.nextInt(numMembers);
+      } while (start > stop);
+      results = jedis.zrangeWithScores(key, start, stop);
+      List<Entry<String, Double>> resultList = new ArrayList<Entry<String, Double>>();
+      for (Tuple t: results)
+        resultList.add(new AbstractMap.SimpleEntry<String, Double>(t.getElement(), t.getScore()));
+      List<Entry<String, Double>> list = new ArrayList<Entry<String, Double>>(scoreMembers.entrySet());
+      Collections.sort(list, new EntryCmp());
+      list = list.subList(start, stop + 1);
+      assertEquals(list, resultList);
+    }
+  }
+
+  @Test
+  public void testZRevRange() {
+    int numMembers = 10;
+    String key = randString();
+
+    Map<String, Double> scoreMembers = new HashMap<String, Double>();
+
+    for (int i = 0; i < numMembers; i++)
+      scoreMembers.put(randString(), rand.nextDouble());
+
+    jedis.zadd(key, scoreMembers);
+
+    Set<Tuple> results;
+
+    for (int i = 0; i < 10; i++) {
+      int start;
+      int stop;
+      do {
+        start = rand.nextInt(numMembers);
+        stop = rand.nextInt(numMembers);
+      } while (start > stop);
+      results = jedis.zrevrangeWithScores(key, start, stop);
+      List<Entry<String, Double>> resultList = new ArrayList<Entry<String, Double>>();
+      for (Tuple t: results)
+        resultList.add(new AbstractMap.SimpleEntry<String, Double>(t.getElement(), t.getScore()));
+      List<Entry<String, Double>> list = new ArrayList<Entry<String, Double>>(scoreMembers.entrySet());
+      Collections.sort(list, new EntryRevCmp());
+      list = list.subList(start, stop + 1);
+      assertEquals(list, resultList);
+    }
+  }
+
+  @Test
+  public void testZCount() {
+    int num = 10;
+    int runs = 2;
+    for (int i = 0; i < runs; i++) {
+      Double min;
+      Double max;
+      do {
+        min = rand.nextDouble();
+        max = rand.nextDouble();
+      } while (min > max);
+
+
+      int count = 0;
+
+      String key = randString();
+      Map<String, Double> scoreMembers = new HashMap<String, Double>();
+
+      for (int j = 0; j < num; j++) {
+        Double nextDouble = rand.nextDouble();
+        if (nextDouble >= min && nextDouble <= max)
+          count++;
+        scoreMembers.put(randString(), nextDouble);
+      }
+
+      jedis.zadd(key, scoreMembers);
+      Long countResult = jedis.zcount(key, min, max);
+      assertTrue(count == countResult);
+    }
+
+  }
+
+  @Test
+  public void testZIncrBy() {
+    String key = randString();
+    String member = randString();
+    Double score = 0.0;
+    for (int i = 0; i < 20; i++) {
+      Double incr = rand.nextDouble();
+      Double result = jedis.zincrby(key, incr, member);
+      score += incr;
+      assertEquals(score, result, 1.0/100000000.0);
+    }
+
+
+    jedis.zincrby(key, Double.MAX_VALUE, member);
+    Double infResult = jedis.zincrby(key, Double.MAX_VALUE, member);
+
+
+    assertEquals(infResult, Double.valueOf(Double.POSITIVE_INFINITY));
+  }
+
+  public void testZRangeByScore() {
+    Double min;
+    Double max;
+    for (int j = 0; j < 2; j++) {
+      do {
+        min = rand.nextDouble();
+        max = rand.nextDouble();
+      } while (min > max);
+      int numMembers = 500;
+      String key = randString();
+      Map<String, Double> scoreMembers = new HashMap<String, Double>();
+      List<Entry<String, Double>> expected = new ArrayList<Entry<String, Double>>();
+      for (int i = 0; i < numMembers; i++) {
+        String s = randString();
+        Double d = rand.nextDouble();
+        scoreMembers.put(s, d);
+        if (d > min && d < max)
+          expected.add(new AbstractMap.SimpleEntry<String, Double>(s, d));
+      }
+      jedis.zadd(key, scoreMembers);
+      Set<Tuple> results = jedis.zrangeByScoreWithScores(key, min, max);
+      List<Entry<String, Double>> resultList = new ArrayList<Entry<String, Double>>();
+      for (Tuple t: results)
+        resultList.add(new AbstractMap.SimpleEntry<String, Double>(t.getElement(), t.getScore()));
+      Collections.sort(expected, new EntryCmp());
+
+      assertEquals(expected, resultList);
+      jedis.del(key);
+    }
+  }
+
+  public void testZRevRangeByScore() {
+    Double min;
+    Double max;
+    for (int j = 0; j < 2; j++) {
+      do {
+        min = rand.nextDouble();
+        max = rand.nextDouble();
+      } while (min > max);
+      int numMembers = 500;
+      String key = randString();
+      Map<String, Double> scoreMembers = new HashMap<String, Double>();
+      List<Entry<String, Double>> expected = new ArrayList<Entry<String, Double>>();
+      for (int i = 0; i < numMembers; i++) {
+        String s = randString();
+        Double d = rand.nextDouble();
+        scoreMembers.put(s, d);
+        if (d > min && d < max)
+          expected.add(new AbstractMap.SimpleEntry<String, Double>(s, d));
+      }
+      jedis.zadd(key, scoreMembers);
+      Set<Tuple> results = jedis.zrevrangeByScoreWithScores(key, max, min);
+      List<Entry<String, Double>> resultList = new ArrayList<Entry<String, Double>>();
+      for (Tuple t: results)
+        resultList.add(new AbstractMap.SimpleEntry<String, Double>(t.getElement(), t.getScore()));
+      Collections.sort(expected, new EntryRevCmp());
+
+      assertEquals(expected, resultList);
+      jedis.del(key);
+    }
+  }
+
+  @Test
+  public void testZRemZScore() {
+    Double min;
+    Double max;
+    for (int j = 0; j < 2; j++) {
+      do {
+        min = rand.nextDouble();
+        max = rand.nextDouble();
+      } while (min > max);
+      int numMembers = 5000;
+      String key = randString();
+      Map<String, Double> scoreMembers = new HashMap<String, Double>();
+      List<Entry<String, Double>> expected = new ArrayList<Entry<String, Double>>();
+      for (int i = 0; i < numMembers; i++) {
+        String s = randString();
+        Double d = rand.nextDouble();
+        scoreMembers.put(s, d);
+        if (d > min && d < max)
+          expected.add(new AbstractMap.SimpleEntry<String, Double>(s, d));
+      }
+      jedis.zadd(key, scoreMembers);
+      Collections.sort(expected, new EntryCmp());
+      for (int i = expected.size(); i <= 0; i--) {
+        Entry<String, Double> remEntry = expected.remove(i);
+        String rem = remEntry.getKey();
+        Double val = remEntry.getValue();
+        assertEquals(val, jedis.zscore(key, rem));
+
+        assertTrue(jedis.zrem(key, rem) == 1);
+      }
+      String s = randString();
+      if (!expected.contains(s))
+        assertTrue(jedis.zrem(key, s) == 0);
+      jedis.del(key);
+    }
+  }
+
+  @Test
+  public void testZRank() {
+    for (int j = 0; j < 2; j++) {
+      int numMembers = 10;
+      String key = randString();
+      Map<String, Double> scoreMembers = new HashMap<String, Double>();
+      List<Entry<String, Double>> expected = new ArrayList<Entry<String, Double>>();
+      for (int i = 0; i < numMembers; i++) {
+        String s = randString();
+        Double d = rand.nextDouble();
+        scoreMembers.put(s, d);
+        expected.add(new AbstractMap.SimpleEntry<String, Double>(s, d));
+      }
+      Collections.sort(expected, new EntryCmp());
+      jedis.zadd(key, scoreMembers);
+      for (int i = 0; i < expected.size(); i++) {
+        Entry<String, Double> en = expected.get(i);
+        String field = en.getKey();
+        Long rank = jedis.zrank(key, field);
+        assertEquals(new Long(i), rank);
+      }
+      String field = randString();
+      if (!expected.contains(field))
+        assertNull(jedis.zrank(key, field));
+      jedis.del(key);
+    }
+  }
+
+  @Test
+  public void testZRevRank() {
+    for (int j = 0; j < 2; j++) {
+      int numMembers = 10;
+      String key = randString();
+      Map<String, Double> scoreMembers = new HashMap<String, Double>();
+      List<Entry<String, Double>> expected = new ArrayList<Entry<String, Double>>();
+      for (int i = 0; i < numMembers; i++) {
+        String s = randString();
+        Double d = rand.nextDouble();
+        scoreMembers.put(s, d);
+        expected.add(new AbstractMap.SimpleEntry<String, Double>(s, d));
+      }
+      Collections.sort(expected, new EntryRevCmp());
+      jedis.zadd(key, scoreMembers);
+      for (int i = 0; i < expected.size(); i++) {
+        Entry<String, Double> en = expected.get(i);
+        String field = en.getKey();
+        Long rank = jedis.zrevrank(key, field);
+        assertEquals(new Long(i), rank);
+      }
+      String field = randString();
+      if (!expected.contains(field))
+        assertNull(jedis.zrank(key, field));
+      jedis.del(key);
+    }
+  }
+
+  private class EntryCmp implements Comparator<Entry<String, Double>> {
+
+    @Override
+    public int compare(Entry<String, Double> o1, Entry<String, Double> o2) {
+      Double diff = o1.getValue() - o2.getValue();
+      if (diff == 0) 
+        return o2.getKey().compareTo(o1.getKey());
+      else
+        return diff > 0 ? 1 : -1;
+    }
+
+  }
+
+  private class EntryRevCmp implements Comparator<Entry<String, Double>> {
+
+    @Override
+    public int compare(Entry<String, Double> o1, Entry<String, Double> o2) {
+      Double diff = o2.getValue() - o1.getValue();
+      if (diff == 0) 
+        return o1.getKey().compareTo(o2.getKey());
+      else
+        return diff > 0 ? 1 : -1;
+    }
+
+  }
+
+  @Test
+  public void testZRemRangeByScore() {
+    Double min;
+    Double max;
+    for (int j = 0; j < 3; j++) {
+      do {
+        min = rand.nextDouble();
+        max = rand.nextDouble();
+      } while (min > max);
+      int numMembers = 10;
+      String key = randString();
+      Map<String, Double> scoreMembers = new HashMap<String, Double>();
+      List<Entry<String, Double>> fullList = new ArrayList<Entry<String, Double>>();
+      List<Entry<String, Double>> toRemoveList = new ArrayList<Entry<String, Double>>();
+      for (int i = 0; i < numMembers; i++) {
+        String s = randString();
+        Double d = rand.nextDouble();
+        scoreMembers.put(s, d);
+        fullList.add(new AbstractMap.SimpleEntry<String, Double>(s, d));
+        if (d > min && d < max)
+          toRemoveList.add(new AbstractMap.SimpleEntry<String, Double>(s, d));
+      }
+      jedis.zadd(key, scoreMembers);
+      Long numRemoved = jedis.zremrangeByScore(key, min, max);
+      List<Entry<String, Double>> expectedList = new ArrayList<Entry<String, Double>>(fullList);
+      expectedList.removeAll(toRemoveList);
+      Collections.sort(expectedList, new EntryCmp());
+      Set<Tuple> result = jedis.zrangeWithScores(key, 0, -1);
+      List<Entry<String, Double>> resultList = new ArrayList<Entry<String, Double>>();
+      for (Tuple t: result)
+        resultList.add(new AbstractMap.SimpleEntry<String, Double>(t.getElement(), t.getScore()));
+      assertEquals(expectedList, resultList);
+      jedis.del(key);
+    }
+  }
+
+  private String randString() {
+    return Long.toHexString(Double.doubleToLongBits(Math.random()));
+  }
+
+  @After
+  public void flushAll() {
+    jedis.flushAll();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    jedis.close();
+    cache.close();
+    server.shutdown();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/test/java/com/gemstone/gemfire/redis/StringsJunitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/redis/StringsJunitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/redis/StringsJunitTest.java
new file mode 100755
index 0000000..55ba061
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/redis/StringsJunitTest.java
@@ -0,0 +1,296 @@
+package com.gemstone.gemfire.redis;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import redis.clients.jedis.Jedis;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class StringsJunitTest {
+
+  private static Jedis jedis;
+  private static GemFireRedisServer server;
+  private static GemFireCache cache;
+  private static Random rand;
+  private static int port = 6379;
+
+  @BeforeClass
+  public static void setUp() throws IOException {
+    rand = new Random();
+    CacheFactory cf = new CacheFactory();
+    //cf.set("log-file", "redis.log");
+    cf.set("log-level", "error");
+    cf.set("mcast-port", "0");
+    cf.set("locators", "");
+    cache = cf.create();
+    port = AvailablePortHelper.getRandomAvailableTCPPort();
+    server = new GemFireRedisServer("localhost", port);
+
+    server.start();
+    jedis = new Jedis("localhost", port, 10000000);
+  }
+
+  @Test
+  public void testAppendAndStrlen() {
+    String key = randString();
+    int len = key.length();
+    String full = key;
+    jedis.set(key, key);
+    for (int i = 0; i < 15; i++) {
+      String rand = randString();
+      jedis.append(key, rand);
+      len += rand.length();
+      full += rand;
+    }
+    String ret = jedis.get(key);
+    assertTrue(ret.length() == len);
+    assertTrue(full.equals(ret));
+    assertTrue(full.length() == jedis.strlen(key));
+  }
+
+  @Test
+  public void testDecr() {
+    String key1 = randString();
+    String key2 = randString();
+    String key3 = randString();
+    int num1 = 100;
+    int num2 = -100;
+    jedis.set(key1, ""+num1);
+    //jedis.set(key3, "-100");
+    jedis.set(key2, ""+num2);
+
+    jedis.decr(key1);
+    jedis.decr(key3);
+    jedis.decr(key2);
+    assertTrue(jedis.get(key1).equals("" + (num1 - 1)));
+    assertTrue(jedis.get(key2).equals("" + (num2 - 1)));
+    assertTrue(jedis.get(key3).equals("" + (-1)));
+  }
+
+  @Test
+  public void testIncr() {
+    String key1 = randString();
+    String key2 = randString();
+    String key3 = randString();
+    int num1 = 100;
+    int num2 = -100;
+    jedis.set(key1, ""+num1);
+    //jedis.set(key3, "-100");
+    jedis.set(key2, ""+num2);
+
+    jedis.incr(key1);
+    jedis.incr(key3);
+    jedis.incr(key2);
+
+    assertTrue(jedis.get(key1).equals("" + (num1 + 1)));
+    assertTrue(jedis.get(key2).equals("" + (num2 + 1)));
+    assertTrue(jedis.get(key3).equals("" + (+1)));
+  }
+
+  @Test
+  public void testDecrBy() {
+    String key1 = randString();
+    String key2 = randString();
+    String key3 = randString();
+    int decr1 = rand.nextInt(100);
+    int decr2 = rand.nextInt(100);
+    Long decr3 = Long.MAX_VALUE/2;
+    int num1 = 100;
+    int num2 = -100;
+    jedis.set(key1, ""+num1);
+    jedis.set(key2, ""+num2);
+    jedis.set(key3, ""+Long.MIN_VALUE);
+
+    jedis.decrBy(key1, decr1);
+    jedis.decrBy(key2, decr2);
+
+    assertTrue(jedis.get(key1).equals("" + (num1 - decr1*1)));
+    assertTrue(jedis.get(key2).equals("" + (num2 - decr2*1)));
+
+    Exception ex= null;
+    try {
+      jedis.decrBy(key3, decr3);
+    } catch(Exception e) {
+      ex = e;
+    }
+    assertNotNull(ex);
+
+  }  
+
+  @Test
+  public void testIncrBy() {
+    String key1 = randString();
+    String key2 = randString();
+    String key3 = randString();
+    int incr1 = rand.nextInt(100);
+    int incr2 = rand.nextInt(100);
+    Long incr3 = Long.MAX_VALUE/2;
+    int num1 = 100;
+    int num2 = -100;
+    jedis.set(key1, ""+num1);
+    jedis.set(key2, ""+num2);
+    jedis.set(key3, ""+Long.MAX_VALUE);
+
+    jedis.incrBy(key1, incr1);
+    jedis.incrBy(key2, incr2);
+    assertTrue(jedis.get(key1).equals("" + (num1 + incr1*1)));
+    assertTrue(jedis.get(key2).equals("" + (num2 + incr2*1)));
+
+    Exception ex= null;
+    try {
+      jedis.incrBy(key3, incr3);
+    } catch(Exception e) {
+      ex = e;
+    }
+    assertNotNull(ex);
+  }
+
+  @Test
+  public void testGetRange() {
+    String sent = randString();
+    String contents = randString();
+    jedis.set(sent, contents);
+    for (int i = 0; i < sent.length(); i++) {
+      String range = jedis.getrange(sent, i, -1);
+      assertTrue(contents.substring(i).equals(range));
+    }
+    assertNull(jedis.getrange(sent, 2,0));
+  }
+
+  @Test
+  public void testGetSet() {
+    String key = randString();
+    String contents = randString();
+    jedis.set(key, contents);
+    String newContents = randString();
+    String oldContents = jedis.getSet(key, newContents);
+    assertTrue(oldContents.equals(contents));
+    contents = newContents;
+  }
+
+  @Test
+  public void testMSetAndGet() {
+    int r = 5;
+    String[] keyvals = new String[(r*2)];
+    String[] keys = new String[r];
+    String[] vals = new String[r];
+    for(int i = 0; i < r; i++) {
+      String key = randString();
+      String val = randString();
+      keyvals[2*i] = key;
+      keyvals[2*i+1] = val;
+      keys[i] = key;
+      vals[i] = val;
+    }
+
+    jedis.mset(keyvals);
+
+    List<String> ret = jedis.mget(keys);
+    Object[] retArray =  ret.toArray();
+
+    assertTrue(Arrays.equals(vals, retArray));
+  }
+
+  @Test
+  public void testMSetNX() {
+    Set<String> strings = new HashSet<String>();
+    for(int i = 0; i < 2 * 5; i++)
+      strings.add(randString());
+    String[] array = strings.toArray(new String[0]);
+    long response = jedis.msetnx(array);
+
+    assertTrue(response == 1);
+
+    long response2 = jedis.msetnx(array[0], randString());
+
+    assertTrue(response2 == 0);
+    assertEquals(array[1], jedis.get(array[0]));
+  }
+
+  @Test
+  public void testSetNX() {
+    String key1 = randString();
+    String key2;
+    do {
+      key2 = randString();
+    } while (key2.equals(key1));
+
+    long response1 = jedis.setnx(key1, key1);
+    long response2 = jedis.setnx(key2, key2);
+    long response3 = jedis.setnx(key1, key2);
+
+    assertTrue(response1 == 1);
+    assertTrue(response2 == 1);
+    assertTrue(response3 == 0);
+  }
+
+  @Test
+  public void testPAndSetex() {
+    Random r = new Random();
+    int setex = r.nextInt(5);
+    if (setex == 0)
+      setex = 1;
+    String key = randString();
+    jedis.setex(key, setex, randString());
+    try {
+      Thread.sleep((setex  + 5) * 1000);
+    } catch (InterruptedException e) {
+      return;
+    }
+    String result = jedis.get(key);
+    //System.out.println(result);
+    assertNull(result);
+
+    int psetex = r.nextInt(5000);
+    if (psetex == 0)
+      psetex = 1;
+    key = randString();
+    jedis.psetex(key, psetex, randString());
+    long start = System.currentTimeMillis();
+    try {
+      Thread.sleep(psetex + 5000);
+    } catch (InterruptedException e) {
+      return;
+    }
+    long stop = System.currentTimeMillis();
+    result = jedis.get(key);
+    assertTrue(stop - start >= psetex);
+    assertNull(result);
+  }
+
+  private String randString() {
+    return Long.toHexString(Double.doubleToLongBits(Math.random()));
+  }
+
+  @After
+  public void flushAll() {
+    jedis.flushAll();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    jedis.close();
+    cache.close();
+    server.shutdown();
+  }
+}


[16/50] [abbrv] incubator-geode git commit: GEODE-124: Rename size-minimum to minimum-size

Posted by bs...@apache.org.
GEODE-124: Rename size-minimum to minimum-size

Mimimum-size is easier to read and represents the property better.


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

Branch: refs/heads/feature/GEODE-77
Commit: 0ff54f3e85f078e6c775e36192a5863b18c45f81
Parents: 856fa0c
Author: Ashvin Agrawal <as...@apache.org>
Authored: Mon Aug 3 13:38:29 2015 -0700
Committer: Ashvin Agrawal <as...@apache.org>
Committed: Mon Aug 3 14:02:50 2015 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/cache/util/AutoBalancer.java   | 16 ++++++++--------
 .../gemfire/cache/util/AutoBalancerJUnitTest.java   | 12 ++++++------
 2 files changed, 14 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0ff54f3e/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java b/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
index 72a2f95..00ebc5f 100644
--- a/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
+++ b/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
@@ -99,17 +99,17 @@ public class AutoBalancer implements Declarable {
    * the total number of bytes rebalance operation may move is more than this
    * number of bytes.
    * <P>
-   * Default {@value AutoBalancer#DEFAULT_SIZE_MINIMUM}
+   * Default {@value AutoBalancer#DEFAULT_MINIMUM_SIZE}
    */
-  public static final String SIZE_MINIMUM = "size-minimum";
+  public static final String MINIMUM_SIZE = "minimum-size";
 
   /**
-   * Default value of {@link AutoBalancer#SIZE_MINIMUM}. In the initial data
+   * Default value of {@link AutoBalancer#MINIMUM_SIZE}. In the initial data
    * load phases, {@link AutoBalancer#SIZE_THRESHOLD_PERCENT} based rebalance
    * invocation may be unnecessary. Do not rebalance if the data to be moved is
    * less than 100MB
    */
-  public static final int DEFAULT_SIZE_MINIMUM = 100 * 1024 * 1024;
+  public static final int DEFAULT_MINIMUM_SIZE = 100 * 1024 * 1024;
 
   /**
    * Name of the DistributedLockService that {@link AutoBalancer} will use to
@@ -217,7 +217,7 @@ public class AutoBalancer implements Declarable {
    */
   class SizeBasedOOBAuditor implements OOBAuditor {
     private int sizeThreshold = DEFAULT_SIZE_THRESHOLD_PERCENT;
-    private int sizeMinimum = DEFAULT_SIZE_MINIMUM;
+    private int sizeMinimum = DEFAULT_MINIMUM_SIZE;
 
     @Override
     public void init(Properties props) {
@@ -232,10 +232,10 @@ public class AutoBalancer implements Declarable {
             throw new GemFireConfigException(SIZE_THRESHOLD_PERCENT + " should be integer, 1 to 99");
           }
         }
-        if (props.getProperty(SIZE_MINIMUM) != null) {
-          sizeMinimum = Integer.valueOf(props.getProperty(SIZE_MINIMUM));
+        if (props.getProperty(MINIMUM_SIZE) != null) {
+          sizeMinimum = Integer.valueOf(props.getProperty(MINIMUM_SIZE));
           if (sizeMinimum <= 0) {
-            throw new GemFireConfigException(SIZE_MINIMUM + " should be greater than 0");
+            throw new GemFireConfigException(MINIMUM_SIZE + " should be greater than 0");
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/0ff54f3e/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
index 93680f6..bae5f11 100644
--- a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
+++ b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
@@ -365,7 +365,7 @@ public class AutoBalancerJUnitTest {
     AutoBalancer balancer = new AutoBalancer();
     balancer.setCacheOperationFacade(mockCacheFacade);
     Properties config = getBasicConfig();
-    config.put(AutoBalancer.SIZE_MINIMUM, "10");
+    config.put(AutoBalancer.MINIMUM_SIZE, "10");
     balancer.init(config);
     SizeBasedOOBAuditor auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
 
@@ -398,7 +398,7 @@ public class AutoBalancerJUnitTest {
     AutoBalancer balancer = new AutoBalancer();
     balancer.setCacheOperationFacade(mockCacheFacade);
     Properties config = getBasicConfig();
-    config.put(AutoBalancer.SIZE_MINIMUM, "" + (totalSize * 5));
+    config.put(AutoBalancer.MINIMUM_SIZE, "" + (totalSize * 5));
     balancer.init(config);
     SizeBasedOOBAuditor auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
 
@@ -439,7 +439,7 @@ public class AutoBalancerJUnitTest {
     AutoBalancer balancer = new AutoBalancer();
     balancer.setCacheOperationFacade(mockCacheFacade);
     Properties config = getBasicConfig();
-    config.put(AutoBalancer.SIZE_MINIMUM, "10");
+    config.put(AutoBalancer.MINIMUM_SIZE, "10");
     balancer.init(config);
     SizeBasedOOBAuditor auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
 
@@ -531,11 +531,11 @@ public class AutoBalancerJUnitTest {
     balancer.init(getBasicConfig());
     SizeBasedOOBAuditor auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
     assertEquals(AutoBalancer.DEFAULT_SIZE_THRESHOLD_PERCENT, auditor.getSizeThreshold());
-    assertEquals(AutoBalancer.DEFAULT_SIZE_MINIMUM, auditor.getSizeMinimum());
+    assertEquals(AutoBalancer.DEFAULT_MINIMUM_SIZE, auditor.getSizeMinimum());
 
     Properties props = getBasicConfig();
     props.put(AutoBalancer.SIZE_THRESHOLD_PERCENT, "17");
-    props.put(AutoBalancer.SIZE_MINIMUM, "10");
+    props.put(AutoBalancer.MINIMUM_SIZE, "10");
     balancer = new AutoBalancer();
     balancer.init(props);
     auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
@@ -555,7 +555,7 @@ public class AutoBalancerJUnitTest {
   public void testConfigSizeMinNegative() {
     AutoBalancer balancer = new AutoBalancer();
     Properties props = getBasicConfig();
-    props.put(AutoBalancer.SIZE_MINIMUM, "-1");
+    props.put(AutoBalancer.MINIMUM_SIZE, "-1");
     balancer.init(props);
   }
 


[02/50] [abbrv] incubator-geode git commit: [GEODE-161] Remove commented rerunTest task

Posted by bs...@apache.org.
[GEODE-161] Remove commented rerunTest task

Executed 'clean build -Dskip.tests=true' for verfication.


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

Branch: refs/heads/feature/GEODE-77
Commit: 4f86d40b15e7889c107193249b0c300ab217c732
Parents: b387e35
Author: Mark Bretl <mb...@pivotal.io>
Authored: Mon Jul 27 17:09:25 2015 -0700
Committer: Mark Bretl <mb...@pivotal.io>
Committed: Mon Jul 27 17:09:25 2015 -0700

----------------------------------------------------------------------
 gemfire-core/build.gradle | 10 ----------
 1 file changed, 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4f86d40b/gemfire-core/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-core/build.gradle b/gemfire-core/build.gradle
index 6b8a1df..2992ee9 100755
--- a/gemfire-core/build.gradle
+++ b/gemfire-core/build.gradle
@@ -92,16 +92,6 @@ sourceSets {
   }
 }
  
-
-//  task rerunTest(type:Test) {
-//    include '**/CacheXml81DUnitTest.class'
-//    include '**/CacheXml82DUnitTest.class'
-//    include '**/ServerLauncherLocalDUnitTest.class'
-//    include '**/ServerLauncherLocalFileDUnitTest.class'
-//    include '**/JSONPdxClientServerDUnitTest.class'
-//    include '**/PRColocatedEquiJoinDUnitTest.class'
-//  }
-
 // Creates the version properties file and writes it to the classes dir
 task createVersionPropertiesFile {
   def propertiesFile = file(generatedResources + "/com/gemstone/gemfire/internal/GemFireVersion.properties");


[11/50] [abbrv] incubator-geode git commit: GEODE-168: remove Bug34011JUnitTest

Posted by bs...@apache.org.
GEODE-168: remove Bug34011JUnitTest


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

Branch: refs/heads/feature/GEODE-77
Commit: 07a540742f89ac4d6c12bac7a4eddc3810538be4
Parents: 60ebf92
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Jul 29 14:47:42 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Jul 29 14:51:39 2015 -0700

----------------------------------------------------------------------
 .../internal/cache/Bug34011JUnitTest.java       | 199 -------------------
 1 file changed, 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/07a54074/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug34011JUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug34011JUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug34011JUnitTest.java
deleted file mode 100644
index b673a56..0000000
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/Bug34011JUnitTest.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*=========================================================================
- * Copyright (c) 2010-2014 Pivotal Software, Inc. All Rights Reserved.
- * This product is protected by U.S. and international copyright
- * and intellectual property laws. Pivotal products are covered by
- * one or more patents listed at http://www.pivotal.io/patents.
- *=========================================================================
- */
-package com.gemstone.gemfire.internal.cache;
-
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.Arrays;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.*;
-
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-/**
- * To verify the bug no. 34011 no longer exists:
- * Disk region perf test for Persist only with Async writes. 
- * 
- * The test verifies that puts per second perfomance if bytes threshold being exceeded before time interval is not more than a factor
- * of 2 (the reverse is also tested, time interval causing flush's perfomance should not be better than byte-threshold exceeding by
- * a factor of 2)
- * 
- * Note : This test can fail due to external factors such as filesystem becoming slow or CPU being overloaded
- * during one run and fast during the second run.
- *  
- * @author Vikram Jadhav 
- */
-@Category(IntegrationTest.class)
-public class Bug34011JUnitTest extends DiskRegionTestingBase
-{
-  String stats1 = null;
-  String stats2 = null;
-  DiskRegionProperties diskProps1 = new DiskRegionProperties();
-  DiskRegionProperties diskProps2 = new DiskRegionProperties();
-  Region region1= null;
-  Region region2= null;
-  public float opPerSec1= 0l;
-  public float opPerSec2= 0l;
-  
-  @Before
-  public void setUp() throws Exception
-  {
-    super.setUp();
-    diskProps1.setDiskDirs(dirs);
-    diskProps2.setDiskDirs(dirs);
-    DiskStoreImpl.SET_IGNORE_PREALLOCATE = true;
-  }
-
-  @After
-  public void tearDown() throws Exception
-  {
-    super.tearDown();
-    DiskStoreImpl.SET_IGNORE_PREALLOCATE = false;
-  }
- 
-  private static int ENTRY_SIZE = 2;
-
-  private static int OP_COUNT = 100000; // 100000;
-
-  
-  /**
-   * First, the time interval is set to a low value such that time-interval always elapses before bytes threshold is reached.
-   * Then the bytes-threshold is set in such a way that byte threshold occurs before time-interval. The perfomance
-   * of the first one should not be more than a factor of two as compared to the perfomance of the second scenario. The
-   * reverse also hold true
-   *
-   */
- 
-  @Test
-  public void testpersistASync()
-  {
-
-    // test-persistASync-ByteThreshold
-    try {
-     
-      diskProps1.setTimeInterval(10);
-      diskProps1.setBytesThreshold(Integer.MAX_VALUE); // now a queue size
-      diskProps1.setRegionName("region1");
-      region1 = DiskRegionHelperFactory.getAsyncPersistOnlyRegion(cache, diskProps1);
-      
-    }
-    catch (Exception e) {
-      if(logWriter.fineEnabled()){
-        e.printStackTrace();
-      }
-      fail("failed : test-persistASync-ByteThreshold.Exception="+e);
-    }
-    //Perf test for 1kb writes
-    populateData1();
-    if(logWriter.infoEnabled()){
-    logWriter.info("testpersistASyncByteThreshold:: Stats for 1 kb writes :"
-        + stats1);
-    }
-   //  close region1
-    region1.close();
- 
-  
-    try {      
-      diskProps2.setTimeInterval(150000000l);
-      diskProps2.setBytesThreshold(32); // now a queue size
-      diskProps2.setRegionName("region2");
-      region2 = DiskRegionHelperFactory.getAsyncPersistOnlyRegion(cache, diskProps2);
-    }
-    catch (Exception e) {
-      if(logWriter.fineEnabled()) e.printStackTrace();
-      fail("Failed : test-persistASync-TimeInterval. Exception = "+e);
-    }
-    //Perf test for 1kb writes
-    populateData2();
-    if(logWriter.infoEnabled()) logWriter.info("testpersistASyncTimeInterval:: Stats for 1 kb writes :"
-        + stats2);
-     //close region2
-     region2.close();
-    
-     
-   
-     
-   
-    
-    
-    //validate that the pus/sec in both cases do not differ by twice 
-     if(logWriter.infoEnabled()) logWriter.info("opPerSec1= "+opPerSec1+"_________opPerSec2= "+opPerSec2);
-    assertTrue(opPerSec1/opPerSec2 < 3.0 );
-    assertTrue(opPerSec2/opPerSec1 < 3.0) ;
-        
-  } //end of testpersistASyncTimeInterva
-  
-  public void populateData1 ()
-  {
-    //Put for validation.
-    putForValidation(region1);
-   
-    final byte[] value = new byte[ENTRY_SIZE];
-    Arrays.fill(value, (byte)77);
-    //warm up the system
-    for (int i = 0; i < OP_COUNT; i++) {
-      region1.put("" + i, value);
-     }
-    long startTime = System.currentTimeMillis();
-    for (int i = 0; i < OP_COUNT; i++) {
-      region1.put("" + i, value);
-     }
-    long endTime = System.currentTimeMillis();
-    if(logWriter.fineEnabled()) logWriter.fine(" done with putting");
-    float et = endTime - startTime;
-    float etSecs = et / 1000f;
-    opPerSec1 = etSecs == 0 ? 0 : (OP_COUNT / (et / 1000f));
-    float bytesPerSec = etSecs == 0 ? 0
-        : ((OP_COUNT * ENTRY_SIZE) / (et / 1000f));
-    stats1 = "et=" + et + "ms writes/sec=" + opPerSec1 + " bytes/sec="
-        + bytesPerSec;
-    logWriter.info(stats1);
-   //  validate put operation
-    validatePut(region1);
-    
-  }
-  
-  public void populateData2()
-  {
-    //  Put for validation.
-    putForValidation(region2);
-    final byte[] value = new byte[ENTRY_SIZE];
-    Arrays.fill(value, (byte)77);
-    //warm up the system
-    for (int i = 0; i < OP_COUNT; i++) {
-      region2.put("" + i, value);
-     }
-    long startTime = System.currentTimeMillis();
-    for (int i = 0; i < OP_COUNT; i++) {
-      region2.put("" + i, value);
-     }
-    long endTime = System.currentTimeMillis();
-    if(logWriter.fineEnabled())  logWriter.fine(" done with putting");
-    float et = endTime - startTime;
-    float etSecs = et / 1000f;
-    opPerSec2 = etSecs == 0 ? 0 : (OP_COUNT / (et / 1000f));
-    float bytesPerSec = etSecs == 0 ? 0
-        : ((OP_COUNT * ENTRY_SIZE) / (et / 1000f));
-    stats2 = "et=" + et + "ms writes/sec=" + opPerSec2 + " bytes/sec="
-        + bytesPerSec;
-    logWriter.info(stats2);
-   //  validate put operation
-    validatePut(region2);
-  }
-  
-
-  
-}//end of the test
-


[08/50] [abbrv] incubator-geode git commit: GEODE-86: add a way to get the deserialized value to KeyValueOperationContext

Posted by bs...@apache.org.
GEODE-86: add a way to get the deserialized value to KeyValueOperationContext

KeyValueOperationContext now has a getDeserializedValue method that can be used
to get the operation value as a deserialized object.
The javadocs on getValue now explain that the result may be a serialized blob.
Unit test coverage has been added for PutOperationContext and GetOperationContextImpl
which are subclasses of KeyValueOperationContext.


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

Branch: refs/heads/feature/GEODE-77
Commit: 298ff7911ffcc7b5f7b026de340b95804b7fab5a
Parents: 405daf5
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Fri Jul 10 09:20:47 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Jul 29 10:42:58 2015 -0700

----------------------------------------------------------------------
 .../cache/operations/GetOperationContext.java   |   7 +-
 .../operations/KeyValueOperationContext.java    |  70 +++--
 .../internal/GetOperationContextImpl.java       |  16 +-
 .../PutOperationContextJUnitTest.java           | 248 +++++++++++++++++
 .../GetOperationContextImplJUnitTest.java       | 275 +++++++++++++++++++
 5 files changed, 586 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/298ff791/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/GetOperationContext.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/GetOperationContext.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/GetOperationContext.java
index 700ae00..ceb95b6 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/GetOperationContext.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/GetOperationContext.java
@@ -55,9 +55,14 @@ public class GetOperationContext extends KeyValueOperationContext {
    * 
    * @return the result of get operation; null when the result is a serialized
    *         value in which case user should invoke {@link #getSerializedValue()}
+   *         or {@link #getDeserializedValue()}.
    */
   public Object getObject() {
-    return this.value;
+    if (super.getSerializedValue() != null) {
+      return null;
+    } else {
+      return super.getValue();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/298ff791/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/KeyValueOperationContext.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/KeyValueOperationContext.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/KeyValueOperationContext.java
index e0fefb9..4b1af3f 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/KeyValueOperationContext.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/KeyValueOperationContext.java
@@ -8,6 +8,10 @@
 
 package com.gemstone.gemfire.cache.operations;
 
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.SerializationException;
+import com.gemstone.gemfire.internal.cache.EntryEventImpl;
+
 
 /**
  * Encapsulates a region operation that requires both key and serialized value
@@ -22,14 +26,9 @@ public abstract class KeyValueOperationContext extends KeyOperationContext {
    * The value of the create/update operation.
    * @since 6.5
    */
-  protected Object value;
+  private Object value;
   
   /**
-   * The serialized value of the create/update operation.
-   */
-  private byte[] serializedValue;
-
-  /**
    * True when the serialized object is a normal object; false when it is a raw
    * byte array.
    */
@@ -80,26 +79,51 @@ public abstract class KeyValueOperationContext extends KeyOperationContext {
   /**
    * Get the serialized value for this operation.
    * 
-   * @return the serialized value for this operation.
+   * @return the serialized value for this operation or null if the value is not serialized
    */
   public byte[] getSerializedValue() {
-    return this.serializedValue;
+    if (isObject()) {
+      Object tmp = this.value;
+      if (tmp instanceof byte[]) {
+        return (byte[])tmp;
+      }
+    }
+    return null;
+  }
+  
+  /**
+   * Get the deserialized value for this operation.
+   * Note that if the value is serialized this method will attempt to deserialize it.
+   * If PDX read-serialized is set to true and the value was serialized with PDX
+   * then this method will return a PdxInstance.
+   * 
+   * @return the deserialized value for this operation
+   * @throws SerializationException if deserialization of the value fails
+   * @since Geode 1.0
+   */
+  public Object getDeserializedValue() throws SerializationException {
+    byte[] blob = getSerializedValue();
+    if (blob != null) {
+      return EntryEventImpl.deserialize(blob);
+    }
+    return this.value;
   }
 
   /**
    * Get the value for this operation.
+   * Note that if the value is serialized then a byte array
+   * will be returned that contains the serialized bytes.
+   * To figure out if the returned byte array contains serialized bytes
+   * or is the deserialized value call {@link #isObject()}.
+   * If you need to deserialize the serialized bytes use 
+   * {@link DataSerializer#readObject(java.io.DataInput)}
+   * or you can just call {@link #getDeserializedValue()}.
    * 
-   * @return the value for this operation.
+   * @return the value for this operation
    * @since 6.5
    */
   public Object getValue() {
-    
-    if (serializedValue != null) {
-      return serializedValue;
-    }
-    else {
-      return value;
-    }
+    return this.value;
   }
   
   /**
@@ -121,9 +145,7 @@ public abstract class KeyValueOperationContext extends KeyOperationContext {
    *                byte array
    */
   public void setSerializedValue(byte[] serializedValue, boolean isObject) {
-    this.serializedValue = serializedValue;
-    this.value = null;
-    this.isObject = isObject;
+    setValue(serializedValue, isObject);
   }
   
 
@@ -139,14 +161,8 @@ public abstract class KeyValueOperationContext extends KeyOperationContext {
    * @since 6.5
    */
   public void setValue(Object value, boolean isObject) {
-    if (value instanceof byte[]) {
-      setSerializedValue((byte[])value, isObject);
-    }
-    else {
-      this.serializedValue = null;
-      this.value = value;
-      this.isObject = isObject;
-    }
+    this.value = value;
+    this.isObject = isObject;
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/298ff791/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
index b8e13c6..7772c84 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImpl.java
@@ -1,5 +1,6 @@
 package com.gemstone.gemfire.cache.operations.internal;
 
+import com.gemstone.gemfire.SerializationException;
 import com.gemstone.gemfire.cache.operations.GetOperationContext;
 import com.gemstone.gemfire.internal.offheap.Releasable;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
@@ -62,7 +63,7 @@ public class GetOperationContextImpl extends GetOperationContext implements Rele
   public byte[] getSerializedValue() {
     byte[] result = super.getSerializedValue();
     if (result == null) {
-      Object v = this.value;
+      Object v = super.getValue();
       if (v instanceof StoredObject) {
         checkForReleasedOffHeapValue(v);
         result = ((StoredObject) v).getValueAsHeapByteArray();
@@ -72,10 +73,21 @@ public class GetOperationContextImpl extends GetOperationContext implements Rele
   }
 
   @Override
+  public Object getDeserializedValue() throws SerializationException {
+    Object result = super.getDeserializedValue();
+    if (result instanceof StoredObject) {
+      checkForReleasedOffHeapValue(result);
+      result = ((StoredObject) result).getValueAsDeserializedHeapObject();
+    }
+    return result;
+  }
+
+  @Override
   public Object getValue() {
     Object result = super.getValue();
     if (result instanceof StoredObject) {
       checkForReleasedOffHeapValue(result);
+      // since they called getValue they don't care if it is serialized or deserialized so return it as serialized
       result = ((StoredObject) result).getValueAsHeapByteArray();
     }
     return result;
@@ -88,7 +100,7 @@ public class GetOperationContextImpl extends GetOperationContext implements Rele
     // our value (since this context did not retain it)
     // but we do make sure that any future attempt to access
     // the off-heap value fails.
-    if (this.value instanceof Chunk) {
+    if (super.getValue() instanceof Chunk) {
       this.released = true;
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/298ff791/gemfire-core/src/test/java/com/gemstone/gemfire/cache/operations/PutOperationContextJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/operations/PutOperationContextJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/operations/PutOperationContextJUnitTest.java
new file mode 100644
index 0000000..32db7bd
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/operations/PutOperationContextJUnitTest.java
@@ -0,0 +1,248 @@
+package com.gemstone.gemfire.cache.operations;
+
+import static org.junit.Assert.*;
+import static org.junit.matchers.JUnitMatchers.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.matchers.JUnitMatchers;
+import org.junit.rules.ExpectedException;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.pdx.PdxInstance;
+import com.gemstone.gemfire.pdx.PdxReader;
+import com.gemstone.gemfire.pdx.PdxSerializable;
+import com.gemstone.gemfire.pdx.PdxWriter;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class PutOperationContextJUnitTest {
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+  
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+  }
+
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public void testGetSerializedValue() throws IOException {
+    {
+      byte[] byteArrayValue = new byte[]{1,2,3,4};
+      PutOperationContext poc = new PutOperationContext("key", byteArrayValue, false, PutOperationContext.CREATE, false);
+      Assert.assertFalse(poc.isObject());
+      Assert.assertNull("value is an actual byte array which is not a serialized blob", poc.getSerializedValue());
+    }
+
+    {
+      PutOperationContext poc = new PutOperationContext("key", null, true, PutOperationContext.CREATE, false);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertNull("value is null which is not a serialized blob", poc.getSerializedValue());
+    }
+
+    {
+      PutOperationContext  poc = new PutOperationContext("key", "value", true, PutOperationContext.CREATE, false);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertNull("value is a String which is not a serialized blob", poc.getSerializedValue());
+    }
+
+    {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      DataOutputStream dos = new DataOutputStream(baos);
+      DataSerializer.writeObject("value", dos);
+      dos.close();
+      byte[] blob = baos.toByteArray();
+      PutOperationContext poc = new PutOperationContext("key", blob, true, PutOperationContext.CREATE, false);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertArrayEquals(blob, poc.getSerializedValue());
+    }
+
+    {
+      // create a loner cache so that pdx serialization will work
+      Cache c = (new CacheFactory()).set("locators", "").set("mcast-port", "0").setPdxReadSerialized(true).create();
+      try {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        DataSerializer.writeObject(new PdxValue("value"), dos);
+        dos.close();
+        byte[] blob = baos.toByteArray();
+        PutOperationContext poc = new PutOperationContext("key", blob, true, PutOperationContext.CREATE, false);
+        Assert.assertTrue(poc.isObject());
+        Assert.assertArrayEquals(blob, poc.getSerializedValue());
+      } finally {
+        c.close();
+      }
+    }
+  }
+
+  @Test
+  public void testGetDeserializedValue() throws IOException {
+    {
+      byte[] byteArrayValue = new byte[]{1,2,3,4};
+      PutOperationContext poc = new PutOperationContext("key", byteArrayValue, false, PutOperationContext.CREATE, false);
+      Assert.assertFalse(poc.isObject());
+      Assert.assertArrayEquals(byteArrayValue, (byte[]) poc.getDeserializedValue());
+    }
+
+    {
+      PutOperationContext poc = new PutOperationContext("key", null, true, PutOperationContext.CREATE, false);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals(null, poc.getDeserializedValue());
+    }
+
+    {
+      PutOperationContext  poc = new PutOperationContext("key", "value", true, PutOperationContext.CREATE, false);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals("value", poc.getDeserializedValue());
+    }
+
+    {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      DataOutputStream dos = new DataOutputStream(baos);
+      DataSerializer.writeObject("value", dos);
+      dos.close();
+      byte[] blob = baos.toByteArray();
+      PutOperationContext poc = new PutOperationContext("key", blob, true, PutOperationContext.CREATE, false);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals("value", poc.getDeserializedValue());
+    }
+
+    {
+      // create a loner cache so that pdx serialization will work
+      Cache c = (new CacheFactory()).set("locators", "").set("mcast-port", "0").setPdxReadSerialized(true).create();
+      try {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        DataSerializer.writeObject(new PdxValue("value"), dos);
+        dos.close();
+        byte[] blob = baos.toByteArray();
+        PutOperationContext poc = new PutOperationContext("key", blob, true, PutOperationContext.CREATE, false);
+        Assert.assertTrue(poc.isObject());
+        PdxInstance pi = (PdxInstance) poc.getDeserializedValue();
+        Assert.assertEquals("value", pi.getField("v"));
+      } finally {
+        c.close();
+      }
+    }
+  }
+
+  @Test
+  public void testGetValue() throws IOException {
+    {
+      byte[] byteArrayValue = new byte[]{1,2,3,4};
+      PutOperationContext poc = new PutOperationContext("key", byteArrayValue, false, PutOperationContext.CREATE, false);
+      Assert.assertFalse(poc.isObject());
+      Assert.assertArrayEquals(byteArrayValue, (byte[]) poc.getValue());
+    }
+
+    {
+      PutOperationContext poc = new PutOperationContext("key", null, true, PutOperationContext.CREATE, false);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals(null, poc.getValue());
+    }
+
+    {
+      PutOperationContext  poc = new PutOperationContext("key", "value", true, PutOperationContext.CREATE, false);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals("value", poc.getValue());
+    }
+
+    {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      DataOutputStream dos = new DataOutputStream(baos);
+      DataSerializer.writeObject("value", dos);
+      dos.close();
+      byte[] blob = baos.toByteArray();
+      PutOperationContext poc = new PutOperationContext("key", blob, true, PutOperationContext.CREATE, false);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertArrayEquals(blob, (byte[]) poc.getValue());
+    }
+
+    {
+      // create a loner cache so that pdx serialization will work
+      Cache c = (new CacheFactory()).set("locators", "").set("mcast-port", "0").setPdxReadSerialized(true).create();
+      try {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        DataSerializer.writeObject(new PdxValue("value"), dos);
+        dos.close();
+        byte[] blob = baos.toByteArray();
+        PutOperationContext poc = new PutOperationContext("key", blob, true, PutOperationContext.CREATE, false);
+        Assert.assertTrue(poc.isObject());
+        Assert.assertArrayEquals(blob, (byte[]) poc.getValue());
+      } finally {
+        c.close();
+      }
+    }
+  }
+
+  public static class PdxValue implements PdxSerializable {
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = 1;
+      result = prime * result + ((v == null) ? 0 : v.hashCode());
+      return result;
+    }
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj)
+        return true;
+      if (obj == null)
+        return false;
+      if (getClass() != obj.getClass())
+        return false;
+      PdxValue other = (PdxValue) obj;
+      if (v == null) {
+        if (other.v != null)
+          return false;
+      } else if (!v.equals(other.v))
+        return false;
+      return true;
+    }
+
+    private String v;
+    
+    public PdxValue() {
+    }
+    
+    public PdxValue(String v) {
+      this.v = v;
+    }
+    @Override
+    public void toData(PdxWriter writer) {
+      writer.writeString("v", this.v);
+    }
+
+    @Override
+    public void fromData(PdxReader reader) {
+      this.v = reader.readString("v");
+    }
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/298ff791/gemfire-core/src/test/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImplJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImplJUnitTest.java
new file mode 100644
index 0000000..8607468
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/operations/internal/GetOperationContextImplJUnitTest.java
@@ -0,0 +1,275 @@
+package com.gemstone.gemfire.cache.operations.internal;
+
+import static org.junit.Assert.*;
+import static org.junit.matchers.JUnitMatchers.*;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.Properties;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.matchers.JUnitMatchers;
+import org.junit.rules.ExpectedException;
+
+import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.operations.PutOperationContextJUnitTest;
+import com.gemstone.gemfire.pdx.PdxInstance;
+import com.gemstone.gemfire.pdx.PdxReader;
+import com.gemstone.gemfire.pdx.PdxSerializable;
+import com.gemstone.gemfire.pdx.PdxWriter;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class GetOperationContextImplJUnitTest {
+
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+  
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+  }
+
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test
+  public void testGetSerializedValue() throws IOException {
+    {
+      byte[] byteArrayValue = new byte[]{1,2,3,4};
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(byteArrayValue, false);
+      Assert.assertFalse(poc.isObject());
+      Assert.assertNull("value is an actual byte array which is not a serialized blob", poc.getSerializedValue());
+    }
+
+    {
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(null, true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertNull("value is null which is not a serialized blob", poc.getSerializedValue());
+    }
+
+    {
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject("value", true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertNull("value is a String which is not a serialized blob", poc.getSerializedValue());
+    }
+
+    {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      DataOutputStream dos = new DataOutputStream(baos);
+      DataSerializer.writeObject("value", dos);
+      dos.close();
+      byte[] blob = baos.toByteArray();
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(blob, true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertArrayEquals(blob, poc.getSerializedValue());
+    }
+
+    {
+      // create a loner cache so that pdx serialization will work
+      Cache c = (new CacheFactory()).set("locators", "").set("mcast-port", "0").setPdxReadSerialized(true).create();
+      try {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        DataSerializer.writeObject(new PutOperationContextJUnitTest.PdxValue("value"), dos);
+        dos.close();
+        byte[] blob = baos.toByteArray();
+        GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+        poc.setObject(blob, true);
+        Assert.assertTrue(poc.isObject());
+        Assert.assertArrayEquals(blob, poc.getSerializedValue());
+      } finally {
+        c.close();
+      }
+    }
+  }
+
+  @Test
+  public void testGetDeserializedValue() throws IOException {
+    {
+      byte[] byteArrayValue = new byte[]{1,2,3,4};
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(byteArrayValue, false);
+      Assert.assertFalse(poc.isObject());
+      Assert.assertArrayEquals(byteArrayValue, (byte[]) poc.getDeserializedValue());
+    }
+
+    {
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(null, true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals(null, poc.getDeserializedValue());
+    }
+
+    {
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject("value", true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals("value", poc.getDeserializedValue());
+    }
+
+    {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      DataOutputStream dos = new DataOutputStream(baos);
+      DataSerializer.writeObject("value", dos);
+      dos.close();
+      byte[] blob = baos.toByteArray();
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(blob, true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals("value", poc.getDeserializedValue());
+    }
+
+    {
+      // create a loner cache so that pdx serialization will work
+      Cache c = (new CacheFactory()).set("locators", "").set("mcast-port", "0").setPdxReadSerialized(true).create();
+      try {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        DataSerializer.writeObject(new PutOperationContextJUnitTest.PdxValue("value"), dos);
+        dos.close();
+        byte[] blob = baos.toByteArray();
+        GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+        poc.setObject(blob, true);
+        Assert.assertTrue(poc.isObject());
+        PdxInstance pi = (PdxInstance) poc.getDeserializedValue();
+        Assert.assertEquals("value", pi.getField("v"));
+      } finally {
+        c.close();
+      }
+    }
+  }
+
+  @Test
+  public void testGetValue() throws IOException {
+    {
+      byte[] byteArrayValue = new byte[]{1,2,3,4};
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(byteArrayValue, false);
+      Assert.assertFalse(poc.isObject());
+      Assert.assertArrayEquals(byteArrayValue, (byte[]) poc.getValue());
+    }
+
+    {
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(null, true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals(null, poc.getValue());
+    }
+
+    {
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject("value", true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals("value", poc.getValue());
+    }
+
+    {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      DataOutputStream dos = new DataOutputStream(baos);
+      DataSerializer.writeObject("value", dos);
+      dos.close();
+      byte[] blob = baos.toByteArray();
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(blob, true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertArrayEquals(blob, (byte[]) poc.getValue());
+    }
+
+    {
+      // create a loner cache so that pdx serialization will work
+      Cache c = (new CacheFactory()).set("locators", "").set("mcast-port", "0").setPdxReadSerialized(true).create();
+      try {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        DataSerializer.writeObject(new PutOperationContextJUnitTest.PdxValue("value"), dos);
+        dos.close();
+        byte[] blob = baos.toByteArray();
+        GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+        poc.setObject(blob, true);
+        Assert.assertTrue(poc.isObject());
+        Assert.assertArrayEquals(blob, (byte[]) poc.getValue());
+      } finally {
+        c.close();
+      }
+    }
+  }
+
+  @Test
+  public void testGetObject() throws IOException {
+    {
+      byte[] byteArrayValue = new byte[]{1,2,3,4};
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(byteArrayValue, false);
+      Assert.assertFalse(poc.isObject());
+      Assert.assertArrayEquals(byteArrayValue, (byte[]) poc.getObject());
+    }
+
+    {
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(null, true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals(null, poc.getObject());
+    }
+
+    {
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject("value", true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertEquals("value", poc.getObject());
+    }
+
+    {
+      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      DataOutputStream dos = new DataOutputStream(baos);
+      DataSerializer.writeObject("value", dos);
+      dos.close();
+      byte[] blob = baos.toByteArray();
+      GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+      poc.setObject(blob, true);
+      Assert.assertTrue(poc.isObject());
+      Assert.assertNull("value is a serialized blob which is not an Object", poc.getObject());
+    }
+
+    {
+      // create a loner cache so that pdx serialization will work
+      Cache c = (new CacheFactory()).set("locators", "").set("mcast-port", "0").setPdxReadSerialized(true).create();
+      try {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        DataOutputStream dos = new DataOutputStream(baos);
+        DataSerializer.writeObject(new PutOperationContextJUnitTest.PdxValue("value"), dos);
+        dos.close();
+        byte[] blob = baos.toByteArray();
+        GetOperationContextImpl poc = new GetOperationContextImpl("key", true);
+        poc.setObject(blob, true);
+        Assert.assertTrue(poc.isObject());
+        Assert.assertNull("value is a serialized blob which is not an Object", poc.getObject());
+      } finally {
+        c.close();
+      }
+    }
+  }
+  // @TODO OFFHEAP: add coverage of "release" and the gettors methods with StoreObject and off-heap Chunk values
+}


[25/50] [abbrv] incubator-geode git commit: Fix GEODE-186 by removing sleeps in tests

Posted by bs...@apache.org.
Fix GEODE-186 by removing sleeps in tests

The old test scheduled tx suspension to timeout after 1 minute.
So the test always run for at least 1 minute.
A test hook now exists that allows the test to specify
a different time unit (default is still minutes) for
tx suspension expiration.

The sleeps in a bunch of other tests were not needed
since the tx operation is synchronous. So those sleeps
have simply been removed.

A couple of sleeps in clients waiting for something to
arrive that was done on a server have been converted to
a wait since server to client distribution is async.

Reviewed by Swapnil.


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

Branch: refs/heads/feature/GEODE-77
Commit: ea9f03e778631f35366716a664e1ee6a714ef7b0
Parents: 01145b8
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Aug 4 11:43:39 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Aug 5 14:31:54 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/TXManagerImpl.java   | 11 +++++--
 .../cache/ClientServerTransactionDUnitTest.java | 32 ++++++++++----------
 .../cache/RemoteTransactionDUnitTest.java       | 30 ++++++++++++------
 3 files changed, 45 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ea9f03e7/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
index 88714b0..dde3793 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
@@ -1117,6 +1117,10 @@ public final class TXManagerImpl implements CacheTransactionManager,
   private ConcurrentMap<TransactionId, TXStateProxy> suspendedTXs = new ConcurrentHashMap<TransactionId, TXStateProxy>();
   
   public TransactionId suspend() {
+    return suspend(TimeUnit.MINUTES);
+  }
+  
+  TransactionId suspend(TimeUnit expiryTimeUnit) {
     TXStateProxy result = getTXState();
     if (result != null) {
       TransactionId txId = result.getTransactionId();
@@ -1137,7 +1141,7 @@ public final class TXManagerImpl implements CacheTransactionManager,
           LockSupport.unpark(waitingThread);
         }
       }
-      scheduleExpiry(txId);
+      scheduleExpiry(txId, expiryTimeUnit);
       return txId;
     }
     return null;
@@ -1266,8 +1270,9 @@ public final class TXManagerImpl implements CacheTransactionManager,
   /**
    * schedules the transaction to expire after {@link #suspendedTXTimeout}
    * @param txId
+   * @param expiryTimeUnit the time unit to use when scheduling the expiration
    */
-  private void scheduleExpiry(TransactionId txId) {
+  private void scheduleExpiry(TransactionId txId, TimeUnit expiryTimeUnit) {
     final GemFireCacheImpl cache = (GemFireCacheImpl) this.cache;
     if (suspendedTXTimeout < 0) {
       if (logger.isDebugEnabled()) {
@@ -1279,7 +1284,7 @@ public final class TXManagerImpl implements CacheTransactionManager,
     if (logger.isDebugEnabled()) {
       logger.debug("TX: scheduling transaction: {} to expire after:{}", txId, suspendedTXTimeout);
     }
-    cache.getCCPTimer().schedule(task, suspendedTXTimeout*60*1000);
+    cache.getCCPTimer().schedule(task, TimeUnit.MILLISECONDS.convert(suspendedTXTimeout, expiryTimeUnit));
     this.expiryTasks.put(txId, task);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ea9f03e7/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
index 3ce8cec..d80f6bb 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
@@ -16,6 +16,7 @@ import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 import javax.naming.Context;
 import javax.transaction.UserTransaction;
@@ -78,6 +79,7 @@ import com.gemstone.gemfire.internal.cache.execute.util.CommitFunction;
 import com.gemstone.gemfire.internal.cache.execute.util.RollbackFunction;
 import com.gemstone.gemfire.internal.cache.tx.ClientTXStateStub;
 
+import dunit.DistributedTestCase;
 import dunit.Host;
 import dunit.SerializableCallable;
 import dunit.SerializableRunnable;
@@ -738,7 +740,6 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
       }
     });
     
-    Thread.sleep(10000);
     datastore.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         Region<CustId, Customer> custRegion = getCache().getRegion(CUSTOMER);
@@ -926,14 +927,12 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
       }
     });
     
-    Thread.sleep(10000);
     client.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         Region<CustId, Customer> custRegion = getCache().getRegion(CUSTOMER);
 //        Region<OrderId, Order> orderRegion = getCache().getRegion(ORDER);
 //        Region<CustId,Customer> refRegion = getCache().getRegion(D_REFERENCE);
         ClientListener cl = (ClientListener) custRegion.getAttributes().getCacheListeners()[0];
-        getCache().getLogger().info("SWAP:CLIENTinvoked:"+cl.invoked);
         assertTrue(cl.invoked);
         assertEquals("it should be 1 but its:"+cl.invokeCount,1,cl.invokeCount);
         return null;
@@ -981,14 +980,12 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
 	      }
 	    });
 	    
-	    Thread.sleep(10000);
 	    client.invoke(new SerializableCallable() {
 	      public Object call() throws Exception {
 	        Region<CustId, Customer> custRegion = getCache().getRegion(CUSTOMER);
 //	        Region<OrderId, Order> orderRegion = getCache().getRegion(ORDER);
 //	        Region<CustId,Customer> refRegion = getCache().getRegion(D_REFERENCE);
 	        ClientListener cl = (ClientListener) custRegion.getAttributes().getCacheListeners()[0];
-	        getCache().getLogger().info("SWAP:CLIENTinvoked:"+cl.invoked);
 	        assertTrue(cl.invoked);
 	        assertEquals("it should be 1 but its:"+cl.invokeCount,1,cl.invokeCount);
 	        return null;
@@ -1038,14 +1035,12 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
 	      }
 	    });
 	    
-	    Thread.sleep(10000);
 	    client.invoke(new SerializableCallable() {
 	      public Object call() throws Exception {
 	        Region<CustId, Customer> custRegion = getCache().getRegion(CUSTOMER);
 //	        Region<OrderId, Order> orderRegion = getCache().getRegion(ORDER);
 //	        Region<CustId,Customer> refRegion = getCache().getRegion(D_REFERENCE);
 	        ClientListener cl = (ClientListener) custRegion.getAttributes().getCacheListeners()[0];
-	        getCache().getLogger().info("SWAP:CLIENTinvoked:"+cl.invoked);
 	        assertTrue(cl.invoked);
 	        assertTrue(cl.putAllOp);
 	        assertFalse(cl.isOriginRemote);
@@ -1094,7 +1089,6 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
 	      }
 	    });
 	    
-	    Thread.sleep(10000);
 	    client.invoke(new SerializableCallable() {
 	      public Object call() throws Exception {
 	        Region<CustId, Customer> custRegion = getCache().getRegion(CUSTOMER);
@@ -1150,9 +1144,6 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
       }
     });
     
-    Thread.sleep(10000);
-    
-    
     /*
      * Validate nothing came through
      */
@@ -1199,7 +1190,6 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
     
     client.invoke(doAPutInTx);
     client.invoke(doAnInvalidateInTx);
-    Thread.sleep(10000);
     
     client.invoke(new SerializableCallable() {
       public Object call() throws Exception {
@@ -1365,7 +1355,6 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
           Customer cust = new Customer("name"+i, "address"+i);
           pr.put(custId, cust);
           r.put(i, "value"+i);
-          Thread.sleep(100);
         }
         return null;
       }
@@ -1464,7 +1453,6 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
           getGemfireCache().getLogger().info("SWAP:putting:"+custId);
           pr.put(custId, cust);
           r.put(i, "value"+i);
-          Thread.sleep(100);
         }
         return mgr.getTransactionId();
       }
@@ -2960,9 +2948,21 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
         Region r = getCache().getRegion(CUSTOMER);
         assertNull(r.get(new CustId(101)));
         mgr.begin();
+        final TXStateProxy txState = mgr.getTXState();
+        assertTrue(txState.isInProgress());
         r.put(new CustId(101), new Customer("name101", "address101"));
-        TransactionId txId = mgr.suspend();
-        Thread.sleep(70*1000);
+        TransactionId txId = mgr.suspend(TimeUnit.MILLISECONDS);
+        WaitCriterion waitForTxTimeout = new WaitCriterion() {
+          public boolean done() {
+            return !txState.isInProgress();
+          }
+          public String description() {
+            return "txState stayed in progress indicating that the suspend did not timeout";
+          }
+        };
+        // tx should timeout after 1 ms but to deal with loaded machines and thread
+        // scheduling latency wait for 10 seconds before reporting an error.
+        DistributedTestCase.waitForCriterion(waitForTxTimeout, 10 * 1000, 10, true);
         try {
           mgr.resume(txId);
           fail("expected exception not thrown");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ea9f03e7/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
index 0daaafb..a78fab4 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
@@ -92,6 +92,7 @@ import dunit.Host;
 import dunit.SerializableCallable;
 import dunit.SerializableRunnable;
 import dunit.VM;
+import dunit.DistributedTestCase.WaitCriterion;
 
 /**
  * @author sbawaska
@@ -140,7 +141,6 @@ public class RemoteTransactionDUnitTest extends CacheTestCase {
   
   @Override
   public void tearDown2() throws Exception {
-//    try { Thread.sleep(5000); } catch (InterruptedException e) { } // FOR MANUAL TESTING OF STATS - DON"T KEEP THIS
     try {
       invokeInEveryVM(verifyNoTxState);
     } finally {
@@ -3600,15 +3600,21 @@ protected static class ClientListener extends CacheListenerAdapter {
       }
     });
     
-    Thread.sleep(10000);
     client.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         Region<CustId, Customer> custRegion = getCache().getRegion(CUSTOMER);
         Region<OrderId, Order> orderRegion = getCache().getRegion(ORDER);
         Region<CustId,Customer> refRegion = getCache().getRegion(D_REFERENCE);
-        ClientListener cl = (ClientListener) custRegion.getAttributes().getCacheListeners()[0];
-        getCache().getLogger().info("SWAP:CLIENTinvoked:"+cl.invoked);
-        assertTrue(cl.invoked);
+        final ClientListener cl = (ClientListener) custRegion.getAttributes().getCacheListeners()[0];
+        WaitCriterion waitForListenerInvocation = new WaitCriterion() {
+          public boolean done() {
+            return cl.invoked;
+          }
+          public String description() {
+            return "listener was never invoked";
+          }
+        };
+        DistributedTestCase.waitForCriterion(waitForListenerInvocation, 10 * 1000, 10, true);
         return null;
       }
     });
@@ -3715,15 +3721,21 @@ protected static class ClientListener extends CacheListenerAdapter {
       }
     });
     
-    Thread.sleep(10000);
     client.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         Region<CustId, Customer> custRegion = getCache().getRegion(CUSTOMER);
         Region<OrderId, Order> orderRegion = getCache().getRegion(ORDER);
         Region<CustId,Customer> refRegion = getCache().getRegion(D_REFERENCE);
-        ClientListener cl = (ClientListener) custRegion.getAttributes().getCacheListeners()[0];
-        getCache().getLogger().info("SWAP:CLIENTinvoked:"+cl.invoked);
-        assertTrue(cl.invoked);
+        final ClientListener cl = (ClientListener) custRegion.getAttributes().getCacheListeners()[0];
+        WaitCriterion waitForListenerInvocation = new WaitCriterion() {
+          public boolean done() {
+            return cl.invoked;
+          }
+          public String description() {
+            return "listener was never invoked";
+          }
+        };
+        DistributedTestCase.waitForCriterion(waitForListenerInvocation, 10 * 1000, 10, true);
         return null;
       }
     });


[09/50] [abbrv] incubator-geode git commit: GEODE-124: Address DLock related review comments

Posted by bs...@apache.org.
GEODE-124: Address DLock related review comments

* Do not release Rebalance Dlock after each task. This means a member holds the
  lock for its lifetime
* Increment rebalance attempt stat only if lock is acquired

https://reviews.apache.org/r/36662/


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

Branch: refs/heads/feature/GEODE-77
Commit: 65e2cc69ed3fadeebc5e65a831a51a8c006c34e2
Parents: 1683361
Author: Ashvin Agrawal <as...@apache.org>
Authored: Tue Jul 28 17:01:03 2015 -0700
Committer: Ashvin Agrawal <as...@apache.org>
Committed: Wed Jul 29 14:11:57 2015 -0700

----------------------------------------------------------------------
 .../gemfire/cache/util/AutoBalancer.java        |  47 ++++----
 .../cache/util/AutoBalancerJUnitTest.java       | 118 +++++++++----------
 2 files changed, 75 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/65e2cc69/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java b/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
index ef795b0..72a2f95 100644
--- a/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
+++ b/gemfire-rebalancer/src/main/java/com/gemstone/gemfire/cache/util/AutoBalancer.java
@@ -10,6 +10,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.logging.log4j.Logger;
 import org.quartz.CronExpression;
@@ -122,6 +123,7 @@ public class AutoBalancer implements Declarable {
   private OOBAuditor auditor = new SizeBasedOOBAuditor();
   private TimeProvider clock = new SystemClockTimeProvider();
   private CacheOperationFacade cacheFacade = new GeodeCacheFacade();
+  private AtomicBoolean isLockAcquired = new AtomicBoolean(false);
 
   private static final Logger logger = LogService.getLogger();
 
@@ -241,28 +243,32 @@ public class AutoBalancer implements Declarable {
 
     @Override
     public void execute() {
+      if (!isLockAcquired.get()) {
+        synchronized (isLockAcquired) {
+          if (!isLockAcquired.get()) {
+            boolean result = cacheFacade.acquireAutoBalanceLock();
+            if (result) {
+              isLockAcquired.set(true);
+            } else {
+              if (logger.isDebugEnabled()) {
+                logger.debug("Another member owns auto-balance lock. Skip this attempt to rebalance the cluster");
+              }
+              return;
+            }
+          }
+        }
+      }
+
       cacheFacade.incrementAttemptCounter();
-      boolean result = cacheFacade.acquireAutoBalanceLock();
+      boolean result = needsRebalancing();
       if (!result) {
         if (logger.isDebugEnabled()) {
-          logger.debug("Another member owns auto-balance lock. Skip this attempt to rebalance the cluster");
+          logger.debug("Rebalancing is not needed");
         }
         return;
       }
 
-      try {
-        result = needsRebalancing();
-        if (!result) {
-          if (logger.isDebugEnabled()) {
-            logger.debug("Rebalancing is not needed");
-          }
-          return;
-        }
-
-        cacheFacade.rebalance();
-      } finally {
-        cacheFacade.releaseAutoBalanceLock();
-      }
+      cacheFacade.rebalance();
     }
 
     /**
@@ -402,6 +408,7 @@ public class AutoBalancer implements Declarable {
       return cache;
     }
 
+    @Override
     public boolean acquireAutoBalanceLock() {
       DistributedLockService dls = getDLS();
 
@@ -412,14 +419,6 @@ public class AutoBalancer implements Declarable {
       return result;
     }
 
-    public void releaseAutoBalanceLock() {
-      DistributedLockService dls = getDLS();
-      dls.unlock(AUTO_BALANCER_LOCK);
-      if (logger.isDebugEnabled()) {
-        logger.debug("Successfully released auto-balance ownership");
-      }
-    }
-
     @Override
     public DistributedLockService getDLS() {
       GemFireCacheImpl cache = getCache();
@@ -459,8 +458,6 @@ public class AutoBalancer implements Declarable {
   interface CacheOperationFacade {
     boolean acquireAutoBalanceLock();
 
-    void releaseAutoBalanceLock();
-
     DistributedLockService getDLS();
 
     void rebalance();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/65e2cc69/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
index db225cb..93680f6 100644
--- a/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
+++ b/gemfire-rebalancer/src/test/java/com/gemstone/gemfire/cache/util/AutoBalancerJUnitTest.java
@@ -124,14 +124,6 @@ public class AutoBalancerJUnitTest {
       {
         oneOf(mockCacheFacade).acquireAutoBalanceLock();
         will(returnValue(false));
-        oneOf(mockCacheFacade).incrementAttemptCounter();
-        will(new CustomAction("increment stat") {
-          public Object invoke(Invocation invocation) throws Throwable {
-            new GeodeCacheFacade().incrementAttemptCounter();
-            return null;
-          }
-        });
-        allowing(mockCacheFacade);
       }
     });
 
@@ -139,7 +131,7 @@ public class AutoBalancerJUnitTest {
     AutoBalancer balancer = new AutoBalancer();
     balancer.setCacheOperationFacade(mockCacheFacade);
     balancer.getOOBAuditor().execute();
-    assertEquals(1, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
+    assertEquals(0, cache.getResourceManager().getStats().getAutoRebalanceAttempts());
   }
 
   @Test
@@ -224,79 +216,77 @@ public class AutoBalancerJUnitTest {
   }
 
   @Test
-  public void testReleaseLock() throws InterruptedException {
+  public void testLockStatExecuteInSequence() throws InterruptedException {
     cache = createBasicCache();
 
-    final AtomicBoolean success = new AtomicBoolean(false);
-    Thread thread = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        DistributedLockService dls = new GeodeCacheFacade().getDLS();
-        success.set(dls.lock(AutoBalancer.AUTO_BALANCER_LOCK, 0, -1));
-      }
-    });
-    thread.start();
-    thread.join();
-
-    final DistributedLockService mockDLS = mockContext.mock(DistributedLockService.class);
+    final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
+    final Sequence sequence = mockContext.sequence("sequence");
     mockContext.checking(new Expectations() {
       {
-        oneOf(mockDLS).unlock(AutoBalancer.AUTO_BALANCER_LOCK);
-        will(new CustomAction("release lock") {
-          @Override
-          public Object invoke(Invocation invocation) throws Throwable {
-            DistributedLockService dls = new GeodeCacheFacade().getDLS();
-            dls.unlock(AutoBalancer.AUTO_BALANCER_LOCK);
-            return null;
-          }
-        });
+        oneOf(mockCacheFacade).acquireAutoBalanceLock();
+        inSequence(sequence);
+        will(returnValue(true));
+        oneOf(mockCacheFacade).incrementAttemptCounter();
+        inSequence(sequence);
+        oneOf(mockCacheFacade).getTotalTransferSize();
+        inSequence(sequence);
+        will(returnValue(0L));
       }
     });
 
-    success.set(true);
-    thread = new Thread(new Runnable() {
-      @Override
-      public void run() {
-        CacheOperationFacade cacheFacade = new GeodeCacheFacade() {
-          public DistributedLockService getDLS() {
-            return mockDLS;
-          }
-        };
-        try {
-          cacheFacade.releaseAutoBalanceLock();
-        } catch (Exception e) {
-          success.set(false);
-        }
-      }
-    });
-    thread.start();
-    thread.join();
-    assertTrue(success.get());
+    AutoBalancer balancer = new AutoBalancer();
+    balancer.setCacheOperationFacade(mockCacheFacade);
+    balancer.getOOBAuditor().execute();
   }
 
   @Test
-  public void testLockSequence() throws InterruptedException {
+  public void testReusePreAcquiredLock() throws InterruptedException {
     cache = createBasicCache();
 
     final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
-    final Sequence lockingSequence = mockContext.sequence("lockingSequence");
     mockContext.checking(new Expectations() {
       {
         oneOf(mockCacheFacade).acquireAutoBalanceLock();
-        inSequence(lockingSequence);
         will(returnValue(true));
-        oneOf(mockCacheFacade).releaseAutoBalanceLock();
-        inSequence(lockingSequence);
-        allowing(mockCacheFacade);
+        exactly(2).of(mockCacheFacade).incrementAttemptCounter();
+        exactly(2).of(mockCacheFacade).getTotalTransferSize();
+        will(returnValue(0L));
       }
     });
 
     AutoBalancer balancer = new AutoBalancer();
     balancer.setCacheOperationFacade(mockCacheFacade);
     balancer.getOOBAuditor().execute();
+    balancer.getOOBAuditor().execute();
   }
 
   @Test
+  public void testAcquireLockAfterReleasedRemotely() throws InterruptedException {
+    cache = createBasicCache();
+    
+    final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
+    final Sequence sequence = mockContext.sequence("sequence");
+    mockContext.checking(new Expectations() {
+      {
+        oneOf(mockCacheFacade).acquireAutoBalanceLock();
+        inSequence(sequence);
+        will(returnValue(false));
+        oneOf(mockCacheFacade).acquireAutoBalanceLock();
+        inSequence(sequence);
+        will(returnValue(true));
+        oneOf(mockCacheFacade).incrementAttemptCounter();
+        oneOf(mockCacheFacade).getTotalTransferSize();
+        will(returnValue(0L));
+      }
+    });
+    
+    AutoBalancer balancer = new AutoBalancer();
+    balancer.setCacheOperationFacade(mockCacheFacade);
+    balancer.getOOBAuditor().execute();
+    balancer.getOOBAuditor().execute();
+  }
+  
+  @Test
   public void testFailExecuteIfLockedElsewhere() throws InterruptedException {
     cache = createBasicCache();
 
@@ -305,7 +295,6 @@ public class AutoBalancerJUnitTest {
       {
         oneOf(mockCacheFacade).acquireAutoBalanceLock();
         will(returnValue(false));
-        oneOf(mockCacheFacade).incrementAttemptCounter();
         // no other methods, rebalance, will be called
       }
     });
@@ -333,7 +322,6 @@ public class AutoBalancerJUnitTest {
         will(returnValue(true));
         never(mockCacheFacade).rebalance();
         oneOf(mockCacheFacade).incrementAttemptCounter();
-        oneOf(mockCacheFacade).releaseAutoBalanceLock();
       }
     });
 
@@ -424,21 +412,21 @@ public class AutoBalancerJUnitTest {
   @Test
   public void testOOBWhenAboveThresholdAndMin() {
     final long totalSize = 1000L;
-    
+
     final Map<PartitionedRegion, InternalPRInfo> details = new HashMap<>();
     final CacheOperationFacade mockCacheFacade = mockContext.mock(CacheOperationFacade.class);
     mockContext.checking(new Expectations() {
       {
         allowing(mockCacheFacade).getRegionMemberDetails();
         will(returnValue(details));
-        
+
         // first run
         oneOf(mockCacheFacade).getTotalDataSize(details);
         will(returnValue(totalSize));
         oneOf(mockCacheFacade).getTotalTransferSize();
         // twice threshold
         will(returnValue((AutoBalancer.DEFAULT_SIZE_THRESHOLD_PERCENT * totalSize / 100) * 2));
-        
+
         // second run
         oneOf(mockCacheFacade).getTotalDataSize(details);
         will(returnValue(totalSize));
@@ -447,21 +435,21 @@ public class AutoBalancerJUnitTest {
         will(returnValue(2 * totalSize));
       }
     });
-    
+
     AutoBalancer balancer = new AutoBalancer();
     balancer.setCacheOperationFacade(mockCacheFacade);
     Properties config = getBasicConfig();
     config.put(AutoBalancer.SIZE_MINIMUM, "10");
     balancer.init(config);
     SizeBasedOOBAuditor auditor = (SizeBasedOOBAuditor) balancer.getOOBAuditor();
-    
+
     // first run
     assertTrue(auditor.needsRebalancing());
-    
+
     // second run
     assertTrue(auditor.needsRebalancing());
   }
-  
+
   @Test
   public void testInitializerCacheXML() {
     String configStr = "<cache xmlns=\"http://schema.pivotal.io/gemfire/cache\"                          "


[33/50] [abbrv] incubator-geode git commit: Fix for GEODE-109

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/FlushAllExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/FlushAllExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/FlushAllExecutor.java
index 7f8d4e0..40c22b6 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/FlushAllExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/FlushAllExecutor.java
@@ -2,6 +2,7 @@ package com.gemstone.gemfire.internal.redis.executor;
 
 import java.util.Map.Entry;
 
+import com.gemstone.gemfire.cache.EntryDestroyedException;
 import com.gemstone.gemfire.cache.UnsupportedOperationInTransactionException;
 import com.gemstone.gemfire.internal.redis.Coder;
 import com.gemstone.gemfire.internal.redis.Command;
@@ -15,11 +16,15 @@ public class FlushAllExecutor extends AbstractExecutor {
     if (context.hasTransaction())
       throw new UnsupportedOperationInTransactionException();
 
-    for (Entry<String, RedisDataType> e: context.getRegionCache().metaEntrySet()) {
-      String skey = e.getKey();
-      RedisDataType type = e.getValue();
-      removeEntry(Coder.stringToByteWrapper(skey), type, context);
-        
+    for (Entry<String, RedisDataType> e: context.getRegionProvider().metaEntrySet()) {
+      try {
+        String skey = e.getKey();
+        RedisDataType type = e.getValue();
+        removeEntry(Coder.stringToByteWrapper(skey), type, context);
+      } catch (EntryDestroyedException e1) {
+        continue;
+      }
+
     }
 
     command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), "OK"));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/KeysExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/KeysExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/KeysExecutor.java
index 4276e9f..2588217 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/KeysExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/KeysExecutor.java
@@ -11,7 +11,7 @@ import com.gemstone.gemfire.internal.redis.Command;
 import com.gemstone.gemfire.internal.redis.ExecutionHandlerContext;
 import com.gemstone.gemfire.internal.redis.RedisConstants;
 import com.gemstone.gemfire.internal.redis.RedisConstants.ArityDef;
-import com.gemstone.gemfire.internal.redis.executor.org.apache.hadoop.fs.GlobPattern;
+import com.gemstone.gemfire.internal.redis.org.apache.hadoop.fs.GlobPattern;
 import com.gemstone.gemfire.redis.GemFireRedisServer;
 
 public class KeysExecutor extends AbstractExecutor {
@@ -25,7 +25,7 @@ public class KeysExecutor extends AbstractExecutor {
     }
 
     String glob = Coder.bytesToString(commandElems.get(1));
-    Set<String> allKeys = context.getRegionCache().metaKeySet();
+    Set<String> allKeys = context.getRegionProvider().metaKeySet();
     List<String> matchingKeys = new ArrayList<String>();
 
     Pattern pattern;
@@ -37,8 +37,7 @@ public class KeysExecutor extends AbstractExecutor {
     }
 
     for (String key: allKeys) {
-      if (!(key.equals(GemFireRedisServer.LISTS_META_DATA_REGION) ||
-              key.equals(GemFireRedisServer.REDIS_META_DATA_REGION) ||
+      if (!(key.equals(GemFireRedisServer.REDIS_META_DATA_REGION) ||
               key.equals(GemFireRedisServer.STRING_REGION) ||
               key.equals(GemFireRedisServer.HLL_REGION))
               && pattern.matcher(key).matches())

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ListQuery.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ListQuery.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ListQuery.java
index 3980061..4ae9990 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ListQuery.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ListQuery.java
@@ -4,19 +4,19 @@ public enum ListQuery {
 
   LINDEX {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry ORDER BY key asc LIMIT $1";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE key != 'head' AND key != 'tail' ORDER BY key asc LIMIT $1";
     }
   }, LRANGE {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry ORDER BY key asc LIMIT $1";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE key != 'head' AND key != 'tail' ORDER BY key asc LIMIT $1";
     }
   }, LREMG {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value = $1 ORDER BY key asc LIMIT $2";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value = $1 AND key != 'head' AND key != 'tail' ORDER BY key asc LIMIT $2";
     }
   }, LREML {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value = $1 ORDER BY key desc LIMIT $2";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value = $1 AND key != 'head' AND key != 'tail' ORDER BY key desc LIMIT $2";
     }
   }, LREME {
     public String getQueryString(String fullpath) {
@@ -24,11 +24,11 @@ public enum ListQuery {
     }
   }, LSET {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT * FROM " + fullpath + ".keySet key ORDER BY key asc LIMIT $1";
+      return "SELECT DISTINCT * FROM " + fullpath + ".keySet key WHERE key != 'head' AND key != 'tail' ORDER BY key asc LIMIT $1";
     }
   }, LTRIM {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT * FROM " + fullpath + ".keySet key ORDER BY key asc LIMIT $1";
+      return "SELECT DISTINCT * FROM " + fullpath + ".keySet key WHERE key != 'head' AND key != 'tail' ORDER BY key asc LIMIT $1";
     }
   };
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/PersistExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/PersistExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/PersistExecutor.java
index b027c9f..85ed437 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/PersistExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/PersistExecutor.java
@@ -25,7 +25,7 @@ public class PersistExecutor extends AbstractExecutor {
 
     ByteArrayWrapper key = command.getKey();
     
-    boolean canceled = context.getRegionCache().cancelKeyExpiration(key);
+    boolean canceled = context.getRegionProvider().cancelKeyExpiration(key);
     
     if (canceled)
       command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), TIMEOUT_REMOVED));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/PingExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/PingExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/PingExecutor.java
index 9585b61..3387687 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/PingExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/PingExecutor.java
@@ -1,7 +1,7 @@
 package com.gemstone.gemfire.internal.redis.executor;
 
-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;
 
 public class PingExecutor extends AbstractExecutor {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/QuitExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/QuitExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/QuitExecutor.java
index 88f0574..2bca4fb 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/QuitExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/QuitExecutor.java
@@ -1,9 +1,9 @@
 package com.gemstone.gemfire.internal.redis.executor;
 
+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;
-import com.gemstone.gemfire.internal.redis.Coder;
 
 public class QuitExecutor extends AbstractExecutor {
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ScanExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ScanExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ScanExecutor.java
index a87c725..9257aff 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ScanExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/ScanExecutor.java
@@ -84,7 +84,7 @@ public class ScanExecutor extends AbstractScanExecutor {
     }
 
     @SuppressWarnings("unchecked")
-    List<String> returnList = (List<String>) getIteration(context.getRegionCache().metaKeySet(), matchPattern, count, cursor);
+    List<String> returnList = (List<String>) getIteration(context.getRegionProvider().metaKeySet(), matchPattern, count, cursor);
 
     command.setResponse(Coder.getScanResponse(context.getByteBufAllocator(), returnList));
   }
@@ -98,7 +98,7 @@ public class ScanExecutor extends AbstractScanExecutor {
     int numElements = 0;
     int i = -1;
     for (String key: (Collection<String>) list) {
-      if (key.equals(GemFireRedisServer.REDIS_META_DATA_REGION) || key.equals(GemFireRedisServer.LISTS_META_DATA_REGION) || key.equals(GemFireRedisServer.STRING_REGION) || key.equals(GemFireRedisServer.HLL_REGION))
+      if (key.equals(GemFireRedisServer.REDIS_META_DATA_REGION) || key.equals(GemFireRedisServer.STRING_REGION) || key.equals(GemFireRedisServer.HLL_REGION))
         continue;
       i++;
       if (beforeCursor < cursor) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/SortedSetQuery.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/SortedSetQuery.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/SortedSetQuery.java
index 97d0398..24788dd 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/SortedSetQuery.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/SortedSetQuery.java
@@ -104,67 +104,67 @@ public enum SortedSetQuery {
     }
   }, ZRBSNINFI {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score <= $1 ORDER BY value asc LIMIT $2";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score <= $1 ORDER BY entry.value asc LIMIT $2";
     }
   }, ZRBSNINF {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score < $1 ORDER BY value asc LIMIT $2";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score < $1 ORDER BY entry.value asc LIMIT $2";
     }
   }, ZRBSPINFI {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score >= $1 ORDER BY value asc LIMIT $2";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score >= $1 ORDER BY entry.value asc LIMIT $2";
     }
   }, ZRBSPINF {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score > $1 ORDER BY value asc LIMIT $2";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score > $1 ORDER BY entry.value asc LIMIT $2";
     }
   }, ZRBSSTISI {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score >= $1 AND value.score <= $2 ORDER BY value asc LIMIT $3";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score >= $1 AND entry.value.score <= $2 ORDER BY entry.value asc LIMIT $3";
     }
   }, ZRBSSTI {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score >= $1 AND value.score < $2 ORDER BY value asc LIMIT $3";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score >= $1 AND entry.value.score < $2 ORDER BY entry.value asc LIMIT $3";
     }
   }, ZRBSSI {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score > $1 AND value.score <= $2 ORDER BY value asc LIMIT $3";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score > $1 AND entry.value.score <= $2 ORDER BY entry.value asc LIMIT $3";
     }
   }, ZRBS {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score > $1 AND value.score < $2 ORDER BY value asc LIMIT $3";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score > $1 AND entry.value.score < $2 ORDER BY entry.value asc LIMIT $3";
     }
   }, ZREVRBSNINFI {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value <= $1 ORDER BY value desc, key desc LIMIT $2";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value <= $1 ORDER BY entry.value desc, entry.key desc LIMIT $2";
     }
   }, ZREVRBSNINF {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score < $1 ORDER BY value desc, key desc LIMIT $2";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score < $1 ORDER BY entry.value desc, entry.key desc LIMIT $2";
     }
   }, ZREVRBSPINFI {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score >= $1 ORDER BY value desc, key desc LIMIT $2";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score >= $1 ORDER BY entry.value desc, entry.key desc LIMIT $2";
     }
   }, ZREVRBSPINF {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score > $1 ORDER BY value desc, key desc LIMIT $2";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score > $1 ORDER BY entry.value desc, entry.key desc LIMIT $2";
     }
   }, ZREVRBSSTISI {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score >= $1 AND value.score <= $2 ORDER BY value desc, key desc LIMIT $3";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score >= $1 AND entry.value.score <= $2 ORDER BY entry.value desc, entry.key desc LIMIT $3";
     }
   }, ZREVRBSSTI {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score >= $1 AND value.score < $2 ORDER BY value desc, key desc LIMIT $3";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score >= $1 AND entry.value.score < $2 ORDER BY entry.value desc, entry.key desc LIMIT $3";
     }
   }, ZREVRBSSI {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score > $1 AND value.score <= $2 ORDER BY value desc, key desc LIMIT $3";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score > $1 AND entry.value.score <= $2 ORDER BY entry.value desc, entry.key desc LIMIT $3";
     }
   }, ZREVRBS {
     public String getQueryString(String fullpath) {
-      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE value.score > $1 AND value.score < $2 ORDER BY value desc, key desc LIMIT $3";
+      return "SELECT DISTINCT entry.key, entry.value FROM " + fullpath + ".entrySet entry WHERE entry.value.score > $1 AND entry.value.score < $2 ORDER BY entry.value desc, entry.key desc LIMIT $3";
     }
   }, ZREVRANGE {
     public String getQueryString(String fullpath) {
@@ -176,11 +176,11 @@ public enum SortedSetQuery {
     }
   }, ZRANK {
     public String getQueryString(String fullpath) {
-      return "SELECT COUNT(*) FROM " + fullpath + ".entrySet entry WHERE value < $1 OR (value = $2 AND key.compareTo($3) < 0)";
+      return "SELECT COUNT(*) FROM " + fullpath + ".entrySet entry WHERE entry.value < $1 OR (entry.value = $2 AND entry.key.compareTo($3) < 0)";
     }
   }, ZREVRANK {
     public String getQueryString(String fullpath) {
-      return "SELECT COUNT(*) FROM " + fullpath + ".entrySet entry WHERE value > $1 OR (value = $2 AND key.compareTo($3) > 0)";
+      return "SELECT COUNT(*) FROM " + fullpath + ".entrySet entry WHERE entry.value > $1 OR (entry.value = $2 AND entry.key.compareTo($3) > 0)";
     }
   };
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/TTLExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/TTLExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/TTLExecutor.java
index 91ed98c..47f2f49 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/TTLExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/TTLExecutor.java
@@ -9,7 +9,7 @@ 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.RedisDataType;
-import com.gemstone.gemfire.internal.redis.RegionCache;
+import com.gemstone.gemfire.internal.redis.RegionProvider;
 
 public class TTLExecutor extends AbstractExecutor implements Extendable {
 
@@ -27,7 +27,7 @@ public class TTLExecutor extends AbstractExecutor implements Extendable {
     }
 
     ByteArrayWrapper key = command.getKey();
-    RegionCache rC = context.getRegionCache();
+    RegionProvider rC = context.getRegionProvider();
     boolean exists = false;
     RedisDataType val = rC.getRedisDataType(key);
     if (val != null)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/TypeExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/TypeExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/TypeExecutor.java
index dc4172f..75e5e53 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/TypeExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/TypeExecutor.java
@@ -21,7 +21,7 @@ public class TypeExecutor extends AbstractExecutor {
 
     ByteArrayWrapper key = command.getKey();
 
-    RedisDataType type = context.getRegionCache().getRedisDataType(key);
+    RedisDataType type = context.getRegionProvider().getRedisDataType(key);
 
     if (type == null)
       command.setResponse(Coder.getBulkStringResponse(context.getByteBufAllocator(), "none"));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/UnkownExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/UnkownExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/UnkownExecutor.java
index 524003d..fd64c39 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/UnkownExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/UnkownExecutor.java
@@ -1,9 +1,9 @@
 package com.gemstone.gemfire.internal.redis.executor;
 
+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;
-import com.gemstone.gemfire.internal.redis.Coder;
 
 public class UnkownExecutor extends AbstractExecutor {
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HDelExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HDelExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HDelExecutor.java
index bd8777b..11122fe 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HDelExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HDelExecutor.java
@@ -43,7 +43,7 @@ public class HDelExecutor extends HashExecutor {
         numDeleted++;
     }
     if (keyRegion.isEmpty()) {
-      context.getRegionCache().removeKey(key, RedisDataType.REDIS_HASH);
+      context.getRegionProvider().removeKey(key, RedisDataType.REDIS_HASH);
     }
     command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), numDeleted));
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HGetAllExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HGetAllExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HGetAllExecutor.java
index 96eb5a9..ae8100e 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HGetAllExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HGetAllExecutor.java
@@ -1,16 +1,17 @@
 package com.gemstone.gemfire.internal.redis.executor.hash;
 
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.internal.redis.ByteArrayWrapper;
+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;
 import com.gemstone.gemfire.internal.redis.RedisDataType;
-import com.gemstone.gemfire.internal.redis.Coder;
 
 public class HGetAllExecutor extends HashExecutor {
 
@@ -33,7 +34,7 @@ public class HGetAllExecutor extends HashExecutor {
       return;
     }
 
-    Set<Map.Entry<ByteArrayWrapper,ByteArrayWrapper>> entries = keyRegion.entrySet();
+    Collection<Map.Entry<ByteArrayWrapper,ByteArrayWrapper>> entries = new ArrayList(keyRegion.entrySet()); // This creates a CopyOnRead behavior
    
    if (entries.isEmpty()) {
      command.setResponse(Coder.getEmptyArrayResponse(context.getByteBufAllocator()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HKeysExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HKeysExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HKeysExecutor.java
index 988ac37..5c401ff 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HKeysExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HKeysExecutor.java
@@ -1,15 +1,16 @@
 package com.gemstone.gemfire.internal.redis.executor.hash;
 
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.internal.redis.ByteArrayWrapper;
+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.RedisDataType;
-import com.gemstone.gemfire.internal.redis.Coder;
 import com.gemstone.gemfire.internal.redis.RedisConstants.ArityDef;
+import com.gemstone.gemfire.internal.redis.RedisDataType;
 
 public class HKeysExecutor extends HashExecutor {
 
@@ -32,7 +33,7 @@ public class HKeysExecutor extends HashExecutor {
       return;
     }
 
-   Set<ByteArrayWrapper> keys = keyRegion.keySet();
+   Set<ByteArrayWrapper> keys = new HashSet(keyRegion.keySet());
    
    if (keys.isEmpty()) {
      command.setResponse(Coder.getEmptyArrayResponse(context.getByteBufAllocator()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HScanExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HScanExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HScanExecutor.java
index fb0ddc0..ded681a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HScanExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HScanExecutor.java
@@ -2,6 +2,7 @@ package com.gemstone.gemfire.internal.redis.executor.hash;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.regex.Pattern;
@@ -30,7 +31,7 @@ public class HScanExecutor extends AbstractScanExecutor {
 
     ByteArrayWrapper key = command.getKey();
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, ByteArrayWrapper> keyRegion = (Region<ByteArrayWrapper, ByteArrayWrapper>) context.getRegionCache().getRegion(key);
+    Region<ByteArrayWrapper, ByteArrayWrapper> keyRegion = (Region<ByteArrayWrapper, ByteArrayWrapper>) context.getRegionProvider().getRegion(key);
     checkDataType(key, RedisDataType.REDIS_HASH, context);
     if (keyRegion == null) {
       command.setResponse(Coder.getScanResponse(context.getByteBufAllocator(), new ArrayList<String>()));
@@ -100,7 +101,7 @@ public class HScanExecutor extends AbstractScanExecutor {
       return;
     }
 
-    List<Object> returnList = getIteration(keyRegion.entrySet(), matchPattern, count, cursor);
+    List<Object> returnList = getIteration(new HashSet(keyRegion.entrySet()), matchPattern, count, cursor);
 
     command.setResponse(Coder.getScanResponse(context.getByteBufAllocator(), returnList));
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HValsExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HValsExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HValsExecutor.java
index a17b054..8d5edaf 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HValsExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HValsExecutor.java
@@ -1,15 +1,16 @@
 package com.gemstone.gemfire.internal.redis.executor.hash;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.internal.redis.ByteArrayWrapper;
+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.RedisDataType;
-import com.gemstone.gemfire.internal.redis.Coder;
 import com.gemstone.gemfire.internal.redis.RedisConstants.ArityDef;
+import com.gemstone.gemfire.internal.redis.RedisDataType;
 
 public class HValsExecutor extends HashExecutor {
 
@@ -31,8 +32,8 @@ public class HValsExecutor extends HashExecutor {
       command.setResponse(Coder.getEmptyArrayResponse(context.getByteBufAllocator()));
       return;
     }
-
-    Collection<ByteArrayWrapper> vals = keyRegion.values();
+    
+    Collection<ByteArrayWrapper> vals = new ArrayList(keyRegion.values());
 
     if (vals.isEmpty()) {
       command.setResponse(Coder.getEmptyArrayResponse(context.getByteBufAllocator()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HashExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HashExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HashExecutor.java
index 0abe42f..94b5115 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HashExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hash/HashExecutor.java
@@ -12,12 +12,12 @@ public abstract class HashExecutor extends AbstractExecutor {
   
   @SuppressWarnings("unchecked")
   protected Region<ByteArrayWrapper, ByteArrayWrapper> getOrCreateRegion(ExecutionHandlerContext context, ByteArrayWrapper key, RedisDataType type) {
-   return (Region<ByteArrayWrapper, ByteArrayWrapper>) context.getRegionCache().getOrCreateRegion(key, type, context);
+   return (Region<ByteArrayWrapper, ByteArrayWrapper>) context.getRegionProvider().getOrCreateRegion(key, type, context);
   }
   
   @SuppressWarnings("unchecked")
   protected Region<ByteArrayWrapper, ByteArrayWrapper> getRegion(ExecutionHandlerContext context, ByteArrayWrapper key) {
-   return (Region<ByteArrayWrapper, ByteArrayWrapper>) context.getRegionCache().getRegion(key);
+   return (Region<ByteArrayWrapper, ByteArrayWrapper>) context.getRegionProvider().getRegion(key);
   }
   
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/HllExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/HllExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/HllExecutor.java
index 1744f37..2f2a963 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/HllExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/HllExecutor.java
@@ -13,7 +13,7 @@ public abstract class HllExecutor extends AbstractExecutor {
   public static final Integer DEFAULT_HLL_SPARSE = 32;
   
   protected final void checkAndSetDataType(ByteArrayWrapper key, ExecutionHandlerContext context) {
-    Object oldVal = context.getRegionCache().metaPutIfAbsent(key, RedisDataType.REDIS_HLL);
+    Object oldVal = context.getRegionProvider().metaPutIfAbsent(key, RedisDataType.REDIS_HLL);
     if (oldVal == RedisDataType.REDIS_PROTECTED)
       throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is protected");
     if (oldVal != null && oldVal != RedisDataType.REDIS_HLL)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFAddExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFAddExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFAddExecutor.java
index 6177a0a..1fbe9e1 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFAddExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFAddExecutor.java
@@ -22,7 +22,7 @@ public class PFAddExecutor extends HllExecutor {
 
     ByteArrayWrapper key = command.getKey();
     checkAndSetDataType(key, context);
-    Region<ByteArrayWrapper, HyperLogLogPlus> keyRegion = context.getRegionCache().gethLLRegion();
+    Region<ByteArrayWrapper, HyperLogLogPlus> keyRegion = context.getRegionProvider().gethLLRegion();
 
     HyperLogLogPlus hll = keyRegion.get(key);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFCountExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFCountExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFCountExecutor.java
index d056907..bdced51 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFCountExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFCountExecutor.java
@@ -22,7 +22,7 @@ public class PFCountExecutor extends HllExecutor {
       return;
     }
 
-    Region<ByteArrayWrapper, HyperLogLogPlus> keyRegion = context.getRegionCache().gethLLRegion();
+    Region<ByteArrayWrapper, HyperLogLogPlus> keyRegion = context.getRegionProvider().gethLLRegion();
 
     List<HyperLogLogPlus> hlls = new ArrayList<HyperLogLogPlus>();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFMergeExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFMergeExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFMergeExecutor.java
index cc1ccf7..f646791 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFMergeExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/hll/PFMergeExecutor.java
@@ -24,7 +24,7 @@ public class PFMergeExecutor extends HllExecutor {
 
     ByteArrayWrapper destKey = command.getKey();
     checkAndSetDataType(destKey, context);
-    Region<ByteArrayWrapper, HyperLogLogPlus> keyRegion = context.getRegionCache().gethLLRegion();
+    Region<ByteArrayWrapper, HyperLogLogPlus> keyRegion = context.getRegionProvider().gethLLRegion();
     HyperLogLogPlus mergedHLL = keyRegion.get(destKey);
     if (mergedHLL == null)
       mergedHLL = new HyperLogLogPlus(DEFAULT_HLL_DENSE);
@@ -38,7 +38,7 @@ public class PFMergeExecutor extends HllExecutor {
         hlls.add(h);
     }
     if (hlls.isEmpty()) {
-      context.getRegionCache().removeKey(destKey);
+      context.getRegionProvider().removeKey(destKey);
       command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), "OK"));
       return;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LIndexExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LIndexExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LIndexExecutor.java
index c2064a1..997f1af 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LIndexExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LIndexExecutor.java
@@ -38,7 +38,7 @@ public class LIndexExecutor extends ListExecutor {
       return;
     }
 
-    int listSize = keyRegion.size();
+    int listSize = keyRegion.size() - LIST_EMPTY_SIZE;
 
     Integer redisIndex;
 
@@ -90,7 +90,7 @@ public class LIndexExecutor extends ListExecutor {
 
     Query query = getQuery(key, ListQuery.LINDEX, context);
 
-    Object[] params = {new Integer(index + 1)};
+    Object[] params = {Integer.valueOf(index + 1)};
 
     SelectResults<?> results = (SelectResults<?>) query.execute(params);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LLenExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LLenExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LLenExecutor.java
index 224cf60..63f0e1a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LLenExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LLenExecutor.java
@@ -35,7 +35,7 @@ public class LLenExecutor extends ListExecutor {
       return;
     }
     
-    listSize = keyRegion.size();
+    listSize = keyRegion.size() - LIST_EMPTY_SIZE;
 
     command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), listSize));
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LRangeExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LRangeExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LRangeExecutor.java
index a0b9901..84b5193 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LRangeExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LRangeExecutor.java
@@ -43,7 +43,7 @@ public class LRangeExecutor extends ListExecutor {
       return;
     }
 
-    int listSize = keyRegion.size();
+    int listSize = keyRegion.size() - LIST_EMPTY_SIZE;
     if (listSize == 0) {
       command.setResponse(Coder.getEmptyArrayResponse(context.getByteBufAllocator()));
       return;
@@ -70,7 +70,7 @@ public class LRangeExecutor extends ListExecutor {
     
     List<Struct> range;
     try {
-      range = getRange(context, key, redisStart, redisStop);
+      range = getRange(context, key, redisStart, redisStop, keyRegion);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -81,14 +81,12 @@ public class LRangeExecutor extends ListExecutor {
       command.setResponse(Coder.getBulkStringArrayResponseOfValues(context.getByteBufAllocator(), range));
   }
 
-  private List<Struct> getRange(ExecutionHandlerContext context, ByteArrayWrapper key, int start, int stop) throws Exception {
+  private List<Struct> getRange(ExecutionHandlerContext context, ByteArrayWrapper key, int start, int stop, Region r) throws Exception {
 
     Query query = getQuery(key, ListQuery.LRANGE, context);
 
-    Object[] params = {new Integer(stop + 1)};
-
+    Object[] params = {Integer.valueOf(stop + 1)};
     SelectResults<Struct> results = (SelectResults<Struct>) query.execute(params);
-
     int size = results.size();
     if (results == null || size <= start) {
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LRemExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LRemExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LRemExecutor.java
index 58daba3..8a50b4f 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LRemExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LRemExecutor.java
@@ -79,10 +79,10 @@ public class LRemExecutor extends ListExecutor {
     Query query;
     if (count > 0) {
       query = getQuery(key, ListQuery.LREMG, context);
-      params = new Object[]{value, new Integer(count)};
+      params = new Object[]{value, Integer.valueOf(count)};
     } else if (count < 0) {
       query = getQuery(key, ListQuery.LREML, context);
-      params = new Object[]{value, new Integer(-count)};
+      params = new Object[]{value, Integer.valueOf(-count)};
     } else {
       query = getQuery(key, ListQuery.LREME, context);
       params = new Object[]{value};

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LSetExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LSetExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LSetExecutor.java
index ae09878..0668f02 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LSetExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LSetExecutor.java
@@ -52,7 +52,7 @@ public class LSetExecutor extends ListExecutor {
       return;
     }
 
-    int listSize = keyRegion.size();
+    int listSize = keyRegion.size() - LIST_EMPTY_SIZE;
     if (index < 0)
       index += listSize;
     if (index < 0 || index > listSize) {
@@ -79,7 +79,7 @@ public class LSetExecutor extends ListExecutor {
   private Integer getIndexKey(ExecutionHandlerContext context, ByteArrayWrapper key, int index) throws Exception {
     Query query = getQuery(key, ListQuery.LSET, context);
 
-    Object[] params = {new Integer(index + 1)};
+    Object[] params = {Integer.valueOf(index + 1)};
     
     SelectResults<Integer> results = (SelectResults<Integer>) query.execute(params);
     int size = results.size();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LTrimExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LTrimExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LTrimExecutor.java
index cb3e9a4..11ae338 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LTrimExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/LTrimExecutor.java
@@ -40,16 +40,16 @@ public class LTrimExecutor extends ListExecutor {
 
 
     checkDataType(key, RedisDataType.REDIS_LIST, context);
-    Region<Integer, ByteArrayWrapper> keyRegion = getRegion(context, key);
+    Region keyRegion = getRegion(context, key);
 
     if (keyRegion == null) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ERROR_KEY_NOT_EXISTS));
       return;
     }
 
-    int listSize = keyRegion.size();
+    int listSize = keyRegion.size() - LIST_EMPTY_SIZE;
     if (listSize == 0) {
-      command.setResponse(Coder.getEmptyArrayResponse(context.getByteBufAllocator()));
+      command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), SUCCESS));
       return;
     }
 
@@ -70,37 +70,35 @@ public class LTrimExecutor extends ListExecutor {
       command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), SUCCESS));
       return;
     } else if (redisStart == 0 && redisStop < redisStart) {
-      context.getRegionCache().removeKey(key, RedisDataType.REDIS_LIST);
+      context.getRegionProvider().removeKey(key, RedisDataType.REDIS_LIST);
       command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), SUCCESS));
       return;
     }
 
     List<Integer> keepList;
     try {
-      keepList = getRange(context, key, redisStart, redisStop);
+      keepList = getRange(context, key, redisStart, redisStop, keyRegion);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
     
-    for (Integer keyElement: keyRegion.keySet()) {
-      if (!keepList.contains(keyElement))
+    for (Object keyElement: keyRegion.keySet()) {
+      if (!keepList.contains(keyElement) && keyElement instanceof Integer)
         keyRegion.remove(keyElement);
     }
     
     // Reset indexes in meta data region
-    Region<String, Integer> meta = context.getRegionCache().getListsMetaRegion();
-    meta.put(key + "head", keepList.get(0));
-    meta.put(key + "tail", keepList.get(keepList.size() - 1));
+    keyRegion.put("head", keepList.get(0));
+    keyRegion.put("tail", keepList.get(keepList.size() - 1));
     command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), SUCCESS));
   }
 
-  private List<Integer> getRange(ExecutionHandlerContext context, ByteArrayWrapper key, int start, int stop) throws Exception {
+  private List<Integer> getRange(ExecutionHandlerContext context, ByteArrayWrapper key, int start, int stop, Region r) throws Exception {
     Query query = getQuery(key, ListQuery.LTRIM, context);
 
-    Object[] params = {new Integer(stop + 1)};
-
+    Object[] params = {Integer.valueOf(stop + 1)};
+    
     SelectResults<Integer> results = (SelectResults<Integer>) query.execute(params);
-
     if (results == null || results.size() <= start) {
       return null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/ListExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/ListExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/ListExecutor.java
index 5b8e0cc..ebb36a1 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/ListExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/ListExecutor.java
@@ -3,6 +3,7 @@ package com.gemstone.gemfire.internal.redis.executor.list;
 import java.util.List;
 
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.query.QueryService;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.redis.ByteArrayWrapper;
@@ -13,6 +14,8 @@ import com.gemstone.gemfire.internal.redis.executor.AbstractExecutor;
 
 public abstract class ListExecutor extends AbstractExecutor {
 
+  protected static final int LIST_EMPTY_SIZE = 2;
+  
   protected static enum ListDirection {LEFT, RIGHT};
 
   protected final static QueryService getQueryService() {
@@ -22,12 +25,12 @@ public abstract class ListExecutor extends AbstractExecutor {
   @SuppressWarnings("unchecked")
   @Override
   protected Region<Integer, ByteArrayWrapper> getOrCreateRegion(ExecutionHandlerContext context, ByteArrayWrapper key, RedisDataType type) {
-    return (Region<Integer, ByteArrayWrapper>) context.getRegionCache().getOrCreateRegion(key, type, context);
+    return (Region<Integer, ByteArrayWrapper>) context.getRegionProvider().getOrCreateRegion(key, type, context);
   }
   
   @SuppressWarnings("unchecked")
   protected Region<Integer, ByteArrayWrapper> getRegion(ExecutionHandlerContext context, ByteArrayWrapper key) {
-    return (Region<Integer, ByteArrayWrapper>) context.getRegionCache().getRegion(key);
+    return (Region<Integer, ByteArrayWrapper>) context.getRegionProvider().getRegion(key);
   }
 
   /**
@@ -45,14 +48,12 @@ public abstract class ListExecutor extends AbstractExecutor {
    * @param context Context of this push
    */
   protected void pushElements(ByteArrayWrapper key, List<byte[]> commandElems, int startIndex, int endIndex,
-      Region<Integer, ByteArrayWrapper> keyRegion, ListDirection pushType, ExecutionHandlerContext context) {
-    Region<String, Integer> meta = context.getRegionCache().getListsMetaRegion();
-
-    String indexKey = pushType == ListDirection.LEFT ? key + "head" : key + "tail";
-    String oppositeKey = pushType == ListDirection.RIGHT ? key + "head" : key + "tail";
-    Integer index = meta.get(indexKey);
-    Integer opp = meta.get(oppositeKey);
+      Region keyRegion, ListDirection pushType, ExecutionHandlerContext context) {
 
+    String indexKey = pushType == ListDirection.LEFT ? "head" : "tail";
+    String oppositeKey = pushType == ListDirection.RIGHT ? "head" : "tail";
+    Integer index = (Integer) keyRegion.get(indexKey);
+    Integer opp = (Integer) keyRegion.get(oppositeKey);
     if (index != opp)
       index += pushType == ListDirection.LEFT ? -1 : 1; // Subtract index if left push, add if right push
 
@@ -76,8 +77,9 @@ public abstract class ListExecutor extends AbstractExecutor {
       Object oldValue;
       do {
         oldValue = keyRegion.putIfAbsent(index, wrapper);
-        if (oldValue != null)
+        if (oldValue != null) {
           index += pushType == ListDirection.LEFT ? -1 : 1; // Subtract index if left push, add if right push
+        }
       } while (oldValue != null);
 
       /**
@@ -119,9 +121,9 @@ public abstract class ListExecutor extends AbstractExecutor {
 
       boolean indexSet = false;
       do {
-        Integer existingIndex = meta.get(indexKey);
+        Integer existingIndex = (Integer) keyRegion.get(indexKey);
         if ((pushType == ListDirection.RIGHT && existingIndex < index) || (pushType == ListDirection.LEFT && existingIndex > index))
-          indexSet = meta.replace(indexKey, existingIndex, index);
+          indexSet = keyRegion.replace(indexKey, existingIndex, index);
         else
           break;
       } while (!indexSet);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PopExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PopExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PopExecutor.java
index af3f8b0..e501de0 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PopExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PopExecutor.java
@@ -3,6 +3,7 @@ package com.gemstone.gemfire.internal.redis.executor.list;
 import java.util.List;
 
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.internal.redis.ByteArrayWrapper;
 import com.gemstone.gemfire.internal.redis.Coder;
 import com.gemstone.gemfire.internal.redis.Command;
@@ -24,16 +25,15 @@ public abstract class PopExecutor extends ListExecutor implements Extendable {
     ByteArrayWrapper key = command.getKey();
 
     checkDataType(key, RedisDataType.REDIS_LIST, context);
-    Region<Integer, ByteArrayWrapper> keyRegion = getRegion(context, key);
-    Region<String, Integer> meta = context.getRegionCache().getListsMetaRegion();
+    Region keyRegion = getRegion(context, key);
 
-    if (keyRegion == null || keyRegion.size() == 0) {
+    if (keyRegion == null || keyRegion.size() == LIST_EMPTY_SIZE) {
       command.setResponse(Coder.getNilResponse(context.getByteBufAllocator()));
       return;
     }
 
-    String indexKey = popType() == ListDirection.LEFT ? key + "head" : key + "tail";
-    String oppositeKey = popType() == ListDirection.RIGHT ? key + "head" : key + "tail";
+    String indexKey = popType() == ListDirection.LEFT ? "head" : "tail";
+    String oppositeKey = popType() == ListDirection.RIGHT ? "head" : "tail";
     Integer index = 0;
     int originalIndex = index;
     int incr = popType() == ListDirection.LEFT ? 1 : -1;
@@ -49,10 +49,11 @@ public abstract class PopExecutor extends ListExecutor implements Extendable {
     
     boolean indexChanged = false;
     do {
-      index = meta.get(indexKey);
-      if (index == meta.get(oppositeKey))
+      index = (Integer) keyRegion.get(indexKey);
+      Integer opp = (Integer) keyRegion.get(oppositeKey);
+      if (index.equals(opp))
         break;
-      indexChanged = meta.replace(indexKey, index, index + incr);
+      indexChanged = keyRegion.replace(indexKey, index, index + incr);
     } while(!indexChanged);
     
     /**
@@ -67,7 +68,7 @@ public abstract class PopExecutor extends ListExecutor implements Extendable {
     boolean removed = false;
     int i = 0;
     do {
-      valueWrapper = keyRegion.get(index);
+      valueWrapper = (ByteArrayWrapper) keyRegion.get(index);
       if (valueWrapper != null)
         removed = keyRegion.remove(index, valueWrapper);
       
@@ -116,12 +117,12 @@ public abstract class PopExecutor extends ListExecutor implements Extendable {
        */
       
       index += incr;
-      int metaIndex = meta.get(indexKey);
+      Integer metaIndex = (Integer) keyRegion.get(indexKey);
       if (i < 1 && (popType() == ListDirection.LEFT && metaIndex < originalIndex ||
           popType() == ListDirection.RIGHT && metaIndex > originalIndex))
         index = metaIndex;
       i++;
-    } while (!removed && keyRegion.size() != 0);
+    } while (!removed && keyRegion.size() != LIST_EMPTY_SIZE);
     if (valueWrapper != null)
       command.setResponse(Coder.getBulkStringResponse(context.getByteBufAllocator(), valueWrapper.toBytes()));
     else

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PushExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PushExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PushExecutor.java
index 4414e0a..fdc7208 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PushExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PushExecutor.java
@@ -29,7 +29,7 @@ public abstract class PushExecutor extends PushXExecutor implements Extendable {
 
     Region<Integer, ByteArrayWrapper> keyRegion = getOrCreateRegion(context, key, RedisDataType.REDIS_LIST);
     pushElements(key, commandElems, START_VALUES_INDEX, commandElems.size(), keyRegion, pushType(), context);
-    int listSize = keyRegion.size();
+    int listSize = keyRegion.size() - LIST_EMPTY_SIZE;
     command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), listSize));
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PushXExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PushXExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PushXExecutor.java
index fbbad1e..ac89522 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PushXExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/list/PushXExecutor.java
@@ -33,7 +33,7 @@ public abstract class PushXExecutor extends ListExecutor implements Extendable {
     checkDataType(key, RedisDataType.REDIS_LIST, context);    
     pushElements(key, commandElems, 2, 3, keyRegion, pushType(), context);
     
-    int listSize = keyRegion.size();
+    int listSize = keyRegion.size() - LIST_EMPTY_SIZE;
 
     command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), listSize));
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/org/apache/hadoop/fs/GlobPattern.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/org/apache/hadoop/fs/GlobPattern.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/org/apache/hadoop/fs/GlobPattern.java
deleted file mode 100644
index b6cd906..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/org/apache/hadoop/fs/GlobPattern.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package com.gemstone.gemfire.internal.redis.executor.org.apache.hadoop.fs;
-
-import java.util.regex.Pattern;
-import java.util.regex.PatternSyntaxException;
-
-/**
- * A class for POSIX glob pattern with brace expansions.
- */
-public class GlobPattern {
-  private static final char BACKSLASH = '\\';
-  private Pattern compiled;
-  private boolean hasWildcard = false;
-
-  /**
-   * Construct the glob pattern object with a glob pattern string
-   * @param globPattern the glob pattern string
-   */
-  public GlobPattern(String globPattern) {
-    set(globPattern);
-  }
-
-  /**
-   * @return the compiled pattern
-   */
-  public Pattern compiled() {
-    return compiled;
-  }
-
-  /**
-   * Compile glob pattern string
-   * @param globPattern the glob pattern
-   * @return the pattern object
-   */
-  public static Pattern compile(String globPattern) {
-    return new GlobPattern(globPattern).compiled();
-  }
-
-  /**
-   * Match input against the compiled glob pattern
-   * @param s input chars
-   * @return true for successful matches
-   */
-  public boolean matches(CharSequence s) {
-    return compiled.matcher(s).matches();
-  }
-
-  /**
-   * Set and compile a glob pattern
-   * @param glob  the glob pattern string
-   */
-  public void set(String glob) {
-    StringBuilder regex = new StringBuilder();
-    int setOpen = 0;
-    int curlyOpen = 0;
-    int len = glob.length();
-    hasWildcard = false;
-
-    for (int i = 0; i < len; i++) {
-      char c = glob.charAt(i);
-
-      switch (c) {
-      case BACKSLASH:
-        if (++i >= len) {
-          error("Missing escaped character", glob, i);
-        }
-        regex.append(c).append(glob.charAt(i));
-        continue;
-      case '.':
-      case '$':
-      case '(':
-      case ')':
-      case '|':
-      case '+':
-        // escape regex special chars that are not glob special chars
-        regex.append(BACKSLASH);
-        break;
-      case '*':
-        regex.append('.');
-        hasWildcard = true;
-        break;
-      case '?':
-        regex.append('.');
-        hasWildcard = true;
-        continue;
-      case '{': // start of a group
-        regex.append("(?:"); // non-capturing
-        curlyOpen++;
-        hasWildcard = true;
-        continue;
-      case ',':
-        regex.append(curlyOpen > 0 ? '|' : c);
-        continue;
-      case '}':
-        if (curlyOpen > 0) {
-          // end of a group
-          curlyOpen--;
-          regex.append(")");
-          continue;
-        }
-        break;
-      case '[':
-        if (setOpen > 0) {
-          error("Unclosed character class", glob, i);
-        }
-        setOpen++;
-        hasWildcard = true;
-        break;
-      case '^': // ^ inside [...] can be unescaped
-        if (setOpen == 0) {
-          regex.append(BACKSLASH);
-        }
-        break;
-      case '!': // [! needs to be translated to [^
-        regex.append(setOpen > 0 && '[' == glob.charAt(i - 1) ? '^' : '!');
-        continue;
-      case ']':
-        // Many set errors like [][] could not be easily detected here,
-        // as []], []-] and [-] are all valid POSIX glob and java regex.
-        // We'll just let the regex compiler do the real work.
-        setOpen = 0;
-        break;
-      default:
-      }
-      regex.append(c);
-    }
-
-    if (setOpen > 0) {
-      error("Unclosed character class", glob, len);
-    }
-    if (curlyOpen > 0) {
-      error("Unclosed group", glob, len);
-    }
-    compiled = Pattern.compile(regex.toString());
-  }
-
-  /**
-   * @return true if this is a wildcard pattern (with special chars)
-   */
-  public boolean hasWildcard() {
-    return hasWildcard;
-  }
-
-  private static void error(String message, String pattern, int pos) {
-    throw new PatternSyntaxException(message, pattern, pos);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SAddExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SAddExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SAddExecutor.java
index aa4a5f2..fd33104 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SAddExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SAddExecutor.java
@@ -25,7 +25,7 @@ public class SAddExecutor extends SetExecutor {
 
     ByteArrayWrapper key = command.getKey();
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionCache().getOrCreateRegion(key, RedisDataType.REDIS_SET, context);
+    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionProvider().getOrCreateRegion(key, RedisDataType.REDIS_SET, context);
 
     if (commandElems.size() >= 4) {
       Map<ByteArrayWrapper, Boolean> entries = new HashMap<ByteArrayWrapper, Boolean>();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SCardExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SCardExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SCardExecutor.java
index 4677565..5b8d842 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SCardExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SCardExecutor.java
@@ -26,7 +26,7 @@ public class SCardExecutor extends SetExecutor {
 
     ByteArrayWrapper key = command.getKey();
     checkDataType(key, RedisDataType.REDIS_SET, context);
-    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionCache().getRegion(key);
+    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionProvider().getRegion(key);
 
     if (keyRegion == null) {
       command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), NOT_EXISTS));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SIsMemberExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SIsMemberExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SIsMemberExecutor.java
index 355f70f..33de3aa 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SIsMemberExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SIsMemberExecutor.java
@@ -30,7 +30,7 @@ public class SIsMemberExecutor extends SetExecutor {
     
     checkDataType(key, RedisDataType.REDIS_SET, context);
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionCache().getRegion(key);
+    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionProvider().getRegion(key);
 
     if (keyRegion == null) {
       command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), NOT_EXISTS));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SMembersExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SMembersExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SMembersExecutor.java
index bc68033..9536fa5 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SMembersExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SMembersExecutor.java
@@ -1,15 +1,16 @@
 package com.gemstone.gemfire.internal.redis.executor.set;
 
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.internal.redis.ByteArrayWrapper;
+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;
 import com.gemstone.gemfire.internal.redis.RedisDataType;
-import com.gemstone.gemfire.internal.redis.Coder;
 
 public class SMembersExecutor extends SetExecutor {
 
@@ -25,14 +26,14 @@ public class SMembersExecutor extends SetExecutor {
     ByteArrayWrapper key = command.getKey();
     checkDataType(key, RedisDataType.REDIS_SET, context);
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionCache().getRegion(key);
+    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionProvider().getRegion(key);
     
     if (keyRegion == null) {
       command.setResponse(Coder.getEmptyArrayResponse(context.getByteBufAllocator()));
       return;
     }
     
-    Set<ByteArrayWrapper> members = keyRegion.keySet();
+    Set<ByteArrayWrapper> members = new HashSet(keyRegion.keySet()); // Emulate copy on read
     
     command.setResponse(Coder.getBulkStringArrayResponse(context.getByteBufAllocator(), members));
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SMoveExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SMoveExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SMoveExecutor.java
index f7f5c86..a73bbd9 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SMoveExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SMoveExecutor.java
@@ -32,7 +32,7 @@ public class SMoveExecutor extends SetExecutor {
     checkDataType(source, RedisDataType.REDIS_SET, context);
     checkDataType(destination, RedisDataType.REDIS_SET, context);
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, Boolean> sourceRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionCache().getRegion(source);
+    Region<ByteArrayWrapper, Boolean> sourceRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionProvider().getRegion(source);
 
     if (sourceRegion == null) {
       command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), NOT_MOVED));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SPopExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SPopExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SPopExecutor.java
index 73e59de..6461f34 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SPopExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SPopExecutor.java
@@ -23,7 +23,7 @@ public class SPopExecutor extends SetExecutor {
     
     ByteArrayWrapper key = command.getKey();
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionCache().getRegion(key);
+    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionProvider().getRegion(key);
     if (keyRegion == null || keyRegion.isEmpty()) {
       command.setResponse(Coder.getNilResponse(context.getByteBufAllocator()));
       return;
@@ -37,7 +37,7 @@ public class SPopExecutor extends SetExecutor {
     
     keyRegion.remove(pop);
     if (keyRegion.isEmpty()) {
-      context.getRegionCache().removeKey(key);
+      context.getRegionProvider().removeKey(key);
     }
     command.setResponse(Coder.getBulkStringResponse(context.getByteBufAllocator(), pop.toBytes()));
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SRandMemberExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SRandMemberExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SRandMemberExecutor.java
index 5494b30..34954c5 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SRandMemberExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SRandMemberExecutor.java
@@ -28,7 +28,7 @@ public class SRandMemberExecutor extends SetExecutor {
 
     ByteArrayWrapper key = command.getKey();
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionCache().getRegion(key);
+    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionProvider().getRegion(key);
 
     int count = 1;
 
@@ -49,7 +49,7 @@ public class SRandMemberExecutor extends SetExecutor {
     int members = keyRegion.size();
 
     if (members <= count && count != 1) {
-      command.setResponse(Coder.getBulkStringArrayResponse(context.getByteBufAllocator(), keyRegion.keySet()));
+      command.setResponse(Coder.getBulkStringArrayResponse(context.getByteBufAllocator(), new HashSet<ByteArrayWrapper>(keyRegion.keySet())));
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SRemExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SRemExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SRemExecutor.java
index f85e095..482d83e 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SRemExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SRemExecutor.java
@@ -26,7 +26,7 @@ public class SRemExecutor extends SetExecutor {
     ByteArrayWrapper key = command.getKey();
     checkDataType(key, RedisDataType.REDIS_SET, context);
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionCache().getRegion(key);
+    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionProvider().getRegion(key);
     
     if (keyRegion == null) {
       command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), NONE_REMOVED));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SScanExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SScanExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SScanExecutor.java
index 87605df..7c0133b 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SScanExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SScanExecutor.java
@@ -30,7 +30,7 @@ public class SScanExecutor extends AbstractScanExecutor {
     ByteArrayWrapper key = command.getKey();
     checkDataType(key, RedisDataType.REDIS_SET, context);
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionCache().getRegion(key);
+    Region<ByteArrayWrapper, Boolean> keyRegion = (Region<ByteArrayWrapper, Boolean>) context.getRegionProvider().getRegion(key);
     if (keyRegion == null) {
       command.setResponse(Coder.getScanResponse(context.getByteBufAllocator(), new ArrayList<String>()));
       return;
@@ -96,7 +96,7 @@ public class SScanExecutor extends AbstractScanExecutor {
     }
 
     @SuppressWarnings("unchecked")
-    List<ByteArrayWrapper> returnList = (List<ByteArrayWrapper>) getIteration(keyRegion.keySet(), matchPattern, count, cursor);
+    List<ByteArrayWrapper> returnList = (List<ByteArrayWrapper>) getIteration(new ArrayList(keyRegion.keySet()), matchPattern, count, cursor);
 
     command.setResponse(Coder.getScanResponse(context.getByteBufAllocator(), returnList));
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SetOpExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SetOpExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SetOpExecutor.java
index f2cfd9d..7cd1932 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SetOpExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/set/SetOpExecutor.java
@@ -14,7 +14,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.RedisDataType;
-import com.gemstone.gemfire.internal.redis.RegionCache;
+import com.gemstone.gemfire.internal.redis.RegionProvider;
 
 public abstract class SetOpExecutor extends SetExecutor implements Extendable {
 
@@ -27,12 +27,11 @@ public abstract class SetOpExecutor extends SetExecutor implements Extendable {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), getArgsError()));
       return;
     }
-    RegionCache rC = context.getRegionCache();
+    RegionProvider rC = context.getRegionProvider();
     ByteArrayWrapper destination = null;
     if (isStorage())
       destination = command.getKey();
 
-
     ByteArrayWrapper firstSetKey = new ByteArrayWrapper(commandElems.get(setsStartIndex++));
     if (!isStorage())
       checkDataType(firstSetKey, RedisDataType.REDIS_SET, context);
@@ -53,10 +52,8 @@ public abstract class SetOpExecutor extends SetExecutor implements Extendable {
     }
     if (setList.isEmpty()) {
       if (isStorage()) {
-        if (firstSet == null) {
           command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), 0));
-          context.getRegionCache().removeKey(destination);
-        }
+          context.getRegionProvider().removeKey(destination);
       } else {
         if (firstSet == null)
           command.setResponse(Coder.getNilResponse(context.getByteBufAllocator()));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/SortedSetExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/SortedSetExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/SortedSetExecutor.java
index 7443019..f193727 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/SortedSetExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/SortedSetExecutor.java
@@ -12,13 +12,13 @@ public abstract class SortedSetExecutor extends AbstractExecutor {
   @Override
   protected Region<ByteArrayWrapper, DoubleWrapper> getOrCreateRegion(ExecutionHandlerContext context, ByteArrayWrapper key, RedisDataType type) {
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, DoubleWrapper> r = (Region<ByteArrayWrapper, DoubleWrapper>) context.getRegionCache().getOrCreateRegion(key, type, context);
+    Region<ByteArrayWrapper, DoubleWrapper> r = (Region<ByteArrayWrapper, DoubleWrapper>) context.getRegionProvider().getOrCreateRegion(key, type, context);
     return r;
   }
   
   protected Region<ByteArrayWrapper, DoubleWrapper> getRegion(ExecutionHandlerContext context, ByteArrayWrapper key) {
     @SuppressWarnings("unchecked")
-    Region<ByteArrayWrapper, DoubleWrapper> r = (Region<ByteArrayWrapper, DoubleWrapper>) context.getRegionCache().getRegion(key);
+    Region<ByteArrayWrapper, DoubleWrapper> r = (Region<ByteArrayWrapper, DoubleWrapper>) context.getRegionProvider().getRegion(key);
     return r;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRangeByLexExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRangeByLexExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRangeByLexExecutor.java
index 2e91ffb..e5bb73f 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRangeByLexExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRangeByLexExecutor.java
@@ -163,9 +163,7 @@ public class ZRangeByLexExecutor extends SortedSetExecutor {
     }
     if (limit > 0)
       params[params.length - 1] =  (limit + offset);
-
     SelectResults<ByteArrayWrapper> results = (SelectResults<ByteArrayWrapper>) query.execute(params);
-
     List<ByteArrayWrapper> list = results.asList();
     int size = list.size();
     return list.subList(Math.min(size, offset), size);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRangeByScoreExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRangeByScoreExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRangeByScoreExecutor.java
index e554618..fc73713 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRangeByScoreExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRangeByScoreExecutor.java
@@ -1,6 +1,7 @@
 package com.gemstone.gemfire.internal.redis.executor.sortedset;
 
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
 
 import com.gemstone.gemfire.cache.Region;
@@ -135,7 +136,7 @@ public class ZRangeByScoreExecutor extends SortedSetExecutor implements Extendab
     if (start == Double.POSITIVE_INFINITY || stop == Double.NEGATIVE_INFINITY || start > stop || (start == stop && (!startInclusive || !stopInclusive)))
       return null;
     if (start == Double.NEGATIVE_INFINITY && stop == Double.POSITIVE_INFINITY)
-      return keyRegion.entrySet();
+      return new HashSet(keyRegion.entrySet());
 
     Query query;
     Object[] params;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemExecutor.java
index c119281..d5d7a75 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemExecutor.java
@@ -42,7 +42,7 @@ public class ZRemExecutor extends SortedSetExecutor {
         numDeletedMembers++;
     }
     if (keyRegion.isEmpty())
-      context.getRegionCache().removeKey(key);
+      context.getRegionProvider().removeKey(key);
     command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), numDeletedMembers));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByLexExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByLexExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByLexExecutor.java
index 6dc7f51..2b26100 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByLexExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByLexExecutor.java
@@ -1,5 +1,6 @@
 package com.gemstone.gemfire.internal.redis.executor.sortedset;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 
@@ -90,7 +91,7 @@ public class ZRemRangeByLexExecutor extends SortedSetExecutor {
 
   private Collection<ByteArrayWrapper> getRange(ByteArrayWrapper key, Region<ByteArrayWrapper, DoubleWrapper> keyRegion, ExecutionHandlerContext context, ByteArrayWrapper start, ByteArrayWrapper stop, boolean startInclusive, boolean stopInclusive) throws Exception {
     if (start.equals("-") && stop.equals("+"))
-      return keyRegion.keySet();
+      return new ArrayList<ByteArrayWrapper>(keyRegion.keySet());
     else if (start.equals("+") || stop.equals("-"))
       return null;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByRankExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByRankExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByRankExecutor.java
index 24a4ed2..34452e3 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByRankExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByRankExecutor.java
@@ -69,7 +69,7 @@ public class ZRemRangeByRankExecutor extends SortedSetExecutor {
     try {
       if (startRank == 0 && stopRank == sSetSize- 1) {
         numRemoved = keyRegion.size();
-        context.getRegionCache().removeKey(key);
+        context.getRegionProvider().removeKey(key);
       } else {
         removeList = getRemoveKeys(context, key, startRank, stopRank);
       }
@@ -89,7 +89,7 @@ public class ZRemRangeByRankExecutor extends SortedSetExecutor {
           numRemoved++;
       }
       if (keyRegion.isEmpty())
-        context.getRegionCache().removeKey(key);
+        context.getRegionProvider().removeKey(key);
     }
     command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), numRemoved));
   }



[35/50] [abbrv] incubator-geode git commit: [GEODE-177] - Moving Dockerfile into Geode codebase

Posted by bs...@apache.org.
[GEODE-177] - Moving Dockerfile into Geode codebase


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

Branch: refs/heads/feature/GEODE-77
Commit: fcb42ad52ba6b01c7fde18f0864946d0200f990d
Parents: 1a6a0ef
Author: William Markito <wm...@pivotal.io>
Authored: Thu Aug 6 14:51:25 2015 -0700
Committer: William Markito <wm...@pivotal.io>
Committed: Thu Aug 6 14:51:25 2015 -0700

----------------------------------------------------------------------
 docker/Dockerfile | 56 +++++++++++++++++++++++++++++++++++++++++
 docker/README.md  | 67 ++++++++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 123 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fcb42ad5/docker/Dockerfile
----------------------------------------------------------------------
diff --git a/docker/Dockerfile b/docker/Dockerfile
new file mode 100644
index 0000000..ecdd712
--- /dev/null
+++ b/docker/Dockerfile
@@ -0,0 +1,56 @@
+FROM centos:latest
+MAINTAINER William Markito <wi...@gmail.com>
+
+LABEL Vendor="Apache Geode (incubating)"
+LABEL version=unstable
+
+# download JDK 8
+ENV	JAVA_HOME /jdk1.8.0_51
+
+RUN	yum install -y wget which tar git \
+	&& wget --no-cookies --no-check-certificate --header "Cookie: gpw_e24=http%3A%2F%2Fwww.oracle.com%2F; oraclelicense=accept-securebackup-cookie" "http://download.oracle.com/otn-pub/java/jdk/8u51-b16/jdk-8u51-linux-x64.tar.gz" \
+	&& tar xf jdk-8u51-linux-x64.tar.gz \
+	&& git clone -b develop https://github.com/apache/incubator-geode.git \
+	&& cd incubator-geode \
+	&& ./gradlew build -Dskip.tests=true \
+	&& ls /incubator-geode | grep -v gemfire-assembly | xargs rm -rf \
+	&& rm -rf /root/.gradle/ \
+	&& rm -rf /incubator-geode/gemfire-assembly/build/distributions/ \
+	&& rm -rf /jdk-8u51-linux-x64.tar.gz \
+	&& rm -rf $JAVA_HOME/*src.zip \
+						$JAVA_HOME/lib/missioncontrol \
+						$JAVA_HOME/lib/visualvm \
+						$JAVA_HOME/lib/*javafx* \
+						$JAVA_HOME/jre/lib/plugin.jar \
+						$JAVA_HOME/jre/lib/ext/jfxrt.jar \
+						$JAVA_HOME/jre/bin/javaws \
+						$JAVA_HOME/jre/lib/javaws.jar \
+						$JAVA_HOME/jre/lib/desktop \
+						$JAVA_HOME/jre/plugin \
+						$JAVA_HOME/jre/lib/deploy* \
+						$JAVA_HOME/jre/lib/*javafx* \
+						$JAVA_HOME/jre/lib/*jfx* \
+						$JAVA_HOME/jre/lib/amd64/libdecora_sse.so \
+						$JAVA_HOME/jre/lib/amd64/libprism_*.so \
+						$JAVA_HOME/jre/lib/amd64/libfxplugins.so \
+						$JAVA_HOME/jre/lib/amd64/libglass.so \
+						$JAVA_HOME/jre/lib/amd64/libgstreamer-lite.so \
+						$JAVA_HOME/jre/lib/amd64/libjavafx*.so \
+						$JAVA_HOME/jre/lib/amd64/libjfx*.so \
+	&& rm -rf /usr/share/locale/* \
+	&& yum remove -y perl \
+	&& yum clean all
+
+ENV GEODE_HOME /incubator-geode/gemfire-assembly/build/install/apache-geode
+ENV PATH $PATH:$GEODE_HOME/bin:$JAVA_HOME/bin
+
+# Default ports:
+# RMI/JMX 1099
+# REST 8080
+# PULE 7070
+# LOCATOR 10334
+# CACHESERVER 40404
+EXPOSE  8080 10334 40404 1099 7070
+VOLUME ["/data/"]
+CMD ["gfsh"]
+#ENTRYPOINT ["gfsh"]

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fcb42ad5/docker/README.md
----------------------------------------------------------------------
diff --git a/docker/README.md b/docker/README.md
new file mode 100644
index 0000000..4692940
--- /dev/null
+++ b/docker/README.md
@@ -0,0 +1,67 @@
+# Building the container image
+
+The current Dockerfile is based on a CentOS 6 image, downloads JDK 8, clone the Apache Geode git repository, starts a build and execute the basic tests.
+
+```
+docker build -t apachegeode/geode:unstable .
+```
+
+This may take a while depending on your internet connection, but it's worth since this is a one time step and you endup with a container that is tested and ready to be used for development. It will download Gradle and as part of the build, project dependencies as well.
+
+# Starting a locator and gfsh
+
+1. Then you can start gfsh as well in order to perform more commands:
+
+```
+docker run -it -p 10334:10334 -p 7575:7575 -p 1099:1099  apachegeode/geode:unstable gfsh
+```
+
+
+From this point you can pretty much follow [Apache Geode in 5 minutes](https://cwiki.apache.org/confluence/display/GEODE/Index#Index-Geodein5minutes) for example:
+
+```
+start server --name=server1
+```
+
+But in order to have real fun with containers you are probably better off using something like docker-compose or kubernetes. Those examples will come next.
+
+# Creating a cluster using multiple containers
+
+Install docker-compose following the instructions on this [link](https://docs.docker.com/compose/install/) and move into the composer directory.
+
+There is a docker-compose.yml example file there with a locator and a server.  To start the cluster execute:
+
+```
+docker-compose up
+```
+
+Or in order to start it in background:
+
+```
+docker-compose up -d
+```
+
+Do a docker ps and identify the container ID for the locator.  Now you can use *gfsh* on this container and connect to the distributed system:
+
+```
+docker exec -it <locator_container_id> gfsh
+gfsh>connect --locator=locator[10334]
+Connecting to Locator at [host=locator, port=10334] ..
+Connecting to Manager at [host=192.168.99.100, port=1099] ..
+Successfully connected to: [host=192.168.99.100, port=1099]
+
+gfsh>list members
+    Name     | Id
+    ------------ | --------------------------------------
+    locator      | locator(locator:33:locator)<v0>:1351
+    6e96cc0f6b72 | 172.17.1.92(6e96cc0f6b72:34)<v1>:28140
+```
+
+Type exit and now to scale the cluster you can leverage docker-compose scale command. For example:
+
+```
+docker-compose scale server=3
+```
+
+This will start 2 extra Geode server containers. You can verify this step by repeating the last GFSH step and listing the members.
+


[30/50] [abbrv] incubator-geode git commit: GEODE-137: use local GemFire server to initialize LocalCache whenever possible.

Posted by bs...@apache.org.
GEODE-137: use local GemFire server to initialize LocalCache whenever possible.


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

Branch: refs/heads/feature/GEODE-77
Commit: 2e2a795db57e21cc83784f0b3111b0b5e355ad87
Parents: 7d4ae09
Author: Qihong Chen <qc...@pivotal.io>
Authored: Wed Jul 29 10:02:26 2015 -0700
Committer: Qihong Chen <qc...@pivotal.io>
Committed: Thu Aug 6 10:44:36 2015 -0700

----------------------------------------------------------------------
 .../connector/GemFirePairRDDFunctions.scala     |  5 +-
 .../spark/connector/GemFireRDDFunctions.scala   |  5 +-
 .../internal/DefaultGemFireConnection.scala     | 30 +++++--
 .../connector/internal/LocatorHelper.scala      | 91 +++++++++++++++++++-
 .../internal/rdd/GemFireRegionRDD.scala         |  2 +-
 .../gemfire/spark/connector/package.scala       |  7 ++
 .../spark/connector/LocatorHelperTest.scala     | 77 +++++++++++++++++
 7 files changed, 208 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2e2a795d/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFirePairRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFirePairRDDFunctions.scala b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFirePairRDDFunctions.scala
index 86ec596..8050a5e 100644
--- a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFirePairRDDFunctions.scala
+++ b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFirePairRDDFunctions.scala
@@ -23,7 +23,10 @@ class GemFirePairRDDFunctions[K, V](val rdd: RDD[(K, V)]) extends Serializable w
       connConf: GemFireConnectionConf = defaultConnectionConf, 
       opConf: Map[String, String] = Map.empty): Unit = {    
     connConf.getConnection.validateRegion[K, V](regionPath)
-    logInfo(s"Save RDD id=${rdd.id} to region $regionPath")
+    if (log.isDebugEnabled)
+      logDebug(s"""Save RDD id=${rdd.id} to region $regionPath, partitions:\n  ${getRddPartitionsInfo(rdd)}""")
+    else
+      logInfo(s"""Save RDD id=${rdd.id} to region $regionPath""")
     val writer = new GemFirePairRDDWriter[K, V](regionPath, connConf, opConf)
     rdd.sparkContext.runJob(rdd, writer.write _)
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2e2a795d/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireRDDFunctions.scala b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireRDDFunctions.scala
index 3aa1ebd..5415727 100644
--- a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireRDDFunctions.scala
+++ b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/GemFireRDDFunctions.scala
@@ -25,7 +25,10 @@ class GemFireRDDFunctions[T](val rdd: RDD[T]) extends Serializable with Logging
       connConf: GemFireConnectionConf = defaultConnectionConf,
       opConf: Map[String, String] = Map.empty): Unit = {
     connConf.getConnection.validateRegion[K, V](regionPath)
-    logInfo(s"Save RDD id=${rdd.id} to region $regionPath")
+    if (log.isDebugEnabled)
+      logDebug(s"""Save RDD id=${rdd.id} to region $regionPath, partitions:\n  ${getRddPartitionsInfo(rdd)}""")
+    else
+      logInfo(s"""Save RDD id=${rdd.id} to region $regionPath""")
     val writer = new GemFireRDDWriter[T, K, V](regionPath, connConf, opConf)
     rdd.sparkContext.runJob(rdd, writer.write(func) _)
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2e2a795d/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnection.scala
----------------------------------------------------------------------
diff --git a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnection.scala b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnection.scala
index bba6c69..3fcb496 100644
--- a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnection.scala
+++ b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/DefaultGemFireConnection.scala
@@ -1,5 +1,7 @@
 package io.pivotal.gemfire.spark.connector.internal
 
+import java.net.InetAddress
+
 import com.gemstone.gemfire.cache.client.{ClientCache, ClientCacheFactory, ClientRegionShortcut}
 import com.gemstone.gemfire.cache.execute.{FunctionException, FunctionService}
 import com.gemstone.gemfire.cache.query.Query
@@ -7,7 +9,7 @@ import com.gemstone.gemfire.cache.{Region, RegionService}
 import com.gemstone.gemfire.internal.cache.execute.InternalExecution
 import io.pivotal.gemfire.spark.connector.internal.oql.QueryResultCollector
 import io.pivotal.gemfire.spark.connector.internal.rdd.GemFireRDDPartition
-import org.apache.spark.Logging
+import org.apache.spark.{SparkEnv, Logging}
 import io.pivotal.gemfire.spark.connector.GemFireConnection
 import io.pivotal.gemfire.spark.connector.internal.gemfirefunctions._
 import java.util.{Set => JSet, List => JList }
@@ -30,10 +32,7 @@ private[connector] class DefaultGemFireConnection (
 
   private def initClientCache() : ClientCache = {
     try {
-      import io.pivotal.gemfire.spark.connector.map2Properties
-      logInfo(s"""Init ClientCache: locators=${locators.mkString(",")}, props=$gemFireProps""")
-      val ccf = new ClientCacheFactory(gemFireProps)
-      locators.foreach { case (host, port)  => ccf.addPoolLocator(host, port) }
+      val ccf = getClientCacheFactory
       ccf.create()
     } catch {
       case e: Exception =>
@@ -41,6 +40,27 @@ private[connector] class DefaultGemFireConnection (
         throw new RuntimeException(e)
     }
   }
+  
+  private def getClientCacheFactory: ClientCacheFactory = {
+    import io.pivotal.gemfire.spark.connector.map2Properties
+    val ccf = new ClientCacheFactory(gemFireProps)
+    ccf.setPoolReadTimeout(30000)
+    val servers = LocatorHelper.getAllGemFireServers(locators)
+    if (servers.isDefined && servers.get.size > 0) {
+      val sparkIp = System.getenv("SPARK_LOCAL_IP")
+      val hostName = if (sparkIp != null) InetAddress.getByName(sparkIp).getCanonicalHostName
+                     else InetAddress.getLocalHost.getCanonicalHostName
+      val executorId = SparkEnv.get.executorId      
+      val pickedServers = LocatorHelper.pickPreferredGemFireServers(servers.get, hostName, executorId)
+      logInfo(s"""Init ClientCache: severs=${pickedServers.mkString(",")}, host=$hostName executor=$executorId props=$gemFireProps""")
+      logDebug(s"""Init ClientCache: all-severs=${pickedServers.mkString(",")}""")
+      pickedServers.foreach{ case (host, port)  => ccf.addPoolServer(host, port) }
+    } else {
+      logInfo(s"""Init ClientCache: locators=${locators.mkString(",")}, props=$gemFireProps""")
+      locators.foreach { case (host, port)  => ccf.addPoolLocator(host, port) }
+    }
+    ccf
+  }
 
   /** close the clientCache */
   override def close(): Unit =

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2e2a795d/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
----------------------------------------------------------------------
diff --git a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
index 550e0bc..a010c62 100644
--- a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
+++ b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
@@ -1,9 +1,17 @@
 package io.pivotal.gemfire.spark.connector.internal
 
+import java.net.InetSocketAddress
+import java.util.{ArrayList => JArrayList}
+
+import com.gemstone.gemfire.cache.client.internal.locator.{GetAllServersResponse, GetAllServersRequest}
+import com.gemstone.gemfire.distributed.internal.ServerLocation
+import com.gemstone.gemfire.distributed.internal.tcpserver.TcpClient
+import org.apache.spark.Logging
+
 import scala.util.{Failure, Success, Try}
 
 
-object LocatorHelper {
+object LocatorHelper extends Logging {
 
   /** valid locator strings are: host[port] and host:port */
   final val LocatorPattern1 = """([\w-_]+(\.[\w-_]+)*)\[([0-9]{2,5})\]""".r
@@ -27,4 +35,85 @@ object LocatorHelper {
   def parseLocatorsString(locatorsStr: String): Seq[(String, Int)] =
     locatorsStr.split(",").map(locatorStr2HostPortPair).map(_.get)
 
+
+  /**
+   * Return the list of live GemFire servers for the given locators. 
+   * @param locators locators for the given GemFire cluster
+   * @param serverGroup optional server group name, default is "" (empty string)
+   */
+  def getAllGemFireServers(locators: Seq[(String, Int)], serverGroup: String = ""): Option[Seq[(String, Int)]] = {
+    var result: Option[Seq[(String, Int)]] = None
+    locators.find { case (host, port) =>
+      try {
+        val addr = new InetSocketAddress(host, port)
+        val req = new GetAllServersRequest(serverGroup)
+        val res = TcpClient.requestToServer(addr.getAddress, addr.getPort, req, 2000)
+        if (res != null) {
+          import scala.collection.JavaConverters._
+          val servers = res.asInstanceOf[GetAllServersResponse].getServers.asInstanceOf[JArrayList[ServerLocation]]
+          if (servers.size > 0)
+            result = Some(servers.asScala.map(e => (e.getHostName, e.getPort)))
+        }
+      } catch { case e: Exception => logWarning("getAllGemFireServers error", e)
+      }
+      result.isDefined
+    }
+    result
+  }
+
+  /**
+   * Pick up at most 3 preferred servers from all available servers based on
+   * host name and Spark executor id.
+   *
+   * This method is used by DefaultGemFireConnection to create LocalCache. Usually 
+   * one server is enough to initialize LocalCacheFactory, but this provides two 
+   * backup servers in case of the 1st server can't be connected.
+   *   
+   * @param servers all available servers in the form of (hostname, port) pairs
+   * @param hostName the host name of the Spark executor
+   * @param executorId the Spark executor Id, such as "<driver>", "0", "1", ...
+   * @return Seq[(hostname, port)] of preferred servers
+   */
+  def pickPreferredGemFireServers(
+    servers: Seq[(String, Int)], hostName: String, executorId: String): Seq[(String, Int)] = {
+
+    // pick up `length` items form the Seq starts at the `start` position.
+    //  The Seq is treated as a ring, so at most `Seq.size` items can be picked
+    def circularTake[T](seq: Seq[T], start: Int, length: Int): Seq[T] = {
+      val size = math.min(seq.size, length)
+      (start until start + size).map(x => seq(x % seq.size))
+    }
+
+    // map executor id to int: "<driver>" (or non-number string) to 0, and "n" to n + 1
+    val id = try { executorId.toInt + 1 } catch { case e: NumberFormatException => 0 }
+    
+    // algorithm: 
+    // 1. sort server list
+    // 2. split sorted server list into 3 sub-lists a, b, and c:
+    //      list-a: servers on the given host
+    //      list-b: servers that are in front of list-a on the sorted server list
+    //      list-c: servers that are behind list-a on the sorted server list
+    //    then rotate list-a based on executor id, then create new server list:
+    //      modified list-a ++ list-c ++ list-b
+    // 3. if there's no server on the given host, then create new server list
+    //    by rotating sorted server list based on executor id.
+    // 4. take up to 3 servers from the new server list
+    val sortedServers = servers.sorted
+    val firstIdx = sortedServers.indexWhere(p => p._1 == hostName)
+    val lastIdx = if (firstIdx < 0) -1 else sortedServers.lastIndexWhere(p => p._1 == hostName)
+
+    if (firstIdx < 0) { // no local server
+      circularTake(sortedServers, id, 3)
+    } else {
+      val (seq1, seq2) = sortedServers.splitAt(firstIdx)
+      val seq = if (firstIdx == lastIdx) {  // one local server
+        seq2 ++ seq1
+      } else { // multiple local server
+        val (seq3, seq4) = seq2.splitAt(lastIdx - firstIdx + 1)
+        val seq3b = if (id % seq3.size == 0) seq3 else circularTake(seq3, id, seq3.size)
+        seq3b ++ seq4 ++ seq1
+      }
+      circularTake(seq, 0, 3)
+    }
+  }  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2e2a795d/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRegionRDD.scala
----------------------------------------------------------------------
diff --git a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRegionRDD.scala b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRegionRDD.scala
index cff61d6..3a987b2 100644
--- a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRegionRDD.scala
+++ b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/rdd/GemFireRegionRDD.scala
@@ -82,7 +82,7 @@ class GemFireRegionRDD[K, V] private[connector]
         logInfo(s"""RDD id=${this.id} region=$regionPath conn=${connConf.locators.mkString(",")}, env=$opConf""")
         val p = if (data.isPartitioned) preferredPartitioner else defaultReplicatedRegionPartitioner
         val splits = p.partitions[K, V](conn, data, opConf)
-        logDebug(s"""RDD id=${this.id} region=$regionPath partitions=${splits.mkString(",")}""")
+        logDebug(s"""RDD id=${this.id} region=$regionPath partitions=\n  ${splits.mkString("\n  ")}""")
         splits
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2e2a795d/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/package.scala
----------------------------------------------------------------------
diff --git a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/package.scala b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/package.scala
index 72a5bb1..d08e96c 100644
--- a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/package.scala
+++ b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/package.scala
@@ -25,6 +25,8 @@ package object connector {
   final val RDDSaveBatchSizePropKey = "rdd.save.batch.size"
   final val RDDSaveBatchSizeDefault = 10000
   
+  /** implicits */
+  
   implicit def toSparkContextFunctions(sc: SparkContext): GemFireSparkContextFunctions =
     new GemFireSparkContextFunctions(sc)
 
@@ -43,4 +45,9 @@ package object connector {
   implicit def map2Properties(map: Map[String,String]): java.util.Properties =
     (new java.util.Properties /: map) {case (props, (k,v)) => props.put(k,v); props}
 
+  /** internal util methods */
+  
+  private[connector] def getRddPartitionsInfo(rdd: RDD[_], sep: String = "\n  "): String =
+    rdd.partitions.zipWithIndex.map{case (p,i) => s"$i: $p loc=${rdd.preferredLocations(p)}"}.mkString(sep)
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2e2a795d/gemfire-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/LocatorHelperTest.scala
----------------------------------------------------------------------
diff --git a/gemfire-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/LocatorHelperTest.scala b/gemfire-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/LocatorHelperTest.scala
index 508666a..de4b7a7 100644
--- a/gemfire-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/LocatorHelperTest.scala
+++ b/gemfire-spark-connector/gemfire-spark-connector/src/test/scala/unittest/io/pivotal/gemfire/spark/connector/LocatorHelperTest.scala
@@ -1,5 +1,7 @@
 package unittest.io.pivotal.gemfire.spark.connector
 
+import java.net.InetAddress
+
 import io.pivotal.gemfire.spark.connector.internal.LocatorHelper
 import org.scalatest.FunSuite
 
@@ -72,4 +74,79 @@ class LocatorHelperTest extends FunSuite {
     intercept[Exception] { LocatorHelper.parseLocatorsString("local^host:2345,localhost.1234") }
   }
 
+  test("pickPreferredGemFireServers: shared servers and one gf-server per host") {
+    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host2", 4002), ("host3", 4003),("host4", 4004))
+    val servers = Seq(srv1, srv2, srv3, srv4)
+    verifyPickPreferredGemFireServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
+    verifyPickPreferredGemFireServers(servers, "host2", "0", Seq(srv2, srv3, srv4))
+    verifyPickPreferredGemFireServers(servers, "host3", "1", Seq(srv3, srv4, srv1))
+    verifyPickPreferredGemFireServers(servers, "host4", "2", Seq(srv4, srv1, srv2))
+  }
+
+  test("pickPreferredGemFireServers: shared servers, one gf-server per host, un-sorted list") {
+    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host2", 4002), ("host3", 4003),("host4", 4004))
+    val servers = Seq(srv4, srv2, srv3, srv1)
+    verifyPickPreferredGemFireServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
+    verifyPickPreferredGemFireServers(servers, "host2", "0", Seq(srv2, srv3, srv4))
+    verifyPickPreferredGemFireServers(servers, "host3", "1", Seq(srv3, srv4, srv1))
+    verifyPickPreferredGemFireServers(servers, "host4", "2", Seq(srv4, srv1, srv2))
+  }
+
+  test("pickPreferredGemFireServers: shared servers and two gf-server per host") {
+    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host1", 4002), ("host2", 4003), ("host2", 4004))
+    val servers = Seq(srv1, srv2, srv3, srv4)
+    verifyPickPreferredGemFireServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
+    verifyPickPreferredGemFireServers(servers, "host1", "0", Seq(srv2, srv1, srv3))
+    verifyPickPreferredGemFireServers(servers, "host2", "1", Seq(srv3, srv4, srv1))
+    verifyPickPreferredGemFireServers(servers, "host2", "2", Seq(srv4, srv3, srv1))
+  }
+
+  test("pickPreferredGemFireServers: shared servers, two gf-server per host, un-sorted server list") {
+    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host1", 4002), ("host2", 4003), ("host2", 4004))
+    val servers = Seq(srv1, srv4, srv3, srv2)
+    verifyPickPreferredGemFireServers(servers, "host1", "<driver>", Seq(srv1, srv2, srv3))
+    verifyPickPreferredGemFireServers(servers, "host1", "0", Seq(srv2, srv1, srv3))
+    verifyPickPreferredGemFireServers(servers, "host2", "1", Seq(srv3, srv4, srv1))
+    verifyPickPreferredGemFireServers(servers, "host2", "2", Seq(srv4, srv3, srv1))
+  }
+
+  test("pickPreferredGemFireServers: no shared servers and one gf-server per host") {
+    val (srv1, srv2, srv3, srv4) = (("host1", 4001), ("host2", 4002), ("host3", 4003),("host4", 4004))
+    val servers = Seq(srv1, srv2, srv3, srv4)
+    verifyPickPreferredGemFireServers(servers, "host5", "<driver>", Seq(srv1, srv2, srv3))
+    verifyPickPreferredGemFireServers(servers, "host6", "0", Seq(srv2, srv3, srv4))
+    verifyPickPreferredGemFireServers(servers, "host7", "1", Seq(srv3, srv4, srv1))
+    verifyPickPreferredGemFireServers(servers, "host8", "2", Seq(srv4, srv1, srv2))
+  }
+
+  test("pickPreferredGemFireServers: no shared servers, one gf-server per host, and less gf-server") {
+    val (srv1, srv2) = (("host1", 4001), ("host2", 4002))
+    val servers = Seq(srv1, srv2)
+    verifyPickPreferredGemFireServers(servers, "host5", "<driver>", Seq(srv1, srv2))
+    verifyPickPreferredGemFireServers(servers, "host6", "0", Seq(srv2, srv1))
+    verifyPickPreferredGemFireServers(servers, "host7", "1", Seq(srv1, srv2))
+    verifyPickPreferredGemFireServers(servers, "host8", "2", Seq(srv2, srv1))
+
+
+    println("host name: " + InetAddress.getLocalHost.getHostName)
+    println("canonical host name: " + InetAddress.getLocalHost.getCanonicalHostName)
+    println("canonical host name 2: " + InetAddress.getByName(InetAddress.getLocalHost.getHostName).getCanonicalHostName)
+  }
+
+  test("pickPreferredGemFireServers: ad-hoc") {
+    val (srv4, srv5, srv6) = (
+      ("w2-gst-pnq-04.gemstone.com", 40411), ("w2-gst-pnq-05.gemstone.com", 40411), ("w2-gst-pnq-06.gemstone.com", 40411))
+    val servers = Seq(srv6, srv5, srv4)
+    verifyPickPreferredGemFireServers(servers, "w2-gst-pnq-03.gemstone.com", "<driver>", Seq(srv4, srv5, srv6))
+    verifyPickPreferredGemFireServers(servers, "w2-gst-pnq-04.gemstone.com", "1", Seq(srv4, srv5, srv6))
+    verifyPickPreferredGemFireServers(servers, "w2-gst-pnq-05.gemstone.com", "0", Seq(srv5, srv6, srv4))
+    verifyPickPreferredGemFireServers(servers, "w2-gst-pnq-06.gemstone.com", "2", Seq(srv6, srv4, srv5))
+  }
+  
+  def verifyPickPreferredGemFireServers(
+    servers: Seq[(String, Int)], hostName: String, executorId: String, expectation: Seq[(String, Int)]): Unit = {
+    val result = LocatorHelper.pickPreferredGemFireServers(servers, hostName, executorId)
+    assert(result == expectation, s"pick servers for $hostName:$executorId")
+  }
+
 }


[14/50] [abbrv] incubator-geode git commit: GEODE-95:PartitionedRegionSingleHopDUnitTest.test_MetadataContents fails intermittently There is a race between updating the metadata by client and creation of secondary bucket. Once, the client fetches the me

Posted by bs...@apache.org.
GEODE-95:PartitionedRegionSingleHopDUnitTest.test_MetadataContents fails intermittently
 There is a race between updating the metadata by client and creation of secondary bucket.
Once, the client fetches the metadata without information of secondary bucket(due to race), any further put/get will go to the correct server which hosts the bucket for which client has information. This will not cause any re-fetching of metadata from server as there is no hop. Hence only making test changes.


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

Branch: refs/heads/feature/GEODE-77
Commit: b460df43b1f2dbf06ad37f4175ca58ea49c450b3
Parents: 64a9bd4
Author: kbachhav <kbachhav.pivotal.io>
Authored: Thu Jul 30 18:46:08 2015 +0530
Committer: kbachhav <kbachhav.pivotal.io>
Committed: Thu Jul 30 18:46:55 2015 +0530

----------------------------------------------------------------------
 .../internal/cache/PartitionedRegionSingleHopDUnitTest.java      | 4 ----
 1 file changed, 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b460df43/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
index 95fbe70..39618bb 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/PartitionedRegionSingleHopDUnitTest.java
@@ -2216,10 +2216,6 @@ public class PartitionedRegionSingleHopDUnitTest extends CacheTestCase {
       }
     };
     DistributedTestCase.waitForCriterion(wc, 60000, 1000, true);
-    for (Entry entry : prMetaData.getBucketServerLocationsMap_TEST_ONLY()
-        .entrySet()) {
-      assertEquals(2, ((List)entry.getValue()).size());
-    }
   }
 }
 


[03/50] [abbrv] incubator-geode git commit: [GEODE-160] Modify Git workspace check to use '--is-inside-working-tree' git command

Posted by bs...@apache.org.
[GEODE-160] Modify Git workspace check to use '--is-inside-working-tree' git command

Tested with using Git checkout and outside Git workspace using git archive


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

Branch: refs/heads/feature/GEODE-77
Commit: ea8ce3f54f196d26d0f5290ec3efb179720bdb62
Parents: 4f86d40
Author: Mark Bretl <mb...@pivotal.io>
Authored: Mon Jul 27 13:00:36 2015 -0700
Committer: Mark Bretl <mb...@pivotal.io>
Committed: Tue Jul 28 11:54:31 2015 -0700

----------------------------------------------------------------------
 gemfire-core/build.gradle | 20 +++++++++++++++++---
 1 file changed, 17 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/ea8ce3f5/gemfire-core/build.gradle
----------------------------------------------------------------------
diff --git a/gemfire-core/build.gradle b/gemfire-core/build.gradle
index 2992ee9..1c17474 100755
--- a/gemfire-core/build.gradle
+++ b/gemfire-core/build.gradle
@@ -99,10 +99,22 @@ task createVersionPropertiesFile {
   inputs.dir compileJava.destinationDir
 
   doLast {
-    def gitFolder = new File ("${rootProject.projectDir}/.git")
-    if ( gitFolder.exists() ) {
+    new ByteArrayOutputStream().withStream { gitWorkspaceStream ->
+      def result = exec {
+        workingDir = "${projectDir}"
+        standardOutput = gitWorkspaceStream
+        ignoreExitValue = true
+        executable = "git"
+        args = ['rev-parse', '--is-inside-work-tree']
+      }
+      ext.isGitWorkspace = gitWorkspaceStream.toString()
+      ext.isGitWorkspace = ext.isGitWorkspace.trim()
+    }
+
+    if ( isGitWorkspace.equalsIgnoreCase('true') ) {
       new ByteArrayOutputStream().withStream { gitBranchStream ->
         def result = exec {
+          workingDir = "${projectDir}"
           standardOutput = gitBranchStream
           executable = "git"
           args = ['rev-parse', '--abbrev-ref', 'HEAD']
@@ -113,6 +125,7 @@ task createVersionPropertiesFile {
 
       new ByteArrayOutputStream().withStream { commitStream ->
         def result = exec {
+          workingDir = "${projectDir}"
           standardOutput = commitStream
           executable = "git"
           args = ['rev-parse', 'HEAD']
@@ -123,6 +136,7 @@ task createVersionPropertiesFile {
 
       new ByteArrayOutputStream().withStream { sourceDateStream ->
         def result = exec {
+          workingDir = "${projectDir}"
           standardOutput = sourceDateStream
           executable = "git"
           args = ['show', '-s', '--format=%ci', "${ext.commitId}"]
@@ -146,7 +160,7 @@ task createVersionPropertiesFile {
     ext.jdkVersion = System.getProperty('java.version')
 
     def props = [
-      "Product-Name"      : "Pivotal GemFire",
+      "Product-Name"      : "Apache Geode (incubating)",
       "Product-Version"   : version,
       "Build-Id"          : System.env.USER + ' ' + ext.buildNumber,
       "Build-Date"        : ext.buildDate,


[20/50] [abbrv] incubator-geode git commit: GEODE-175: change test to use a wait criteria instead of pause and use millis instead of seconds

Posted by bs...@apache.org.
GEODE-175: change test to use a wait criteria instead of pause and use millis instead of seconds


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

Branch: refs/heads/feature/GEODE-77
Commit: fa9bd37ab26856fb5c4a7222edd5b96c8f06ffa5
Parents: bc4508c
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Aug 4 10:42:50 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Tue Aug 4 11:20:25 2015 -0700

----------------------------------------------------------------------
 .../cache/RemoteTransactionDUnitTest.java       | 42 +++++++++++++-------
 1 file changed, 27 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa9bd37a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
index 7c6d133..0daaafb 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RemoteTransactionDUnitTest.java
@@ -4006,10 +4006,15 @@ protected static class ClientListener extends CacheListenerAdapter {
     vm1.invoke(new SerializableCallable() {
       @Override
       public Object call() throws Exception {
+        System.setProperty(LocalRegion.EXPIRY_MS_PROPERTY, "true");
+        try {
         RegionFactory<String, String> rf = getCache().createRegionFactory();
         rf.setEntryTimeToLive(new ExpirationAttributes(1, ExpirationAction.LOCAL_DESTROY));
         rf.setScope(Scope.DISTRIBUTED_ACK);
         rf.create(regionName);
+        } finally {
+          System.getProperties().remove(LocalRegion.EXPIRY_MS_PROPERTY);
+        }
         return null;
       }
     });
@@ -4027,20 +4032,6 @@ protected static class ClientListener extends CacheListenerAdapter {
       @Override
       public Object call() throws Exception {
         final Region<String, String> r = getCache().getRegion(regionName);
-        r.put("key", "value");
-        r.put("nonTXkey", "nonTXvalue");
-        getCache().getCacheTransactionManager().begin();
-        r.put("key", "newvalue");
-        // wait for entry to expire
-        DistributedTestCase.pause(5000);
-        TransactionId tx = getCache().getCacheTransactionManager().suspend();
-        // A remote tx will allow expiration to happen on the side that
-        // is not hosting the tx. But it will not allow an expiration
-        // initiated on the hosting jvm.
-        assertFalse(r.containsKey("key"));
-        assertFalse(r.containsKey("nonTXkey"));
-        getCache().getCacheTransactionManager().resume(tx);
-        getCache().getCacheTransactionManager().commit();
         WaitCriterion wc2 = new WaitCriterion() {
           @Override
           public boolean done() {
@@ -4049,9 +4040,30 @@ protected static class ClientListener extends CacheListenerAdapter {
           
           @Override
           public String description() {
-            return "did not expire";
+            return "did not expire containsKey(key)=" + r.containsKey("key") + " r.containsKey(nonTXKey)=" + r.containsKey("nonTXKey");
           }
         };
+        ExpiryTask.suspendExpiration();
+        Region.Entry entry = null;
+        long tilt;
+        try {
+          r.put("key", "value");
+          r.put("nonTXkey", "nonTXvalue");
+          getCache().getCacheTransactionManager().begin();
+          r.put("key", "newvalue");
+        } 
+        finally {
+          ExpiryTask.permitExpiration();
+        }
+        TransactionId tx = getCache().getCacheTransactionManager().suspend();
+        // A remote tx will allow expiration to happen on the side that
+        // is not hosting the tx. But it will not allow an expiration
+        // initiated on the hosting jvm.
+        // tx is hosted in vm2 so expiration can happen in vm1.
+        DistributedTestCase.waitForCriterion(wc2, 30000, 5, true);
+        getCache().getCacheTransactionManager().resume(tx);
+        assertTrue(r.containsKey("key"));
+        getCache().getCacheTransactionManager().commit();
         DistributedTestCase.waitForCriterion(wc2, 30000, 5, true);
         return null;
       }


[43/50] [abbrv] incubator-geode git commit: GEODE-211: shorten the time ConcurrentRegionOperationsJUnitTest runs

Posted by bs...@apache.org.
GEODE-211: shorten the time ConcurrentRegionOperationsJUnitTest runs


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

Branch: refs/heads/feature/GEODE-77
Commit: 9de95d63275c5e5a13b35d5bb7744c399916d7f5
Parents: fcd0340
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Aug 12 10:12:44 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Wed Aug 12 10:41:27 2015 -0700

----------------------------------------------------------------------
 .../internal/cache/ConcurrentRegionOperationsJUnitTest.java   | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9de95d63/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentRegionOperationsJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentRegionOperationsJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentRegionOperationsJUnitTest.java
index ce0882a..a036e6c 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentRegionOperationsJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ConcurrentRegionOperationsJUnitTest.java
@@ -86,7 +86,7 @@ public class ConcurrentRegionOperationsJUnitTest extends DiskRegionTestingBase
   protected boolean exceptionOccuredInForceRolls = false;
 
   // if this test is to run for a longer time, make this true
-  private static final boolean longTest = true;
+  private static final boolean longTest = false;
 
   protected boolean failure = false;
 
@@ -408,7 +408,7 @@ public class ConcurrentRegionOperationsJUnitTest extends DiskRegionTestingBase
           region.forceRolling();
 
           try {
-            Thread.sleep(250);
+            Thread.sleep(TIME_TO_RUN/100);
           }
           catch (InterruptedException e) {
             fail("interrupted");
@@ -531,7 +531,6 @@ public class ConcurrentRegionOperationsJUnitTest extends DiskRegionTestingBase
     } finally {
       this.timeToStop.set(true);
     }
-
     for (int i = 0; i < numberOfPutsThreads; i++) {
       DistributedTestCase.join(putThreads[i], 10*1000, null);
     }
@@ -872,7 +871,7 @@ public class ConcurrentRegionOperationsJUnitTest extends DiskRegionTestingBase
       waitForAllStartersToBeReady();
       while (!isItTimeToStop()) {
         try {
-          Thread.sleep(100);
+          Thread.sleep(TIME_TO_RUN/100);
         }
         catch (InterruptedException e) {
           fail("interrupted");


[26/50] [abbrv] incubator-geode git commit: Fix GEODE-154: HDFS region with HeapLRU reports LowMemoryExceptions even when region size is less then the eviction threshold

Posted by bs...@apache.org.
    Fix GEODE-154: HDFS region with HeapLRU reports LowMemoryExceptions
    even when region size is less then the eviction threshold

    - calling setUsageThreshold() on MemoryPoolMXBean resulted in
    it reporting incorrect memory usage causing temporary spikes.
    Since we already set CollectionThreshold at 1 byte and also have a
    stat and heap poller thread, we now skip setting the usage threshold.


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

Branch: refs/heads/feature/GEODE-77
Commit: c575983f14c5417cd631f0d144f514d4c6cf79c1
Parents: ea9f03e
Author: Namrata Thanvi <nt...@pivotal.io>
Authored: Thu Aug 6 14:22:53 2015 +0530
Committer: Namrata Thanvi <nt...@pivotal.io>
Committed: Thu Aug 6 14:22:53 2015 +0530

----------------------------------------------------------------------
 .../internal/cache/control/HeapMemoryMonitor.java      | 13 +------------
 1 file changed, 1 insertion(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/c575983f/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/HeapMemoryMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/HeapMemoryMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/HeapMemoryMonitor.java
index 8346245..fda337b 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/HeapMemoryMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/HeapMemoryMonitor.java
@@ -554,18 +554,7 @@ public void stopMonitoring() {
    *          Number of bytes of heap memory currently used.
    */
   private void setUsageThresholdOnMXBean(final long bytesUsed) {
-    if (testDisableMemoryUpdates) {
-      return;
-    }
-    
-    final MemoryPoolMXBean memoryPoolMXBean = getTenuredMemoryPoolMXBean();
-    final MemoryThresholds saveThresholds = this.thresholds;
-
-    if (bytesUsed < saveThresholds.getEvictionThresholdBytes()) {
-      memoryPoolMXBean.setUsageThreshold(saveThresholds.getEvictionThresholdBytes());
-    } else {
-      memoryPoolMXBean.setUsageThreshold(saveThresholds.getCriticalThresholdBytes());
-    }
+	  //// this method has been made a no-op to fix bug 49064 
   }
   
   /**


[39/50] [abbrv] incubator-geode git commit: GEODE-138: remove race condition from testEventDelivery

Posted by bs...@apache.org.
GEODE-138: remove race condition from testEventDelivery

The race due to the test expecting local event delivery to be synchronous
but the implementation of off-heap resource manager events being async.
The test has been changed to always use a wait criteria when waiting
for a memory event to arrive.


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

Branch: refs/heads/feature/GEODE-77
Commit: bd84581e3d5a93cbbb3fcdf06ebe60bb9460e702
Parents: e255c75
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Mon Aug 10 14:24:58 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Mon Aug 10 16:29:09 2015 -0700

----------------------------------------------------------------------
 .../MemoryThresholdsOffHeapDUnitTest.java       | 42 ++++++++++----------
 1 file changed, 22 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/bd84581e/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
index 0b704a6..83b8608 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
@@ -146,14 +146,14 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
     // NORMAL -> EVICTION
     setUsageAboveEvictionThreshold(server2, regionName);
     verifyListenerValue(server1, MemoryState.EVICTION, 1, true);
-    verifyListenerValue(server2, MemoryState.EVICTION, 1, false);
+    verifyListenerValue(server2, MemoryState.EVICTION, 1, true);
     
     // EVICTION -> CRITICAL
     setUsageAboveCriticalThreshold(server2, regionName);
     verifyListenerValue(server1, MemoryState.CRITICAL, 1, true);
-    verifyListenerValue(server2, MemoryState.CRITICAL, 1, false);
+    verifyListenerValue(server2, MemoryState.CRITICAL, 1, true);
     verifyListenerValue(server1, MemoryState.EVICTION, 2, true);
-    verifyListenerValue(server2, MemoryState.EVICTION, 2, false);
+    verifyListenerValue(server2, MemoryState.EVICTION, 2, true);
     
     // CRITICAL -> CRITICAL
     server2.invoke(new SerializableCallable() {
@@ -168,9 +168,9 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
       }
     });
     verifyListenerValue(server1, MemoryState.CRITICAL, 1, true);
-    verifyListenerValue(server2, MemoryState.CRITICAL, 1, false);
+    verifyListenerValue(server2, MemoryState.CRITICAL, 1, true);
     verifyListenerValue(server1, MemoryState.EVICTION, 2, true);
-    verifyListenerValue(server2, MemoryState.EVICTION, 2, false);
+    verifyListenerValue(server2, MemoryState.EVICTION, 2, true);
     
     // CRITICAL -> EVICTION
     server2.invoke(new SerializableCallable() {
@@ -185,7 +185,7 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
       }
     });
     verifyListenerValue(server1, MemoryState.EVICTION, 3, true);
-    verifyListenerValue(server2, MemoryState.EVICTION, 3, false);
+    verifyListenerValue(server2, MemoryState.EVICTION, 3, true);
 
     // EVICTION -> EVICTION
     server2.invoke(new SerializableCallable() {
@@ -198,7 +198,7 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
       }
     });
     verifyListenerValue(server1, MemoryState.EVICTION, 3, true);
-    verifyListenerValue(server2, MemoryState.EVICTION, 3, false);
+    verifyListenerValue(server2, MemoryState.EVICTION, 3, true);
 
     // EVICTION -> NORMAL
     server2.invoke(new SerializableCallable() {
@@ -215,9 +215,9 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
     verifyListenerValue(server1, MemoryState.EVICTION, 3, true);
     verifyListenerValue(server1, MemoryState.NORMAL, 1, true);
     
-    verifyListenerValue(server2, MemoryState.CRITICAL, 1, false);
-    verifyListenerValue(server2, MemoryState.EVICTION, 3, false);
-    verifyListenerValue(server2, MemoryState.NORMAL, 1, false);
+    verifyListenerValue(server2, MemoryState.CRITICAL, 1, true);
+    verifyListenerValue(server2, MemoryState.EVICTION, 3, true);
+    verifyListenerValue(server2, MemoryState.NORMAL, 1, true);
   }
   
   /**
@@ -245,23 +245,23 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
     registerTestMemoryThresholdListener(server2);
     
     setUsageAboveEvictionThreshold(server1, regionName);
-    verifyListenerValue(server1, MemoryState.EVICTION, 0, false);
+    verifyListenerValue(server1, MemoryState.EVICTION, 0, true);
     verifyListenerValue(server2, MemoryState.EVICTION, 0, true);
 
     setThresholds(server1, 70f, 0f);
-    verifyListenerValue(server1, MemoryState.EVICTION, 1, false);
+    verifyListenerValue(server1, MemoryState.EVICTION, 1, true);
     verifyListenerValue(server2, MemoryState.EVICTION, 1, true);
 
     setUsageAboveCriticalThreshold(server1, regionName);
-    verifyListenerValue(server1, MemoryState.CRITICAL, 0, false);
+    verifyListenerValue(server1, MemoryState.CRITICAL, 0, true);
     verifyListenerValue(server2, MemoryState.CRITICAL, 0, true);
 
     setThresholds(server1, 0f, 0f);
-    verifyListenerValue(server1, MemoryState.EVICTION_DISABLED, 1, false);
+    verifyListenerValue(server1, MemoryState.EVICTION_DISABLED, 1, true);
     verifyListenerValue(server2, MemoryState.EVICTION_DISABLED, 1, true);
 
     setThresholds(server1, 0f, 90f);
-    verifyListenerValue(server1, MemoryState.CRITICAL, 1, false);
+    verifyListenerValue(server1, MemoryState.CRITICAL, 1, true);
     verifyListenerValue(server2, MemoryState.CRITICAL, 1, true);
 
     //verify that stats on server2 are not changed by events on server1
@@ -373,7 +373,7 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
     setUsageAboveCriticalThreshold(server2, regionName);
 
     verifyListenerValue(server1, MemoryState.CRITICAL, 1, true);
-    verifyListenerValue(server2, MemoryState.CRITICAL, 1, false);
+    verifyListenerValue(server2, MemoryState.CRITICAL, 1, true);
 
     //make sure that client puts are rejected
     doPuts(client, regionName, true/*catchRejectedException*/,
@@ -448,7 +448,7 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
     setUsageAboveCriticalThreshold(server2, regionName);
 
     verifyListenerValue(server1, MemoryState.CRITICAL, 1, true);
-    verifyListenerValue(server2, MemoryState.CRITICAL, 1, false);
+    verifyListenerValue(server2, MemoryState.CRITICAL, 1, true);
 
     //make sure that local server1 puts are rejected
     doPuts(server1, regionName, false/*catchRejectedException*/,
@@ -1552,7 +1552,9 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
 
   /**
    * Verifies that the test listener value on the given vm is what is expected
-   * Note that for remote events useWaitCriterion must be true
+   * Note that for remote events useWaitCriterion must be true.
+   * Note also that since off-heap local events are async local events must also
+   * set useWaitCriterion to true.
    * 
    * @param vm
    *          the vm where verification should take place
@@ -1562,7 +1564,7 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
    * @param value
    *          the expected value
    * @param useWaitCriterion
-   *          must be true for remote events
+   *          must be true for both local and remote events (see GEODE-138)
    */
   private void verifyListenerValue(VM vm, final MemoryState state, final int value, final boolean useWaitCriterion) {
     vm.invoke(new SerializableCallable() {
@@ -1672,7 +1674,7 @@ public class MemoryThresholdsOffHeapDUnitTest extends BridgeTestCase {
           throw new IllegalStateException("Unknown memory state");
         }
         if (useWaitCriterion) {
-          waitForCriterion(wc, 5000, 100, true);
+          waitForCriterion(wc, 5000, 10, true);
         }
         return null;
       }


[22/50] [abbrv] incubator-geode git commit: Fix return type of getExtendedLogger. The type is inferred correctly on 1.8 but not in 1.7. This change makes it correct under both versions of JDK.

Posted by bs...@apache.org.
Fix return type of getExtendedLogger. The type is inferred correctly
on 1.8 but not in 1.7. This change makes it correct under both versions
of JDK.


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

Branch: refs/heads/feature/GEODE-77
Commit: 004a22c6777dd13b8206b2a3586597abff48ea13
Parents: 1c4cd8f
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue Aug 4 12:44:01 2015 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Aug 4 14:41:46 2015 -0700

----------------------------------------------------------------------
 .../com/gemstone/gemfire/internal/logging/log4j/FastLogger.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/004a22c6/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/FastLogger.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/FastLogger.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/FastLogger.java
index 9583429..4af57d5 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/FastLogger.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/log4j/FastLogger.java
@@ -57,7 +57,7 @@ public class FastLogger extends ExtendedLoggerWrapper {
     return delegating;
   }
   
-  public Logger getExtendedLogger() {
+  public ExtendedLogger getExtendedLogger() {
     return super.logger;
   }
 }


[29/50] [abbrv] incubator-geode git commit: GEODE-137: use local GemFire server to initialize ClientCache whenever possible.

Posted by bs...@apache.org.
GEODE-137: use local GemFire server to initialize ClientCache whenever possible.


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

Branch: refs/heads/feature/GEODE-77
Commit: d0d48588c895cff7418090c50b57f9f4a740763c
Parents: 2e2a795
Author: Qihong Chen <qc...@pivotal.io>
Authored: Thu Aug 6 09:51:53 2015 -0700
Committer: Qihong Chen <qc...@pivotal.io>
Committed: Thu Aug 6 10:44:36 2015 -0700

----------------------------------------------------------------------
 .../pivotal/gemfire/spark/connector/internal/LocatorHelper.scala | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0d48588/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
----------------------------------------------------------------------
diff --git a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
index a010c62..3a094cc 100644
--- a/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
+++ b/gemfire-spark-connector/gemfire-spark-connector/src/main/scala/io/pivotal/gemfire/spark/connector/internal/LocatorHelper.scala
@@ -65,8 +65,8 @@ object LocatorHelper extends Logging {
    * Pick up at most 3 preferred servers from all available servers based on
    * host name and Spark executor id.
    *
-   * This method is used by DefaultGemFireConnection to create LocalCache. Usually 
-   * one server is enough to initialize LocalCacheFactory, but this provides two 
+   * This method is used by DefaultGemFireConnection to create ClientCache. Usually
+   * one server is enough to initialize ClientCacheFactory, but this provides two
    * backup servers in case of the 1st server can't be connected.
    *   
    * @param servers all available servers in the form of (hostname, port) pairs


[10/50] [abbrv] incubator-geode git commit: GEODE-124: Merge 'feature/GEODE-124' into develop

Posted by bs...@apache.org.
GEODE-124: Merge 'feature/GEODE-124' into develop


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

Branch: refs/heads/feature/GEODE-77
Commit: 60ebf92b7bd6600441ef5f8ee3b4bed29937c345
Parents: 298ff79 65e2cc6
Author: Ashvin Agrawal <as...@apache.org>
Authored: Wed Jul 29 14:14:59 2015 -0700
Committer: Ashvin Agrawal <as...@apache.org>
Committed: Wed Jul 29 14:15:38 2015 -0700

----------------------------------------------------------------------
 .../cache/control/ResourceManagerStats.java     |  13 +
 gemfire-rebalancer/build.gradle                 |  12 +
 .../gemfire/cache/util/AutoBalancer.java        | 512 ++++++++++++
 .../cache/util/AutoBalancerJUnitTest.java       | 803 +++++++++++++++++++
 settings.gradle                                 |   1 +
 5 files changed, 1341 insertions(+)
----------------------------------------------------------------------



[50/50] [abbrv] incubator-geode git commit: Merge remote-tracking branch 'origin/develop' into feature/GEODE-77

Posted by bs...@apache.org.
Merge remote-tracking branch 'origin/develop' into feature/GEODE-77


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

Branch: refs/heads/feature/GEODE-77
Commit: e0d1c4f908b30bdf239041403bf9c6369f2be22b
Parents: ab68f4e eb7e7b7
Author: Bruce Schuchardt <bs...@pivotal.io>
Authored: Fri Aug 14 13:39:20 2015 -0700
Committer: Bruce Schuchardt <bs...@pivotal.io>
Committed: Fri Aug 14 13:39:20 2015 -0700

----------------------------------------------------------------------
 build.gradle                                    |   1 +
 docker/Dockerfile                               |  56 ++
 docker/README.md                                |  67 ++
 gemfire-assembly/build.gradle                   |   1 +
 gemfire-core/build.gradle                       |  30 +-
 .../cache/operations/GetOperationContext.java   |   7 +-
 .../operations/KeyValueOperationContext.java    |  70 +-
 .../internal/GetOperationContextImpl.java       |  16 +-
 .../cache/query/internal/RuntimeIterator.java   |   3 -
 .../query/internal/index/AbstractMapIndex.java  |   6 +-
 .../internal/index/CompactMapRangeIndex.java    |   7 +-
 .../internal/DistributionConfigImpl.java        |   1 -
 .../gemfire/internal/SharedLibrary.java         |  20 +-
 .../gemfire/internal/cache/CacheConfig.java     |   4 +-
 .../DistTXStateProxyImplOnCoordinator.java      |   2 +
 .../gemfire/internal/cache/ExpiryTask.java      |   2 +-
 .../internal/cache/GemFireCacheImpl.java        |   3 +-
 .../gemfire/internal/cache/LocalRegion.java     |   9 +-
 .../cache/MinimumSystemRequirements.java        |   4 +-
 .../internal/cache/RegionIdleExpiryTask.java    |   3 +-
 .../gemfire/internal/cache/TXManagerImpl.java   |  11 +-
 .../cache/control/HeapMemoryMonitor.java        |  14 +-
 .../internal/cache/control/MemoryEvent.java     |   9 +-
 .../cache/control/OffHeapMemoryMonitor.java     |  88 +-
 .../cache/control/ResourceManagerStats.java     |  13 +
 .../internal/cache/partitioned/GetMessage.java  |   6 +
 .../AbstractGatewaySenderEventProcessor.java    |   1 +
 .../parallel/ParallelGatewaySenderQueue.java    |  41 +-
 .../internal/cache/xmlcache/CacheXmlParser.java |  26 +-
 .../gemfire/internal/lang/SystemUtils.java      |  35 +-
 .../gemfire/internal/logging/LogService.java    |  58 +-
 .../internal/logging/log4j/Configurator.java    |  65 +-
 .../internal/logging/log4j/FastLogger.java      |  50 +-
 .../offheap/SimpleMemoryAllocatorImpl.java      |   2 +-
 .../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 ++++
 .../size/ReflectionSingleObjectSizer.java       |   4 +-
 .../internal/cli/commands/QueueCommands.java    |   4 +-
 .../CreateAsyncEventQueueFunction.java          |   8 +-
 .../internal/cli/i18n/CliStrings.java           |   4 +-
 .../controllers/QueueCommandsController.java    |   6 +-
 .../pdx/ReflectionBasedAutoSerializer.java      |   9 +
 .../pdx/internal/AutoSerializableManager.java   |   8 -
 .../gemfire/pdx/internal/TypeRegistry.java      |   4 +-
 .../gemfire/redis/GemFireRedisServer.java       | 130 +--
 .../internal/RegionWithHDFSBasicDUnitTest.java  |  22 +-
 .../RegionWithHDFSOffHeapBasicDUnitTest.java    |  10 +-
 .../hdfs/internal/RegionWithHDFSTestBase.java   |   6 +-
 .../MemoryThresholdsOffHeapDUnitTest.java       |  42 +-
 .../PutOperationContextJUnitTest.java           | 248 ++++++
 .../GetOperationContextImplJUnitTest.java       | 275 +++++++
 .../query/dunit/CloseCacheAuthorization.java    |   2 -
 .../index/CopyOnReadIndexDUnitTest.java         |  68 +-
 .../index/CopyOnReadIndexJUnitTest.java         |   5 +-
 .../MapRangeIndexMaintenanceJUnitTest.java      |  50 ++
 .../gemfire/cache30/MultiVMRegionTestCase.java  |  94 ++-
 .../gemfire/cache30/RegionTestCase.java         | 137 ++--
 .../LocatorLauncherRemoteFileJUnitTest.java     |   8 +-
 .../LocatorLauncherRemoteJUnitTest.java         |  57 +-
 .../ServerLauncherLocalJUnitTest.java           |  47 +-
 .../ServerLauncherRemoteFileJUnitTest.java      |   2 -
 .../ServerLauncherRemoteJUnitTest.java          |   2 +-
 .../ServerLauncherWithSpringJUnitTest.java      |   2 +
 .../gemfire/disttx/DistTXDebugDUnitTest.java    |   3 +
 .../disttx/DistributedTransactionDUnitTest.java |  19 +-
 .../internal/cache/Bug34011JUnitTest.java       | 199 -----
 .../cache/ClientServerTransactionDUnitTest.java |  38 +-
 .../ConcurrentRegionOperationsJUnitTest.java    |   7 +-
 .../PartitionedRegionSingleHopDUnitTest.java    |   4 -
 .../cache/RemoteTransactionDUnitTest.java       | 113 ++-
 .../control/RebalanceOperationDUnitTest.java    |   2 +
 .../cache/ha/Bug36853EventsExpiryDUnitTest.java |   1 +
 .../DistributedSystemLogFileJUnitTest.java      |  20 +-
 .../logging/LoggingIntegrationTestSuite.java    |  14 +
 .../internal/logging/LoggingUnitTestSuite.java  |  12 +
 .../log4j/FastLoggerIntegrationJUnitTest.java   | 557 +++++++++++++
 .../logging/log4j/FastLoggerJUnitTest.java      | 274 +++----
 .../FastLoggerWithDefaultConfigJUnitTest.java   |  74 ++
 .../log4j/Log4jIntegrationTestSuite.java        |  12 +
 .../logging/log4j/Log4jUnitTestSuite.java       |  16 +
 .../internal/size/ObjectSizerJUnitTest.java     |  14 +-
 .../gemfire/pdx/AutoSerializableJUnitTest.java  |   2 +-
 .../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 +++++++
 .../gemfire/test/golden/ExecutableProcess.java  |   8 +
 .../gemfire/test/golden/FailOutputTestCase.java |  22 +-
 .../golden/FailWithErrorInOutputJUnitTest.java  |  18 +-
 .../FailWithExtraLineInOutputJUnitTest.java     |  41 +-
 ...WithLineMissingFromEndOfOutputJUnitTest.java |  39 +-
 ...hLineMissingFromMiddleOfOutputJUnitTest.java |  39 +-
 .../FailWithLoggerErrorInOutputJUnitTest.java   |  18 +-
 .../FailWithLoggerFatalInOutputJUnitTest.java   |  18 +-
 .../FailWithLoggerWarnInOutputJUnitTest.java    |  18 +-
 .../golden/FailWithProblemInOutputTestCase.java |  30 +-
 .../golden/FailWithSevereInOutputJUnitTest.java |  18 +-
 ...hTimeoutOfWaitForOutputToMatchJUnitTest.java |  42 +-
 .../FailWithWarningInOutputJUnitTest.java       |  18 +-
 .../gemfire/test/golden/GoldenComparator.java   |  64 +-
 .../test/golden/GoldenStringComparator.java     |   5 +-
 .../gemfire/test/golden/GoldenTestCase.java     | 108 +--
 .../golden/GoldenTestFrameworkTestSuite.java    |  27 +
 .../gemfire/test/golden/PassJUnitTest.java      |  50 +-
 .../golden/PassWithExpectedErrorJUnitTest.java  |  18 +-
 .../golden/PassWithExpectedProblemTestCase.java |  58 +-
 .../golden/PassWithExpectedSevereJUnitTest.java |  18 +-
 .../PassWithExpectedWarningJUnitTest.java       |  18 +-
 .../test/golden/RegexGoldenComparator.java      |  10 +-
 .../test/golden/StringGoldenComparator.java     |   4 +-
 .../process/ProcessTestFrameworkTestSuite.java  |  12 +
 .../gemfire/test/process/ProcessWrapper.java    | 251 +++---
 .../test/process/ProcessWrapperJUnitTest.java   |  19 +-
 .../test/java/dunit/DistributedTestCase.java    |  19 +-
 .../gemfire/test/golden/log4j2-test.xml         |  18 +
 gemfire-rebalancer/build.gradle                 |  12 +
 .../gemfire/cache/util/AutoBalancer.java        | 512 ++++++++++++
 .../cache/util/AutoBalancerJUnitTest.java       | 803 +++++++++++++++++++
 .../connector/GemFirePairRDDFunctions.scala     |   5 +-
 .../spark/connector/GemFireRDDFunctions.scala   |   5 +-
 .../internal/DefaultGemFireConnection.scala     |  30 +-
 .../connector/internal/LocatorHelper.scala      |  91 ++-
 .../internal/rdd/GemFireRegionRDD.scala         |   2 +-
 .../gemfire/spark/connector/package.scala       |   7 +
 .../spark/connector/LocatorHelperTest.scala     |  77 ++
 settings.gradle                                 |   1 +
 223 files changed, 7167 insertions(+), 2385 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/gemfire-assembly/build.gradle
----------------------------------------------------------------------
diff --cc gemfire-assembly/build.gradle
index a50d3ff,f65930d..7e7dcb3
--- a/gemfire-assembly/build.gradle
+++ b/gemfire-assembly/build.gradle
@@@ -112,7 -113,7 +112,8 @@@ def cp = 
        it.contains('spring-shell') ||
        it.contains('snappy-java') ||
        it.contains('hbase') ||
 +      it.contains('jgroups')
+       it.contains('netty')
      }.join(' ') 
  }
  

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/gemfire-core/build.gradle
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/gemfire-core/src/main/java/com/gemstone/gemfire/distributed/internal/DistributionConfigImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/LocalRegion.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
index e12f83f,a4a399d..877fdb1
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/logging/LogService.java
@@@ -117,12 -119,13 +119,13 @@@ public class LogService extends LogMana
    /**
     * Check to see if the user has specified a Log4j configuration file.  If not, attempt
     * to find a GemFire Log4j configuration file in various locations.
+    * 
+    * @return true if log4j.configurationFile property was set; false if it was unchanged
     */
-   private static final void setLog4jConfigFileProperty() {
+   private static final boolean setLog4jConfigFileProperty() {
      // fix bug #52175
      final URL configInClasspath = ConfigLocator.findConfigInClasspath();
 -    if (configInClasspath != null ) {
 +    if (configInClasspath != null) {
- //      System.out.println("log config is " + configInClasspath);
        // Log4J 2 will find the configuration file in classpath so do nothing
        configFileInformation = "Using log4j configuration found in classpath: '" + configInClasspath.toString() + "'";
        StatusLogger.getLogger().info(configFileInformation);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionProvider.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/RegionProvider.java
index 0000000,a95f853..18971eb
mode 000000,100644..100644
--- 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
@@@ -1,0 -1,531 +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);
++      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, 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/e0d1c4f9/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/gemfire-core/src/test/java/com/gemstone/gemfire/cache/management/MemoryThresholdsOffHeapDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexDUnitTest.java
----------------------------------------------------------------------
diff --cc gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexDUnitTest.java
index e66c047,61bb55d..5ad8537
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexDUnitTest.java
@@@ -598,12 -601,25 +600,24 @@@ public class CopyOnReadIndexDUnitTest e
      return p;
    }
  
 -  protected Properties getServerProperties(int mcastPort) {
 +  protected Properties getServerProperties() {
      Properties p = new Properties();
 -    p.setProperty(DistributionConfig.MCAST_PORT_NAME, mcastPort+"");
 -    p.setProperty(DistributionConfig.LOCATORS_NAME, "");
 +    p.setProperty(DistributionConfig.LOCATORS_NAME, "localhost["+getDUnitLocatorPort()+"]");
      return p;
    }
-   
+ 
+   private WaitCriterion verifyPortfolioCount(final int expected) {
+     return new WaitCriterion() {
+       private int expectedCount = expected;
+       public boolean done() {
+         return expectedCount == Portfolio.instanceCount.get();
+       }
+       
+       public String description() {
+         return "verifying number of object instances created";
+       }
+     };
+   }
    
  
  }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/gemfire-core/src/test/java/dunit/DistributedTestCase.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e0d1c4f9/settings.gradle
----------------------------------------------------------------------



[06/50] [abbrv] incubator-geode git commit: GEODE-127: Improve test reliability and execution speed.

Posted by bs...@apache.org.
GEODE-127: Improve test reliability and execution speed.

Recategorize tests involving spawned processes and file system I/O as
IntegrationTests.

Improve reliability and shorten execution time. Fix up asynchronous waits,
correct JUnit 4 syntax and misc code tidying.

Add new TestSuite classes for targeted testing of test.golden and
test.process packages.


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

Branch: refs/heads/feature/GEODE-77
Commit: 94939c1f9cc3e09c4beee1155404dd82b9aca13f
Parents: ea8ce3f
Author: Kirk Lund <kl...@pivotal.io>
Authored: Mon Jul 27 16:35:33 2015 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Wed Jul 29 09:33:03 2015 -0700

----------------------------------------------------------------------
 .../ServerLauncherRemoteJUnitTest.java          |   2 +-
 .../gemfire/test/golden/ExecutableProcess.java  |   8 +
 .../gemfire/test/golden/FailOutputTestCase.java |  22 +-
 .../golden/FailWithErrorInOutputJUnitTest.java  |  18 +-
 .../FailWithExtraLineInOutputJUnitTest.java     |  41 +--
 ...WithLineMissingFromEndOfOutputJUnitTest.java |  39 ++-
 ...hLineMissingFromMiddleOfOutputJUnitTest.java |  39 ++-
 .../FailWithLoggerErrorInOutputJUnitTest.java   |  18 +-
 .../FailWithLoggerFatalInOutputJUnitTest.java   |  18 +-
 .../FailWithLoggerWarnInOutputJUnitTest.java    |  18 +-
 .../golden/FailWithProblemInOutputTestCase.java |  30 ++-
 .../golden/FailWithSevereInOutputJUnitTest.java |  18 +-
 ...hTimeoutOfWaitForOutputToMatchJUnitTest.java |  43 ++--
 .../FailWithWarningInOutputJUnitTest.java       |  18 +-
 .../gemfire/test/golden/GoldenComparator.java   |  64 +++--
 .../test/golden/GoldenStringComparator.java     |   5 +-
 .../gemfire/test/golden/GoldenTestCase.java     | 108 +++-----
 .../golden/GoldenTestFrameworkTestSuite.java    |  27 ++
 .../gemfire/test/golden/PassJUnitTest.java      |  50 ++--
 .../golden/PassWithExpectedErrorJUnitTest.java  |  18 +-
 .../golden/PassWithExpectedProblemTestCase.java |  58 +++--
 .../golden/PassWithExpectedSevereJUnitTest.java |  18 +-
 .../PassWithExpectedWarningJUnitTest.java       |  18 +-
 .../test/golden/RegexGoldenComparator.java      |  10 +-
 .../test/golden/StringGoldenComparator.java     |   4 +-
 .../process/ProcessTestFrameworkTestSuite.java  |  12 +
 .../gemfire/test/process/ProcessWrapper.java    | 251 +++++++++----------
 .../test/process/ProcessWrapperJUnitTest.java   |  19 +-
 .../gemfire/test/golden/log4j2-test.xml         |  18 ++
 29 files changed, 486 insertions(+), 526 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
index fb0df63..1ba1189 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteJUnitTest.java
@@ -122,7 +122,7 @@ public class ServerLauncherRemoteJUnitTest extends AbstractServerLauncherJUnitTe
     File file = new File(this.temporaryFolder.getRoot(), ServerLauncherForkingProcess.class.getSimpleName().concat(".log"));
     //-logger.info("KIRK: log file is " + file);
     
-    final ProcessWrapper pw = new ProcessWrapper.Builder().main(ServerLauncherForkingProcess.class).build();
+    final ProcessWrapper pw = new ProcessWrapper.Builder().mainClass(ServerLauncherForkingProcess.class).build();
     pw.execute(null, this.temporaryFolder.getRoot()).waitFor(true);
     //logger.info("[testRunningServerOutlivesForkingProcess] ServerLauncherForkingProcess output is:\n\n"+pw.getOutput());
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/ExecutableProcess.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/ExecutableProcess.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/ExecutableProcess.java
new file mode 100755
index 0000000..bc6d1e3
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/ExecutableProcess.java
@@ -0,0 +1,8 @@
+package com.gemstone.gemfire.test.golden;
+
+/**
+ * Defines the work that is executed within a remote process. 
+ */
+public interface ExecutableProcess {
+  public void executeInProcess() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailOutputTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailOutputTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailOutputTestCase.java
index a3995f8..d460706 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailOutputTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailOutputTestCase.java
@@ -10,11 +10,7 @@ import java.io.InputStreamReader;
  * 
  * @author Kirk Lund
  */
-public abstract class FailOutputTestCase extends GoldenTestCase {
-  
-  FailOutputTestCase(String name) {
-    super(name);
-  }
+public abstract class FailOutputTestCase extends GoldenTestCase implements ExecutableProcess {
   
   @Override
   protected GoldenComparator createGoldenComparator() {
@@ -27,14 +23,14 @@ public abstract class FailOutputTestCase extends GoldenTestCase {
   
   abstract String problem();
   
-  abstract void outputProblem(String message);
+  abstract void outputProblemInProcess(String message);
   
-  void execute() throws IOException {
-    System.out.println("Begin " + name() + ".main");
-    System.out.println("Press Enter to continue.");
-    BufferedReader inputReader = new BufferedReader(new InputStreamReader(System.in));
-    inputReader.readLine();
-    outputProblem(problem());
-    System.out.println("End " + name() + ".main");
+  @Override
+  public final void executeInProcess() throws IOException {
+    outputLine("Begin " + name() + ".main");
+    outputLine("Press Enter to continue.");
+    new BufferedReader(new InputStreamReader(System.in)).readLine();
+    outputProblemInProcess(problem());
+    outputLine("End " + name() + ".main");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithErrorInOutputJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithErrorInOutputJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithErrorInOutputJUnitTest.java
index b3ca93e..d8763b7 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithErrorInOutputJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithErrorInOutputJUnitTest.java
@@ -2,10 +2,9 @@ package com.gemstone.gemfire.test.golden;
 
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogWriterImpl;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that test output containing an unexpected error message
@@ -13,25 +12,20 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FailWithErrorInOutputJUnitTest extends FailWithProblemInOutputTestCase {
   
-  public FailWithErrorInOutputJUnitTest() {
-    super(FailWithErrorInOutputJUnitTest.class.getSimpleName());
-  }
-  
   @Override
   String problem() {
     return "ExpectedStrings: Description of a problem.";
   }
   
   @Override
-  void outputProblem(String message) {
-    LogWriter logWriter = new LocalLogWriter(LogWriterImpl.INFO_LEVEL);
-    logWriter.error(message);
+  void outputProblemInProcess(final String message) {
+    new LocalLogWriter(LogWriterImpl.INFO_LEVEL).error(message);
   }
   
-  public static void main(String[] args) throws Exception {
-    new FailWithErrorInOutputJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new FailWithErrorInOutputJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithExtraLineInOutputJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithExtraLineInOutputJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithExtraLineInOutputJUnitTest.java
index 1c49749..6fb674f 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithExtraLineInOutputJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithExtraLineInOutputJUnitTest.java
@@ -1,13 +1,12 @@
 package com.gemstone.gemfire.test.golden;
 
-import java.io.IOException;
+import static org.junit.Assert.*;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.process.ProcessWrapper;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-import junit.framework.AssertionFailedError;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that test output containing an unexpected extra line
@@ -15,45 +14,47 @@ import junit.framework.AssertionFailedError;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FailWithExtraLineInOutputJUnitTest extends FailOutputTestCase {
   
-  public FailWithExtraLineInOutputJUnitTest() {
-    super("FailWithExtraLineInOutputJUnitTest");
-  }
-  
   @Override
   String problem() {
     return "This is an extra line";
   }
   
   @Override
-  void outputProblem(String message) {
+  void outputProblemInProcess(final String message) {
     System.out.println(message);
   }
   
-  public void testFailWithExtraLineInOutput() throws InterruptedException, IOException {
-    // output has an extra line and should fail
-    final ProcessWrapper process = createProcessWrapper(getClass());
+  /**
+   * Process output has an extra line and should fail
+   */
+  @Test
+  public void testFailWithExtraLineInOutput() throws Exception {
+    final String goldenString = 
+        "Begin " + name() + ".main" + "\n" +
+        "Press Enter to continue." + "\n" + 
+        "End " + name() + ".main" + "\n";
+    debug(goldenString, "GOLDEN");
+
+    final ProcessWrapper process = createProcessWrapper(new ProcessWrapper.Builder(), getClass());
     process.execute(createProperties());
     process.waitForOutputToMatch("Begin " + name() + "\\.main");
     process.waitForOutputToMatch("Press Enter to continue\\.");
     process.sendInput();
     process.waitForOutputToMatch("End " + name() + "\\.main");
     process.waitFor();
-    String goldenString = "Begin " + name() + ".main" + "\n" 
-        + "Press Enter to continue." + "\n" 
-        + "End " + name() + ".main" + "\n";
-    innerPrintOutput(goldenString, "GOLDEN");
+    
     try {
       assertOutputMatchesGoldenFile(process.getOutput(), goldenString);
       fail("assertOutputMatchesGoldenFile should have failed due to " + problem());
-    } catch (AssertionFailedError expected) {
+    } catch (AssertionError expected) {
       assertTrue(expected.getMessage().contains(problem()));
     }
   }
   
-  public static void main(String[] args) throws Exception {
-    new FailWithExtraLineInOutputJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new FailWithExtraLineInOutputJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLineMissingFromEndOfOutputJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLineMissingFromEndOfOutputJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLineMissingFromEndOfOutputJUnitTest.java
index 877f671..a1802c5 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLineMissingFromEndOfOutputJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLineMissingFromEndOfOutputJUnitTest.java
@@ -1,13 +1,12 @@
 package com.gemstone.gemfire.test.golden;
 
-import java.io.IOException;
+import static org.junit.Assert.*;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.process.ProcessWrapper;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-import junit.framework.AssertionFailedError;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that test output missing an expected line (at the end of
@@ -15,46 +14,46 @@ import junit.framework.AssertionFailedError;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FailWithLineMissingFromEndOfOutputJUnitTest extends FailOutputTestCase {
 
-  public FailWithLineMissingFromEndOfOutputJUnitTest() {
-    super("FailWithLineMissingFromOutputJUnitTest");
-  }
-  
   @Override
   String problem() {
     return "This line is missing in actual output.";
   }
   
   @Override
-  void outputProblem(String message) {
+  void outputProblemInProcess(final String message) {
     // this tests that the message is missing from output
   }
   
-  public void testFailWithLineMissingFromEndOfOutput() throws InterruptedException, IOException {
-    final ProcessWrapper process = createProcessWrapper(getClass());
+  @Test
+  public void testFailWithLineMissingFromEndOfOutput() throws Exception {
+    final String goldenString = 
+        "Begin " + name() + ".main" + "\n" +
+        "Press Enter to continue." + "\n" +
+        "End " + name() + ".main" + "\n" +
+        problem() + "\n"; 
+    debug(goldenString, "GOLDEN");
+
+    final ProcessWrapper process = createProcessWrapper(new ProcessWrapper.Builder(), getClass());
     process.execute(createProperties());
     process.waitForOutputToMatch("Begin " + name() + "\\.main");
     process.waitForOutputToMatch("Press Enter to continue\\.");
     process.sendInput();
     process.waitForOutputToMatch("End " + name() + "\\.main");
     process.waitFor();
-    String goldenString = "Begin " + name() + ".main" + "\n" 
-        + "Press Enter to continue." + "\n" 
-        + "End " + name() + ".main" + "\n"
-        + problem() + "\n"; 
-    innerPrintOutput(goldenString, "GOLDEN");
+    
     try {
       assertOutputMatchesGoldenFile(process.getOutput(), goldenString);
       fail("assertOutputMatchesGoldenFile should have failed due to " + problem());
-    } catch (AssertionFailedError expected) {
+    } catch (AssertionError expected) {
       assertTrue("AssertionFailedError message should contain \"" + problem() + "\"", 
           expected.getMessage().contains(problem()));
     }
   }
   
-  public static void main(String[] args) throws Exception {
-    new FailWithLineMissingFromEndOfOutputJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new FailWithLineMissingFromEndOfOutputJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLineMissingFromMiddleOfOutputJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLineMissingFromMiddleOfOutputJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLineMissingFromMiddleOfOutputJUnitTest.java
index 629eb7f..91095ec 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLineMissingFromMiddleOfOutputJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLineMissingFromMiddleOfOutputJUnitTest.java
@@ -1,13 +1,12 @@
 package com.gemstone.gemfire.test.golden;
 
-import java.io.IOException;
+import static org.junit.Assert.*;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.process.ProcessWrapper;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-import junit.framework.AssertionFailedError;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that test output missing an expected line (at the middle 
@@ -15,45 +14,45 @@ import junit.framework.AssertionFailedError;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FailWithLineMissingFromMiddleOfOutputJUnitTest extends FailOutputTestCase {
 
-  public FailWithLineMissingFromMiddleOfOutputJUnitTest() {
-    super("FailWithLineMissingFromMiddleOfOutputJUnitTest");
-  }
-  
   @Override
   String problem() {
     return "This line is missing in actual output.";
   }
   
   @Override
-  void outputProblem(String message) {
+  void outputProblemInProcess(final String message) {
     // this tests that the message is missing from output
   }
   
-  public void testFailWithLineMissingFromEndOfOutput() throws InterruptedException, IOException {
-    final ProcessWrapper process = createProcessWrapper(getClass());
+  @Test
+  public void testFailWithLineMissingFromEndOfOutput() throws Exception {
+    final String goldenString = 
+        "Begin " + name() + ".main" + "\n" +
+        "Press Enter to continue." + "\n" +
+        problem() + "\n" +
+        "End " + name() + ".main" + "\n";
+    debug(goldenString, "GOLDEN");
+
+    final ProcessWrapper process = createProcessWrapper(new ProcessWrapper.Builder(), getClass());
     process.execute(createProperties());
     process.waitForOutputToMatch("Begin " + name() + "\\.main");
     process.waitForOutputToMatch("Press Enter to continue\\.");
     process.sendInput();
     process.waitFor();
-    String goldenString = "Begin " + name() + ".main" + "\n" 
-        + "Press Enter to continue." + "\n" 
-        + problem() + "\n"
-        + "End " + name() + ".main" + "\n";
-    innerPrintOutput(goldenString, "GOLDEN");
+    
     try {
       assertOutputMatchesGoldenFile(process.getOutput(), goldenString);
       fail("assertOutputMatchesGoldenFile should have failed due to " + problem());
-    } catch (AssertionFailedError expected) {
+    } catch (AssertionError expected) {
       assertTrue("AssertionFailedError message should contain \"" + problem() + "\"", 
           expected.getMessage().contains(problem()));
     }
   }
   
-  public static void main(String[] args) throws Exception {
-    new FailWithLineMissingFromMiddleOfOutputJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new FailWithLineMissingFromMiddleOfOutputJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerErrorInOutputJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerErrorInOutputJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerErrorInOutputJUnitTest.java
index b485ba4..5a2c49a 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerErrorInOutputJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerErrorInOutputJUnitTest.java
@@ -1,10 +1,9 @@
 package com.gemstone.gemfire.test.golden;
 
-import org.apache.logging.log4j.Logger;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that test output containing an unexpected ERROR message
@@ -12,25 +11,20 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FailWithLoggerErrorInOutputJUnitTest extends FailWithProblemInOutputTestCase {
   
-  public FailWithLoggerErrorInOutputJUnitTest() {
-    super(FailWithLoggerErrorInOutputJUnitTest.class.getSimpleName());
-  }
-  
   @Override
   String problem() {
     return "ExpectedStrings: Description of a problem.";
   }
   
   @Override
-  void outputProblem(String message) {
-    Logger logger = LogService.getLogger();
-    logger.error(message);
+  void outputProblemInProcess(final String message) {
+    LogService.getLogger().error(message);
   }
   
-  public static void main(String[] args) throws Exception {
-    new FailWithLoggerErrorInOutputJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new FailWithLoggerErrorInOutputJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerFatalInOutputJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerFatalInOutputJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerFatalInOutputJUnitTest.java
index a71ce18..a0c681a 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerFatalInOutputJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerFatalInOutputJUnitTest.java
@@ -1,10 +1,9 @@
 package com.gemstone.gemfire.test.golden;
 
-import org.apache.logging.log4j.Logger;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that test output containing an unexpected FATAL message
@@ -12,25 +11,20 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FailWithLoggerFatalInOutputJUnitTest extends FailWithProblemInOutputTestCase {
   
-  public FailWithLoggerFatalInOutputJUnitTest() {
-    super(FailWithLoggerFatalInOutputJUnitTest.class.getSimpleName());
-  }
-  
   @Override
   String problem() {
     return "ExpectedStrings: Description of a problem.";
   }
   
   @Override
-  void outputProblem(String message) {
-    Logger logger = LogService.getLogger();
-    logger.fatal(message);
+  void outputProblemInProcess(final String message) {
+    LogService.getLogger().fatal(message);
   }
   
-  public static void main(String[] args) throws Exception {
-    new FailWithLoggerFatalInOutputJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new FailWithLoggerFatalInOutputJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerWarnInOutputJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerWarnInOutputJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerWarnInOutputJUnitTest.java
index 211008a..5717f67 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerWarnInOutputJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithLoggerWarnInOutputJUnitTest.java
@@ -1,10 +1,9 @@
 package com.gemstone.gemfire.test.golden;
 
-import org.apache.logging.log4j.Logger;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.internal.logging.LogService;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that test output containing an unexpected WARN message
@@ -12,25 +11,20 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FailWithLoggerWarnInOutputJUnitTest extends FailWithProblemInOutputTestCase {
   
-  public FailWithLoggerWarnInOutputJUnitTest() {
-    super(FailWithLoggerWarnInOutputJUnitTest.class.getSimpleName());
-  }
-  
   @Override
   String problem() {
     return "ExpectedStrings: Description of a problem.";
   }
   
   @Override
-  void outputProblem(String message) {
-    Logger logger = LogService.getLogger();
-    logger.warn(message);
+  void outputProblemInProcess(final String message) {
+    LogService.getLogger().warn(message);
   }
   
-  public static void main(String[] args) throws Exception {
-    new FailWithLoggerWarnInOutputJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new FailWithLoggerWarnInOutputJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithProblemInOutputTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithProblemInOutputTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithProblemInOutputTestCase.java
index 5e1552f..7a68e02 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithProblemInOutputTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithProblemInOutputTestCase.java
@@ -1,10 +1,10 @@
 package com.gemstone.gemfire.test.golden;
 
-import java.io.IOException;
+import static org.junit.Assert.*;
 
-import com.gemstone.gemfire.test.process.ProcessWrapper;
+import org.junit.Test;
 
-import junit.framework.AssertionFailedError;
+import com.gemstone.gemfire.test.process.ProcessWrapper;
 
 /**
  * Abstract test case for tests verifying that test output with a
@@ -14,33 +14,31 @@ import junit.framework.AssertionFailedError;
  */
 public abstract class FailWithProblemInOutputTestCase extends FailOutputTestCase {
 
-  FailWithProblemInOutputTestCase(String name) {
-    super(name);
-  }
-  
   @Override
   protected String[] expectedProblemLines() {
     return new String[] { ".*" + name() + ".*" };
   }
   
-  public void testFailWithProblemLogMessageInOutput() throws InterruptedException, IOException {
-    final ProcessWrapper process = createProcessWrapper(getClass());
+  @Test
+  public void testFailWithProblemLogMessageInOutput() throws Exception {
+    final String goldenString = 
+        "Begin " + name() + ".main" + "\n" + 
+        "Press Enter to continue." + "\n" +
+        "End " + name() + ".main" + "\n";
+    debug(goldenString, "GOLDEN");
+
+    final ProcessWrapper process = createProcessWrapper(new ProcessWrapper.Builder(), getClass());
     process.execute(createProperties());
     process.waitForOutputToMatch("Begin " + name() + "\\.main");
     process.waitForOutputToMatch("Press Enter to continue\\.");
     process.sendInput();
     process.waitForOutputToMatch("End " + name() + "\\.main");
     process.waitFor();
-    String goldenString = "Begin " + name() + ".main" + "\n" 
-        + "Press Enter to continue." + "\n" 
-        + "End " + name() + ".main" + "\n";
-    innerPrintOutput(goldenString, "GOLDEN");
+    
     try {
       assertOutputMatchesGoldenFile(process.getOutput(), goldenString);
       fail("assertOutputMatchesGoldenFile should have failed due to " + problem());
-    } catch (AssertionFailedError expected) {
-//      System.out.println("Problem: " + problem());
-//      System.out.println("AssertionFailedError message: " + expected.getMessage());
+    } catch (AssertionError expected) {
       assertTrue(expected.getMessage().contains(problem()));
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithSevereInOutputJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithSevereInOutputJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithSevereInOutputJUnitTest.java
index 9089545..457c7cc 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithSevereInOutputJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithSevereInOutputJUnitTest.java
@@ -2,10 +2,9 @@ package com.gemstone.gemfire.test.golden;
 
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogWriterImpl;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that test output containing an unexpected severe message
@@ -13,25 +12,20 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FailWithSevereInOutputJUnitTest extends FailWithProblemInOutputTestCase {
   
-  public FailWithSevereInOutputJUnitTest() {
-    super(FailWithSevereInOutputJUnitTest.class.getSimpleName());
-  }
-  
   @Override
   String problem() {
     return "ExpectedStrings: Description of a problem.";
   }
   
   @Override
-  void outputProblem(String message) {
-    LogWriter logWriter = new LocalLogWriter(LogWriterImpl.INFO_LEVEL);
-    logWriter.severe(message);
+  void outputProblemInProcess(final String message) {
+    new LocalLogWriter(LogWriterImpl.INFO_LEVEL).severe(message);
   }
   
-  public static void main(String[] args) throws Exception {
-    new FailWithSevereInOutputJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new FailWithSevereInOutputJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java
index 2417f3d..1b8a312 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithTimeoutOfWaitForOutputToMatchJUnitTest.java
@@ -1,17 +1,23 @@
 package com.gemstone.gemfire.test.golden;
 
+import static org.junit.Assert.*;
+
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.process.ProcessWrapper;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-import junit.framework.AssertionFailedError;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FailWithTimeoutOfWaitForOutputToMatchJUnitTest extends FailOutputTestCase {
   
-  public FailWithTimeoutOfWaitForOutputToMatchJUnitTest() {
-    super("FailWithTimeoutOfWaitForOutputToMatchJUnitTest");
+  private static final long timeoutMillis = 1000;
+
+  private ProcessWrapper process;
+  
+  public void subTearDown() throws Exception {
+    this.process.waitFor();
+    assertFalse(this.process.isAlive());
   }
   
   @Override
@@ -20,29 +26,28 @@ public class FailWithTimeoutOfWaitForOutputToMatchJUnitTest extends FailOutputTe
   }
   
   @Override
-  void outputProblem(String message) {
+  void outputProblemInProcess(final String message) {
     System.out.println(message);
   }
   
+  /**
+   * Process output has an extra line and should fail
+   */
+  @Test
   public void testFailWithTimeoutOfWaitForOutputToMatch() throws Exception {
-    // output has an extra line and should fail
-    final ProcessWrapper process = createProcessWrapper(getClass());
-    process.execute(createProperties());
-    process.waitForOutputToMatch("Begin " + name() + "\\.main");
+    this.process = createProcessWrapper(new ProcessWrapper.Builder().timeoutMillis(timeoutMillis), getClass());
+    this.process.execute(createProperties());
+    this.process.waitForOutputToMatch("Begin " + name() + "\\.main");
+    
     try {
-      process.waitForOutputToMatch(problem());
+      this.process.waitForOutputToMatch(problem());
       fail("assertOutputMatchesGoldenFile should have failed due to " + problem());
-    } catch (AssertionFailedError expected) {
+    } catch (AssertionError expected) {
       assertTrue(expected.getMessage().contains(problem()));
     }
-    // the following should generate no failures if timeout and tearDown are all working properly
-    assertNotNull(process);
-    assertTrue(process.isAlive());
-    tearDown();
-    process.waitFor();
   }
   
   public static void main(String[] args) throws Exception {
-    new FailWithTimeoutOfWaitForOutputToMatchJUnitTest().execute();
+    new FailWithTimeoutOfWaitForOutputToMatchJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithWarningInOutputJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithWarningInOutputJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithWarningInOutputJUnitTest.java
index 89cc7dc..c54ff85 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithWarningInOutputJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/FailWithWarningInOutputJUnitTest.java
@@ -2,10 +2,9 @@ package com.gemstone.gemfire.test.golden;
 
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogWriterImpl;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that test output containing an unexpected warning message
@@ -13,25 +12,20 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class FailWithWarningInOutputJUnitTest extends FailWithProblemInOutputTestCase {
   
-  public FailWithWarningInOutputJUnitTest() {
-    super(FailWithWarningInOutputJUnitTest.class.getSimpleName());
-  }
-  
   @Override
   String problem() {
     return "ExpectedStrings: Description of a problem.";
   }
   
   @Override
-  void outputProblem(String message) {
-    LogWriter logWriter = new LocalLogWriter(LogWriterImpl.INFO_LEVEL);
-    logWriter.warning(message);
+  void outputProblemInProcess(final String message) {
+    new LocalLogWriter(LogWriterImpl.INFO_LEVEL).warning(message);
   }
   
-  public static void main(String[] args) throws Exception {
-    new FailWithWarningInOutputJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new FailWithWarningInOutputJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenComparator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenComparator.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenComparator.java
index 34c1ce7..6244237 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenComparator.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenComparator.java
@@ -1,5 +1,7 @@
 package com.gemstone.gemfire.test.golden;
 
+import static org.junit.Assert.*;
+
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
@@ -9,44 +11,37 @@ import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.logging.log4j.Logger;
-
-import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.process.OutputFormatter;
 
-import junit.framework.Assert;
-
 /**
  * Compares test output to golden text file.
  * 
  * @author Kirk Lund
  * @since 4.1.1
  */
-public abstract class GoldenComparator extends Assert {
+public abstract class GoldenComparator {
 
-  //private static final boolean ALLOW_EXTRA_WHITESPACE = true;
-  
-  protected final Logger logger = LogService.getLogger();
+  //TODO: private static final boolean ALLOW_EXTRA_WHITESPACE = true;
   
   private String[] expectedProblemLines;
   
-  protected GoldenComparator(String[] expectedProblemLines) {
+  protected GoldenComparator(final String[] expectedProblemLines) {
     this.expectedProblemLines = expectedProblemLines;
   }
   
-  protected Reader readGoldenFile(String goldenFileName) throws IOException {
-    InputStream goldenStream = ClassLoader.getSystemResourceAsStream(goldenFileName);
+  protected Reader readGoldenFile(final String goldenFileName) throws IOException {
+    final InputStream goldenStream = ClassLoader.getSystemResourceAsStream(goldenFileName);
     assertNotNull("Golden file " + goldenFileName + " not found.", goldenStream);
     return new InputStreamReader(goldenStream);
   }
   
-  public void assertOutputMatchesGoldenFile(String actualOutput, String goldenFileName) throws IOException {
-    logger.debug(GoldenTestCase.GOLDEN_TEST, "GoldenComparator:assertOutputMatchesGoldenFile");
-    BufferedReader goldenReader = new BufferedReader(readGoldenFile(goldenFileName));
-    BufferedReader actualReader = new BufferedReader(new StringReader(actualOutput));
+  public void assertOutputMatchesGoldenFile(final String actualOutput, final String goldenFileName) throws IOException {
+    debug("GoldenComparator:assertOutputMatchesGoldenFile");
+    final BufferedReader goldenReader = new BufferedReader(readGoldenFile(goldenFileName));
+    final BufferedReader actualReader = new BufferedReader(new StringReader(actualOutput));
     
-    List<String> goldenStrings = readLines(goldenReader);
-    List<String> actualStrings = readLines(actualReader);
+    final List<String> goldenStrings = readLines(goldenReader);
+    final List<String> actualStrings = readLines(actualReader);
 
     scanForProblems(actualStrings);
     
@@ -56,12 +51,11 @@ public abstract class GoldenComparator extends Assert {
     int lineCount = 0;
     do {
       lineCount++;
-      logger.debug(GoldenTestCase.GOLDEN_TEST, "GoldenComparator comparing line {}", lineCount);
+      debug("GoldenComparator comparing line " + lineCount);
 
       actualLine = actualStrings.get(lineCount - 1);
       goldenLine = goldenStrings.get(lineCount - 1);
       
-      //checkForProblem(lineCount, actualLine);
       if (actualLine == null && goldenLine != null) {
         fail("EOF reached in actual output but golden file, " + goldenFileName + ", continues at line " + lineCount + ": " + goldenLine + new OutputFormatter(actualStrings));
       
@@ -69,11 +63,7 @@ public abstract class GoldenComparator extends Assert {
         fail("EOF reached in golden file, " + goldenFileName + ", but actual output continues at line " + lineCount + ": " + actualLine + new OutputFormatter(actualStrings));
       
       } else if (actualLine != null && goldenLine != null) {
-        assertTrue("Actual output \"" + actualLine
-            + "\" did not match expected pattern \"" + goldenLine
-            + "\" at line " + lineCount + " in " + goldenFileName 
-            + ": " + new OutputFormatter(actualStrings), 
-            compareLines(actualLine, goldenLine));
+        assertTrue("Actual output \"" + actualLine + "\" did not match expected pattern \"" + goldenLine + "\" at line " + lineCount + " in " + goldenFileName + ": " + new OutputFormatter(actualStrings), compareLines(actualLine, goldenLine));
       }
     } while (actualLine != null && goldenLine != null);
   }
@@ -81,10 +71,10 @@ public abstract class GoldenComparator extends Assert {
   /**
    * Returns true if the line matches and is ok. Otherwise returns false.
    */
-  protected abstract boolean compareLines(String actualLine, String goldenLine);
+  protected abstract boolean compareLines(final String actualLine, final String goldenLine);
   
-  private List<String> readLines(BufferedReader reader) throws IOException {
-    List<String> listOfLines = new ArrayList<String>();
+  private List<String> readLines(final BufferedReader reader) throws IOException {
+    final List<String> listOfLines = new ArrayList<String>();
     String line = null;
     do {
       line = reader.readLine();
@@ -93,17 +83,17 @@ public abstract class GoldenComparator extends Assert {
     return listOfLines;
   }
   
-  private void scanForProblems(List<String> lines) throws IOException {
-    logger.debug(GoldenTestCase.GOLDEN_TEST, "GoldenComparator:scanForProblems");
+  private void scanForProblems(final List<String> lines) throws IOException {
+    debug("GoldenComparator:scanForProblems");
     int lineCount = 0;
     for (String line : lines) {
       lineCount++;
-      logger.debug(GoldenTestCase.GOLDEN_TEST, "GoldenComparator:scanForProblems scanning line {}", lineCount);
+      debug("GoldenComparator:scanForProblems scanning line " + lineCount);
       checkForProblem(lineCount, line);
     }
   }
   
-  private void checkForProblem(int lineCount, String line) {
+  private void checkForProblem(final int lineCount, final String line) {
     if (line == null) {
       return;
     }
@@ -114,19 +104,23 @@ public abstract class GoldenComparator extends Assert {
     checkLineFor(lineCount, line, "severe");
   }
   
-  private void checkLineFor(int lineCount, String line, String problem) {
+  private void checkLineFor(final int lineCount, final String line, final String problem) {
     if (line != null && line.toLowerCase().contains(problem)) {
       if (this.expectedProblemLines != null && this.expectedProblemLines.length > 0) {
         for (int i = 0; i < this.expectedProblemLines.length; i++) {
-          logger.debug(GoldenTestCase.GOLDEN_TEST, "Comparing \"{}\" against expected \"{}\"", line, this.expectedProblemLines[i]);
+          debug("Comparing \" + line + \" against expected \" + this.expectedProblemLines[i] + \"");
           if (compareLines(line, this.expectedProblemLines[i])) {
             return;
           }
         }
       }
       // TODO: collect up entire stack trace if there is one (might span multiple lines)
-      logger.debug(GoldenTestCase.GOLDEN_TEST, "About to fail because of {}", line);
+      debug("About to fail because of " + line);
       fail("Actual output contains a problem (warning/error/severe) on line " + lineCount + ": " + line);
     }
   }
+
+  protected static void debug(final String string) {
+    GoldenTestCase.debug(string);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenStringComparator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenStringComparator.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenStringComparator.java
index 75a3398..8b9fac1 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenStringComparator.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenStringComparator.java
@@ -12,11 +12,12 @@ import java.io.StringReader;
  */
 public class GoldenStringComparator extends RegexGoldenComparator {
 
-  protected GoldenStringComparator(String[] expectedProblemLines) {
+  protected GoldenStringComparator(final String[] expectedProblemLines) {
     super(expectedProblemLines);
   }
   
-  protected Reader readGoldenFile(String goldenFileName) throws IOException {
+  @Override
+  protected Reader readGoldenFile(final String goldenFileName) throws IOException {
     return new StringReader(goldenFileName);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestCase.java
index 37b4e48..cac6748 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestCase.java
@@ -1,62 +1,43 @@
 package com.gemstone.gemfire.test.golden;
 
 import java.io.IOException;
-import java.net.URL;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
-import org.apache.logging.log4j.Logger;
-import org.apache.logging.log4j.Marker;
-import org.apache.logging.log4j.MarkerManager;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
+import org.junit.After;
+import org.junit.Before;
 
-import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.ClassPathLoader;
-import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.test.process.ProcessWrapper;
 
-import junit.framework.TestCase;
-
 /**
- * The abstract superclass of tests that need to process output from the
- * quickstart examples.
+ * Test framework for launching processes and comparing output to expected golden output.
  *
  * @author Kirk Lund
  * @since 4.1.1
  */
-public abstract class GoldenTestCase extends TestCase {
-  protected static final Marker GOLDEN_TEST = MarkerManager.getMarker("GOLDEN_TEST");
+public abstract class GoldenTestCase {
 
-  protected final Logger logger = LogService.getLogger();
-  
-  private final int mcastPort = AvailablePort.getRandomAvailablePort(AvailablePort.JGROUPS);
-  private final List<ProcessWrapper> processes = new ArrayList<ProcessWrapper>();
+  /** Use to enable debug output in the JUnit process */
+  protected static final String DEBUG_PROPERTY = "golden.test.DEBUG";
+  protected static final boolean DEBUG = Boolean.getBoolean(DEBUG_PROPERTY);
   
-  static {
-    final URL configUrl = GoldenTestCase.class.getResource("log4j2-test.xml");
-    if (configUrl != null) {
-      System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY, configUrl.toString());
-    }
-  }
+  /** The log4j2 config used within the spawned process */
+  private static final String LOG4J2_CONFIG_URL_STRING = GoldenTestCase.class.getResource("log4j2-test.xml").toString();
+  private static final String[] JVM_ARGS = new String[] {
+      "-D"+ConfigurationFactory.CONFIGURATION_FILE_PROPERTY+"="+LOG4J2_CONFIG_URL_STRING
+    };
   
-  private final static String[] jvmArgs = new String[] {
-    "-D"+ConfigurationFactory.CONFIGURATION_FILE_PROPERTY+"="+System.getProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY)
-  };
+  private final List<ProcessWrapper> processes = new ArrayList<ProcessWrapper>();
   
-  public GoldenTestCase(String name) {
-    super(name);
-  }
-
-  @Override
-  public final void setUp() throws Exception {
-    super.setUp();
+  @Before
+  public final void setUpGoldenTest() throws Exception {
     subSetUp();
   }
   
-  @Override
-  public final void tearDown() throws Exception {
-    super.tearDown();
+  @After
+  public final void tearDownGoldenTest() throws Exception {
     try {
       for (ProcessWrapper process : this.processes) {
         process.destroy();
@@ -86,25 +67,13 @@ public abstract class GoldenTestCase extends TestCase {
   public void subTearDown() throws Exception {
     // override me
   }
-  
-  protected final ProcessWrapper createProcessWrapper(Class<?> main) {
-    final ProcessWrapper processWrapper = new ProcessWrapper.Builder().jvmArgs(jvmArgs).main(main).build();
-    this.processes.add(processWrapper);
-    return processWrapper;
-  }
-  
-  protected final ProcessWrapper createProcessWrapper(Class<?> main, String[] mainArgs) {
-    final ProcessWrapper processWrapper = new ProcessWrapper.Builder().jvmArgs(jvmArgs).main(main).mainArgs(mainArgs).build();
+
+  protected final ProcessWrapper createProcessWrapper(final ProcessWrapper.Builder processWrapperBuilder, final Class<?> main) {
+    final ProcessWrapper processWrapper = processWrapperBuilder.jvmArguments(JVM_ARGS).mainClass(main).build();
     this.processes.add(processWrapper);
     return processWrapper;
   }
   
-  protected final ProcessWrapper createProcessWrapper(Class<?> main, String[] mainArgs, boolean useMainLauncher) {
-    final ProcessWrapper processWrapper = new ProcessWrapper.Builder().jvmArgs(jvmArgs).main(main).mainArgs(mainArgs).useMainLauncher(useMainLauncher).build();
-    this.processes.add(processWrapper);
-    return processWrapper;
-  }
-
   /** 
    * Creates and returns a new GoldenComparator instance. Default implementation
    * is RegexGoldenComparator. Override if you need a different implementation
@@ -125,19 +94,19 @@ public abstract class GoldenTestCase extends TestCase {
     return null;
   }
   
-  protected void assertOutputMatchesGoldenFile(String actualOutput, String goldenFileName) throws IOException {
+  protected void assertOutputMatchesGoldenFile(final String actualOutput, final String goldenFileName) throws IOException {
     GoldenComparator comparator = createGoldenComparator();
     comparator.assertOutputMatchesGoldenFile(actualOutput, goldenFileName);
   }
 
-  protected final void assertOutputMatchesGoldenFile(ProcessWrapper process, String goldenFileName) throws IOException {
+  protected final void assertOutputMatchesGoldenFile(final ProcessWrapper process, final String goldenFileName) throws IOException {
     GoldenComparator comparator = createGoldenComparator();
     comparator.assertOutputMatchesGoldenFile(process.getOutput(), goldenFileName);
   }
 
   protected final Properties createProperties() {
     Properties properties = new Properties();
-    properties.setProperty("gemfire.mcast-port", String.valueOf(this.mcastPort));
+    properties.setProperty("gemfire.mcast-port", "0");
     properties.setProperty("gemfire.log-level", "warning");
     properties.setProperty("file.encoding", "UTF-8");
     return editProperties(properties);
@@ -150,30 +119,23 @@ public abstract class GoldenTestCase extends TestCase {
     return properties;
   }
   
-  protected final int getMcastPort() {
-    return this.mcastPort;
+  protected final void outputLine(final String string) {
+    System.out.println(string);
   }
   
-  // TODO: get rid of this to tighten up tests
-  protected final void sleep(long millis) throws InterruptedException {
-    Thread.sleep(millis);
+  protected final void printProcessOutput(final ProcessWrapper process, final boolean ignoreStopped) {
+    debug(process.getOutput(ignoreStopped), "OUTPUT");
   }
   
-  protected final void printProcessOutput(ProcessWrapper process) {
-    innerPrintOutput(process.getOutput(), "OUTPUT");
+  protected static void debug(final String output, final String title) {
+    debug("------------------ BEGIN " + title + " ------------------");
+    debug(output);
+    debug("------------------- END " + title + " -------------------");
   }
   
-  protected final void printProcessOutput(ProcessWrapper process, boolean ignoreStopped) {
-    innerPrintOutput(process.getOutput(ignoreStopped), "OUTPUT");
-  }
-  
-  protected final void printProcessOutput(ProcessWrapper process, String banner) {
-    innerPrintOutput(process.getOutput(), banner);
-  }
-  
-  protected final void innerPrintOutput(String output, String title) {
-    System.out.println("------------------ BEGIN " + title + " ------------------");
-    System.out.println(output);
-    System.out.println("------------------- END " + title + " -------------------");
+  protected static void debug(final String string) {
+    if (DEBUG) {
+      System.out.println(string);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestFrameworkTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestFrameworkTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestFrameworkTestSuite.java
new file mode 100755
index 0000000..ed540be
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/GoldenTestFrameworkTestSuite.java
@@ -0,0 +1,27 @@
+package com.gemstone.gemfire.test.golden;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+  FailWithErrorInOutputJUnitTest.class,
+  FailWithExtraLineInOutputJUnitTest.class,
+  FailWithLineMissingFromEndOfOutputJUnitTest.class,
+  FailWithLineMissingFromMiddleOfOutputJUnitTest.class,
+  FailWithLoggerErrorInOutputJUnitTest.class,
+  FailWithLoggerFatalInOutputJUnitTest.class,
+  FailWithLoggerWarnInOutputJUnitTest.class,
+  FailWithSevereInOutputJUnitTest.class,
+  FailWithTimeoutOfWaitForOutputToMatchJUnitTest.class,
+  FailWithWarningInOutputJUnitTest.class,
+  PassJUnitTest.class,
+  PassWithExpectedErrorJUnitTest.class,
+  PassWithExpectedSevereJUnitTest.class,
+  PassWithExpectedWarningJUnitTest.class,
+})
+/**
+ * Suite of tests for the test.golden Golden Test framework classes.
+ */
+public class GoldenTestFrameworkTestSuite {
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassJUnitTest.java
index 886fc94..5959291 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassJUnitTest.java
@@ -1,13 +1,16 @@
 package com.gemstone.gemfire.test.golden;
 
+import static org.junit.Assert.*;
+
 import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
+import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.test.process.ProcessWrapper;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Basic unit testing of the golden testing framework. This tests an 
@@ -15,13 +18,9 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
-public class PassJUnitTest extends GoldenTestCase {
+@Category(IntegrationTest.class)
+public class PassJUnitTest extends GoldenTestCase implements ExecutableProcess {
   
-  public PassJUnitTest() {
-    super("PassJUnitTest");
-  }
-
   @Override
   protected GoldenComparator createGoldenComparator() {
     return new GoldenStringComparator(expectedProblemLines());
@@ -31,36 +30,41 @@ public class PassJUnitTest extends GoldenTestCase {
     return getClass().getSimpleName();
   }
 
-  public void testPass() throws InterruptedException, IOException {
-    // output has no problems and should pass
-    final ProcessWrapper process = createProcessWrapper(getClass());
+  /**
+   * Process output has no problems and should pass
+   */
+  @Test
+  public void testPass() throws Exception {
+    final String goldenString = 
+        "Begin " + name() + ".main" + "\n" + 
+        "Press Enter to continue." + "\n" +
+        "End " + name() + ".main" + "\n";
+    
+    final ProcessWrapper process = createProcessWrapper(new ProcessWrapper.Builder(), getClass());
     process.execute(createProperties());
     assertTrue(process.isAlive());
+    
     process.waitForOutputToMatch("Begin " + name() + "\\.main");
     process.waitForOutputToMatch("Press Enter to continue\\.");
     process.sendInput();
     process.waitForOutputToMatch("End " + name() + "\\.main");
     process.waitFor();
-    String goldenString = "Begin " + name() + ".main" + "\n" 
-        + "Press Enter to continue." + "\n" 
-        + "End " + name() + ".main" + "\n";
+    
     assertOutputMatchesGoldenFile(process, goldenString);
-
     assertFalse(process.isAlive());
-    //assertFalse(process.getOutputReader());
     assertFalse(process.getStandardOutReader().isAlive());
     assertFalse(process.getStandardErrorReader().isAlive());
   }
 
-  void execute() throws IOException {
-    System.out.println("Begin " + name() + ".main");
-    System.out.println("Press Enter to continue.");
-    BufferedReader inputReader = new BufferedReader(new InputStreamReader(System.in));
-    inputReader.readLine();
-    System.out.println("End " + name() + ".main");
+  @Override
+  public final void executeInProcess() throws IOException {
+    outputLine("Begin " + name() + ".main");
+    outputLine("Press Enter to continue.");
+    new BufferedReader(new InputStreamReader(System.in)).readLine();
+    outputLine("End " + name() + ".main");
   }
   
-  public static void main(String[] args) throws Exception {
-    new PassJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new PassJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedErrorJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedErrorJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedErrorJUnitTest.java
index 39267e9..35f8173 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedErrorJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedErrorJUnitTest.java
@@ -2,10 +2,9 @@ package com.gemstone.gemfire.test.golden;
 
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogWriterImpl;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that an example test should always pass even if the output contains
@@ -13,25 +12,20 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class PassWithExpectedErrorJUnitTest extends PassWithExpectedProblemTestCase {
   
-  public PassWithExpectedErrorJUnitTest() {
-    super("PassWithExpectedErrorJUnitTest");
-  }
-  
   @Override
   String problem() {
     return "error";
   }
   
   @Override
-  void outputProblem(String message) {
-    LogWriter logWriter = new LocalLogWriter(LogWriterImpl.INFO_LEVEL);
-    logWriter.error(message);
+  void outputProblemInProcess(final String message) {
+    new LocalLogWriter(LogWriterImpl.INFO_LEVEL).error(message);
   }
   
-  public static void main(String[] args) throws Exception {
-    new PassWithExpectedErrorJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new PassWithExpectedErrorJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedProblemTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedProblemTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedProblemTestCase.java
index 2958007..edae866 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedProblemTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedProblemTestCase.java
@@ -4,6 +4,8 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
+import org.junit.Test;
+
 import com.gemstone.gemfire.test.process.ProcessWrapper;
 
 /**
@@ -12,11 +14,9 @@ import com.gemstone.gemfire.test.process.ProcessWrapper;
  * 
  * @author Kirk Lund
  */
-public abstract class PassWithExpectedProblemTestCase extends GoldenTestCase {
+public abstract class PassWithExpectedProblemTestCase extends GoldenTestCase implements ExecutableProcess {
 
-  PassWithExpectedProblemTestCase(String name) {
-    super(name);
-  }
+  private int problemLine; 
   
   @Override
   protected GoldenComparator createGoldenComparator() {
@@ -25,7 +25,11 @@ public abstract class PassWithExpectedProblemTestCase extends GoldenTestCase {
 
   @Override
   protected String[] expectedProblemLines() {
-    return new String[] { ".*" + name() + ".*", "^\\[" + problem() + ".*\\] ExpectedStrings: This is an expected problem in the output" };
+    this.problemLine = 1; 
+    return new String[] { 
+        ".*" + name() + ".*", 
+        "^\\[" + problem() + ".*\\] ExpectedStrings: This is an expected problem in the output" 
+    };
   }
   
   String name() {
@@ -34,36 +38,38 @@ public abstract class PassWithExpectedProblemTestCase extends GoldenTestCase {
   
   abstract String problem();
   
-  abstract void outputProblem(String message);
+  abstract void outputProblemInProcess(String message);
   
-  public void testPassWithExpectedProblem() throws InterruptedException, IOException {
-    // output has an expected warning/error/severe message and should pass
-    final ProcessWrapper process = createProcessWrapper(getClass());
+  /**
+   * Process output has an expected warning/error/severe message and should pass
+   */
+  @Test
+  public void testPassWithExpectedProblem() throws Exception {
+    final String goldenString = 
+        "Begin " + name() + ".main" + "\n" + 
+        "Press Enter to continue." + "\n" +
+        "\n" +
+        expectedProblemLines()[this.problemLine] + "\n" +
+        "End " + name() + ".main" + "\n";
+    debug(goldenString, "GOLDEN");
+
+    final ProcessWrapper process = createProcessWrapper(new ProcessWrapper.Builder(), getClass());
     process.execute(createProperties());
     process.waitForOutputToMatch("Begin " + name() + "\\.main");
     process.waitForOutputToMatch("Press Enter to continue\\.");
     process.sendInput();
     process.waitForOutputToMatch("End " + name() + "\\.main");
     process.waitFor();
-    String goldenString = "Begin " + name() + ".main" + "\n" 
-        + "Press Enter to continue." + "\n" 
-        + "\n"
-        + "^\\[" + problem() + ".*\\] ExpectedStrings: This is an expected problem in the output" + "\n"
-        + "End " + name() + ".main" + "\n";
-    innerPrintOutput(goldenString, "GOLDEN");
-    String[] printMe = expectedProblemLines();
-    for (String str : printMe) {
-      System.out.println(str);
-    }
+    
     assertOutputMatchesGoldenFile(process.getOutput(), goldenString);
   }
   
-  void execute() throws IOException {
-    System.out.println("Begin " + name() + ".main");
-    System.out.println("Press Enter to continue.");
-    BufferedReader inputReader = new BufferedReader(new InputStreamReader(System.in));
-    inputReader.readLine();
-    outputProblem("ExpectedStrings: This is an expected problem in the output");
-    System.out.println("End " + name() + ".main");
+  @Override
+  public final void executeInProcess() throws IOException {
+    outputLine("Begin " + name() + ".main");
+    outputLine("Press Enter to continue.");
+    new BufferedReader(new InputStreamReader(System.in)).readLine();
+    outputProblemInProcess("ExpectedStrings: This is an expected problem in the output");
+    outputLine("End " + name() + ".main");
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedSevereJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedSevereJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedSevereJUnitTest.java
index 659e807..225d55b 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedSevereJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedSevereJUnitTest.java
@@ -2,10 +2,9 @@ package com.gemstone.gemfire.test.golden;
 
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogWriterImpl;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that an example test should always pass even if the output contains
@@ -13,25 +12,20 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class PassWithExpectedSevereJUnitTest extends PassWithExpectedProblemTestCase {
   
-  public PassWithExpectedSevereJUnitTest() {
-    super("PassWithExpectedSevereJUnitTest");
-  }
-  
   @Override
   String problem() {
     return "severe";
   }
   
   @Override
-  void outputProblem(String message) {
-    LogWriter logWriter = new LocalLogWriter(LogWriterImpl.INFO_LEVEL);
-    logWriter.severe(message);
+  void outputProblemInProcess(final String message) {
+    new LocalLogWriter(LogWriterImpl.INFO_LEVEL).severe(message);
   }
   
-  public static void main(String[] args) throws Exception {
-    new PassWithExpectedSevereJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new PassWithExpectedSevereJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedWarningJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedWarningJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedWarningJUnitTest.java
index 866e8b2..215f5ca 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedWarningJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/PassWithExpectedWarningJUnitTest.java
@@ -2,10 +2,9 @@ package com.gemstone.gemfire.test.golden;
 
 import org.junit.experimental.categories.Category;
 
-import com.gemstone.gemfire.LogWriter;
 import com.gemstone.gemfire.internal.logging.LocalLogWriter;
 import com.gemstone.gemfire.internal.logging.LogWriterImpl;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
 /**
  * Verifies that an example test should always pass even if the output contains
@@ -13,25 +12,20 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
  * 
  * @author Kirk Lund
  */
-@Category(UnitTest.class)
+@Category(IntegrationTest.class)
 public class PassWithExpectedWarningJUnitTest extends PassWithExpectedProblemTestCase {
   
-  public PassWithExpectedWarningJUnitTest() {
-    super("PassWithExpectedWarningJUnitTest");
-  }
-
   @Override
   String problem() {
     return "warning";
   }
   
   @Override
-  void outputProblem(String message) {
-    LogWriter logWriter = new LocalLogWriter(LogWriterImpl.INFO_LEVEL);
-    logWriter.warning(message);
+  void outputProblemInProcess(final String message) {
+    new LocalLogWriter(LogWriterImpl.INFO_LEVEL).warning(message);
   }
   
-  public static void main(String[] args) throws Exception {
-    new PassWithExpectedWarningJUnitTest().execute();
+  public static void main(final String[] args) throws Exception {
+    new PassWithExpectedWarningJUnitTest().executeInProcess();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/RegexGoldenComparator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/RegexGoldenComparator.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/RegexGoldenComparator.java
index bc80b3a..759e3a3 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/RegexGoldenComparator.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/RegexGoldenComparator.java
@@ -1,6 +1,5 @@
 package com.gemstone.gemfire.test.golden;
 
-import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 /**
@@ -10,14 +9,13 @@ import java.util.regex.Pattern;
  */
 public class RegexGoldenComparator extends GoldenComparator {
   
-  protected RegexGoldenComparator(String[] expectedProblemLines) {
+  protected RegexGoldenComparator(final String[] expectedProblemLines) {
     super(expectedProblemLines);
   }
   
   @Override
-  protected boolean compareLines(String actualLine, String goldenLine) {
-    logger.debug(GoldenTestCase.GOLDEN_TEST, "RegexGoldenComparator:compareLines comparing \"{}\" to \"{}\"", actualLine, goldenLine);
-    Matcher matcher = Pattern.compile(goldenLine).matcher(actualLine);
-    return matcher.matches();
+  protected boolean compareLines(final String actualLine, final String goldenLine) {
+    debug("RegexGoldenComparator:compareLines comparing \" + actualLine + \" to \" + goldenLine + \"");
+    return Pattern.compile(goldenLine).matcher(actualLine).matches();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/StringGoldenComparator.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/StringGoldenComparator.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/StringGoldenComparator.java
index 5547a9e..48f645a 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/StringGoldenComparator.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/golden/StringGoldenComparator.java
@@ -7,12 +7,12 @@ package com.gemstone.gemfire.test.golden;
  */
 public class StringGoldenComparator extends GoldenComparator {
 
-  protected StringGoldenComparator(String[] expectedProblemLines) {
+  protected StringGoldenComparator(final String[] expectedProblemLines) {
     super(expectedProblemLines);
   }
   
   @Override
-  protected boolean compareLines(String actualLine, String goldenLine) {
+  protected boolean compareLines(final String actualLine, final String goldenLine) {
     if (actualLine == null) {
       return goldenLine == null;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/94939c1f/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessTestFrameworkTestSuite.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessTestFrameworkTestSuite.java b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessTestFrameworkTestSuite.java
new file mode 100755
index 0000000..011a62e
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/test/process/ProcessTestFrameworkTestSuite.java
@@ -0,0 +1,12 @@
+package com.gemstone.gemfire.test.process;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+  MainLauncherJUnitTest.class,
+  ProcessWrapperJUnitTest.class,
+})
+public class ProcessTestFrameworkTestSuite {
+}


[13/50] [abbrv] incubator-geode git commit: GEODE-172: Fixing an intermittent failure as reported by closing cache after each individual test.

Posted by bs...@apache.org.
GEODE-172: Fixing an intermittent failure as reported by closing cache after each individual test.

Also temporarily taking care of an assertion failure that is reflected in GEODE-110 and GEODE-141 that are still under investigation.


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

Branch: refs/heads/feature/GEODE-77
Commit: 64a9bd411058127571be61809e0e28d9ce174e2a
Parents: 377beeb
Author: Vivek Bhaskar <vb...@pivotal.io>
Authored: Thu Jul 30 17:14:57 2015 +0530
Committer: Vivek Bhaskar <vb...@pivotal.io>
Committed: Thu Jul 30 17:14:57 2015 +0530

----------------------------------------------------------------------
 .../gemfire/internal/cache/TXRegionState.java    | 17 ++++++++++++-----
 .../gemfire/disttx/DistTXDebugDUnitTest.java     |  3 +++
 .../disttx/DistributedTransactionDUnitTest.java  | 19 +++++++------------
 3 files changed, 22 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/64a9bd41/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
index bb6ae5f..67cb8c5 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/TXRegionState.java
@@ -596,11 +596,18 @@ public class TXRegionState {
     String regionFullPath = this.getRegion().getFullPath();
     int entryModsSize = this.entryMods.size();
     int entryEventListSize = entryEventList.size();
-    if (entryModsSize != entryEventListSize) {
-      throw new UnsupportedOperationInTransactionException(
-          LocalizedStrings.DISTTX_TX_EXPECTED.toLocalizedString(
-              "entry size of " + entryModsSize + " for region "
-                  + regionFullPath, entryEventListSize));
+    /*
+     * [DISTTX] TODO
+     * This assertion is not working for PutAll and RemoveAll operations 
+     * and thus guarding within Debug flags. May be enabled at later stage.
+     */
+    if (logger.isDebugEnabled()) {
+      if (entryModsSize != entryEventListSize) {
+        throw new UnsupportedOperationInTransactionException(
+            LocalizedStrings.DISTTX_TX_EXPECTED
+                .toLocalizedString("entry size of " + entryModsSize
+                    + " for region " + regionFullPath, entryEventListSize));
+      }
     }
 
     int index = 0;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/64a9bd41/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistTXDebugDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistTXDebugDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistTXDebugDUnitTest.java
index ff0a506..668f1e6 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistTXDebugDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistTXDebugDUnitTest.java
@@ -62,6 +62,9 @@ public class DistTXDebugDUnitTest extends CacheTestCase {
       }
     });
     InternalResourceManager.setResourceObserver(null);
+    if (cache != null) {
+      cache.close();
+    }
   }
 
   public static void createCacheInVm() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/64a9bd41/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
index 2522e63..79d6f64 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
@@ -72,13 +72,13 @@ public class DistributedTransactionDUnitTest extends CacheTestCase {
       }
     });
     
-    this.invokeInEveryVM(new SerializableCallable() {
-      @Override
-      public Object call() throws Exception {
-        System.setProperty("gemfire.log-level", "fine");
-        return null;
-      }
-    }); 
+//    this.invokeInEveryVM(new SerializableCallable() {
+//      @Override
+//      public Object call() throws Exception {
+//        System.setProperty("gemfire.log-level", "fine");
+//        return null;
+//      }
+//    });
 
     this.invokeInEveryVM(new SerializableCallable() {
       @Override
@@ -123,11 +123,6 @@ public class DistributedTransactionDUnitTest extends CacheTestCase {
       execute(vm, c);
     }
   }
-
-  protected String reduceLogging() {
-    return "config";
-  }
-
   
   public int startServer(VM vm) {
     return (Integer) vm.invoke(new SerializableCallable() {


[23/50] [abbrv] incubator-geode git commit: GEODE-155: - Fixes intermittent failure in RegionWithHDFS*DUnitTest about incorrect number of hoplog files getting created. - Caused due to an earlier test not cleaning up static variables in ParallelGatewaySen

Posted by bs...@apache.org.
GEODE-155:
- Fixes intermittent failure in RegionWithHDFS*DUnitTest about incorrect number of hoplog files getting created.
- Caused due to an earlier test not cleaning up static variables in ParallelGatewaySenderQueue.
- These are indeed cleaned up during GemFireCacheImpl.close() if there are any GatewaySenders running in the system.
- But if a region with gateway-senders associated with it, is destroyed first before a cache.close(), then the gateway senders are only stopped,
    and removed from allGatewaySenders list. But the static variables are not cleaned up.
- Later, during GemFireCacheImpl.close(), as the allGatewaySenders list is empty, it skips cleaning the static variables.
- As a fix, invoking a static clean-up method of ParallelGatewaySenderQueue explicitly during cache.close().
- Retained the non-static ParallelGatewaySenderQueue.cleanUp(), if required in future development for performing instance-specific clean-up.
- Minor formatting changes in the unit tests.


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

Branch: refs/heads/feature/GEODE-77
Commit: 8e4c70429f30d39c65330d9397325928780e9c56
Parents: 004a22c
Author: ashetkar <as...@pivotal.io>
Authored: Tue Aug 4 14:06:51 2015 +0530
Committer: ashetkar <as...@pivotal.io>
Committed: Wed Aug 5 11:44:16 2015 +0530

----------------------------------------------------------------------
 .../internal/cache/GemFireCacheImpl.java        |  1 +
 .../parallel/ParallelGatewaySenderQueue.java    | 41 +++++++++++++-------
 .../internal/RegionWithHDFSBasicDUnitTest.java  | 22 +++++------
 .../RegionWithHDFSOffHeapBasicDUnitTest.java    | 10 +++--
 .../hdfs/internal/RegionWithHDFSTestBase.java   |  6 +--
 5 files changed, 49 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8e4c7042/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
index 5487000..f5be144 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/GemFireCacheImpl.java
@@ -2018,6 +2018,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
               advisor.close();
             }
           }
+          ParallelGatewaySenderQueue.cleanUpStatics(null);
         } catch (CancelException ce) {
 
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8e4c7042/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
index f4f9528..9141905 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/wan/parallel/ParallelGatewaySenderQueue.java
@@ -633,15 +633,20 @@ public class ParallelGatewaySenderQueue implements RegionQueue {
    * Wait a while for existing tasks to terminate. If the existing tasks still don't 
    * complete, cancel them by calling shutdownNow. 
    */
-  private void cleanupConflationThreadPool() {
+  private static void cleanupConflationThreadPool(AbstractGatewaySender sender) {
     conflationExecutor.shutdown();// Disable new tasks from being submitted
     
     try {
     if (!conflationExecutor.awaitTermination(1, TimeUnit.SECONDS)) {
       conflationExecutor.shutdownNow(); // Cancel currently executing tasks
       // Wait a while for tasks to respond to being cancelled
-      if (!conflationExecutor.awaitTermination(1, TimeUnit.SECONDS))
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ParallelGatewaySenderQueue_COULD_NOT_TERMINATE_CONFLATION_THREADPOOL, this.sender));
+        if (!conflationExecutor.awaitTermination(1, TimeUnit.SECONDS)) {
+          logger
+              .warn(LocalizedMessage
+                  .create(
+                      LocalizedStrings.ParallelGatewaySenderQueue_COULD_NOT_TERMINATE_CONFLATION_THREADPOOL,
+                      (sender == null ? "all" : sender)));
+        }
     }
     } catch (InterruptedException e) {
       // (Re-)Cancel if current thread also interrupted
@@ -1508,25 +1513,33 @@ public class ParallelGatewaySenderQueue implements RegionQueue {
    * by the queue. Note that this cleanup doesn't clean the data held by the queue.
    */
   public void cleanUp() {
-    if(buckToDispatchLock != null){
-      this.buckToDispatchLock = null;
+    cleanUpStatics(this.sender);
+  }
+
+  /**
+   * @param sender
+   *          can be null.
+   */
+  public static void cleanUpStatics(AbstractGatewaySender sender) {
+    if (buckToDispatchLock != null) {
+      buckToDispatchLock = null;
     }
-    if(regionToDispatchedKeysMapEmpty != null) {
-      this.regionToDispatchedKeysMapEmpty = null;
+    if (regionToDispatchedKeysMapEmpty != null) {
+      regionToDispatchedKeysMapEmpty = null;
     }
-    this.regionToDispatchedKeysMap.clear();
+    regionToDispatchedKeysMap.clear();
     synchronized (ParallelGatewaySenderQueue.class) {
-      if (this.removalThread != null) {
-        this.removalThread.shutdown();
-        this.removalThread = null;
+      if (removalThread != null) {
+        removalThread.shutdown();
+        removalThread = null;
       }
     }
     if (conflationExecutor != null) {
-      cleanupConflationThreadPool();
-      this.conflationExecutor = null;
+      cleanupConflationThreadPool(sender);
+      conflationExecutor = null;
     }
   }
-  
+
   @Override
   public void close() {
     // Because of bug 49060 do not close the regions of a parallel queue

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8e4c7042/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSBasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSBasicDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSBasicDUnitTest.java
index cf10b24..162e529 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSBasicDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSBasicDUnitTest.java
@@ -86,7 +86,7 @@ public class RegionWithHDFSBasicDUnitTest extends RegionWithHDFSTestBase {
       final String uniqueName, final int batchInterval,
       final boolean queuePersistent, final boolean writeonly,
       final long timeForRollover, final long maxFileSize) {
-    SerializableCallable createRegion = new SerializableCallable() {
+    SerializableCallable createRegion = new SerializableCallable("Create HDFS region") {
       public Object call() throws Exception {
         AttributesFactory af = new AttributesFactory();
         af.setDataPolicy(DataPolicy.HDFS_PARTITION);
@@ -95,8 +95,8 @@ public class RegionWithHDFSBasicDUnitTest extends RegionWithHDFSTestBase {
         paf.setRedundantCopies(1);
 
         af.setHDFSStoreName(uniqueName);
-
         af.setPartitionAttributes(paf.create());
+
         HDFSStoreFactory hsf = getCache().createHDFSStoreFactory();
         // Going two level up to avoid home directories getting created in
         // VM-specific directory. This avoids failures in those tests where
@@ -860,7 +860,7 @@ public class RegionWithHDFSBasicDUnitTest extends RegionWithHDFSTestBase {
   protected AsyncInvocation doAsyncPuts(VM vm, final String regionName,
       final int start, final int end, final String suffix, final String value)
       throws Exception {
-    return vm.invokeAsync(new SerializableCallable() {
+    return vm.invokeAsync(new SerializableCallable("doAsyncPuts") {
       public Object call() throws Exception {
         Region r = getRootRegion(regionName);
         String v = "V";
@@ -976,8 +976,8 @@ public class RegionWithHDFSBasicDUnitTest extends RegionWithHDFSTestBase {
   }
 
   /**
-   * create server with file rollover time as 2 secs. Insert few entries and
-   * then sleep for 2 sec. A file should be created. Do it again At th end, two
+   * Create server with file rollover time as 5 seconds. Insert few entries and
+   * then sleep for 7 seconds. A file should be created. Do it again. At the end, two
    * files with inserted entries should be created.
    * 
    * @throws Throwable
@@ -991,8 +991,8 @@ public class RegionWithHDFSBasicDUnitTest extends RegionWithHDFSTestBase {
     String homeDir = "./testWOTimeForRollOverParam";
     final String uniqueName = getName();
 
-    createServerRegion(vm0, 1, 1,  500, homeDir, uniqueName, 5, true, false, 4, 1);
-    createServerRegion(vm1, 1, 1,  500, homeDir, uniqueName, 5, true, false, 4, 1);
+    createServerRegion(vm0, 1, 1, 500, homeDir, uniqueName, 5, true, false, 5, 1);
+    createServerRegion(vm1, 1, 1, 500, homeDir, uniqueName, 5, true, false, 5, 1);
 
     AsyncInvocation a1 = doAsyncPuts(vm0, uniqueName, 1, 8, "vm0");
     AsyncInvocation a2 = doAsyncPuts(vm1, uniqueName, 4, 10, "vm1");
@@ -1000,7 +1000,7 @@ public class RegionWithHDFSBasicDUnitTest extends RegionWithHDFSTestBase {
     a1.join();
     a2.join();
 
-    Thread.sleep(8000);
+    Thread.sleep(7000);
 
     a1 = doAsyncPuts(vm0, uniqueName, 10, 18, "vm0");
     a2 = doAsyncPuts(vm1, uniqueName, 14, 20, "vm1");
@@ -1008,13 +1008,13 @@ public class RegionWithHDFSBasicDUnitTest extends RegionWithHDFSTestBase {
     a1.join();
     a2.join();
 
-    Thread.sleep(8000);
+    Thread.sleep(7000);
 
     cacheClose(vm0, false);
     cacheClose(vm1, false);
 
-    AsyncInvocation async1 = createServerRegionAsync(vm0, 1, 1,  500, homeDir, uniqueName, 5, true, false, 4, 1);
-    AsyncInvocation async2 = createServerRegionAsync(vm1, 1, 1,  500, homeDir, uniqueName, 5, true, false, 4, 1);
+    AsyncInvocation async1 = createServerRegionAsync(vm0, 1, 1, 500, homeDir, uniqueName, 5, true, false, 5, 1);
+    AsyncInvocation async2 = createServerRegionAsync(vm1, 1, 1, 500, homeDir, uniqueName, 5, true, false, 5, 1);
     async1.getResult();
     async2.getResult();
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8e4c7042/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSOffHeapBasicDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSOffHeapBasicDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSOffHeapBasicDUnitTest.java
index 21e2986..ee517d2 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSOffHeapBasicDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSOffHeapBasicDUnitTest.java
@@ -45,9 +45,13 @@ public class RegionWithHDFSOffHeapBasicDUnitTest extends
         }
       }
     };
-    checkOrphans.run();
-    invokeInEveryVM(checkOrphans);
-    super.tearDown2();
+    try {
+      checkOrphans.run();
+      invokeInEveryVM(checkOrphans);
+    } finally {
+      // proceed with tearDown2 anyway.
+      super.tearDown2();
+    }
   }
 
    public void testDelta() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8e4c7042/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSTestBase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSTestBase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSTestBase.java
index 8ad57c9..92687ed 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSTestBase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/hdfs/internal/RegionWithHDFSTestBase.java
@@ -85,8 +85,9 @@ public abstract class RegionWithHDFSTestBase extends CacheTestCase {
   }
 
   public SerializableCallable cleanUpStoresAndDisconnect() throws Exception {
-    SerializableCallable cleanUp = new SerializableCallable() {
+    SerializableCallable cleanUp = new SerializableCallable("cleanUpStoresAndDisconnect") {
       public Object call() throws Exception {
+        disconnectFromDS();
         File file;
         if (homeDir != null) {
           file = new File(homeDir);
@@ -95,7 +96,6 @@ public abstract class RegionWithHDFSTestBase extends CacheTestCase {
         }
         file = new File(tmpDir);
         FileUtil.delete(file);
-        disconnectFromDS();
         return 0;
       }
     };
@@ -576,7 +576,7 @@ public abstract class RegionWithHDFSTestBase extends CacheTestCase {
     return entriesToFileMap;
   }
  protected SerializableCallable validateEmpty(VM vm0, final int numEntries, final String uniqueName) {
-    SerializableCallable validateEmpty = new SerializableCallable("validate") {
+    SerializableCallable validateEmpty = new SerializableCallable("validateEmpty") {
       public Object call() throws Exception {
         Region r = getRootRegion(uniqueName);
         


[42/50] [abbrv] incubator-geode git commit: GEODE-105: Null value in Map causes NPE with Map Indexes

Posted by bs...@apache.org.
GEODE-105: Null value in Map causes NPE with Map Indexes

Convert null to NULL tokens when saving to the map indexes
We now ignore non map objects instead of throwing assertion errors.
We do not try to index them for map indexes.


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

Branch: refs/heads/feature/GEODE-77
Commit: fcd03406c13d8f22b0222b337c6309ab94fce69c
Parents: 18bbd9b
Author: Jason Huynh <jh...@pivotal.io>
Authored: Tue Aug 11 14:07:01 2015 -0700
Committer: Jason Huynh <jh...@pivotal.io>
Committed: Tue Aug 11 14:07:01 2015 -0700

----------------------------------------------------------------------
 .../query/internal/index/AbstractMapIndex.java  |  6 +--
 .../internal/index/CompactMapRangeIndex.java    |  7 ++-
 .../MapRangeIndexMaintenanceJUnitTest.java      | 50 ++++++++++++++++++++
 3 files changed, 57 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fcd03406/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractMapIndex.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractMapIndex.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractMapIndex.java
index 599e648..198b6ae 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractMapIndex.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/AbstractMapIndex.java
@@ -319,10 +319,9 @@ public abstract class AbstractMapIndex extends AbstractIndex
   void addMapping(Object key, Object value, RegionEntry entry)
       throws IMQException
   {
-    if(key == QueryService.UNDEFINED) {
+    if(key == QueryService.UNDEFINED || !(key instanceof Map)) {
       return;
     }
-    assert key instanceof Map;
     if (this.isAllKeys) {
       Iterator<Map.Entry<?, ?>> entries = ((Map)key).entrySet().iterator();
       while (entries.hasNext()) {
@@ -346,10 +345,9 @@ public abstract class AbstractMapIndex extends AbstractIndex
   void saveMapping(Object key, Object value, RegionEntry entry)
       throws IMQException
   {
-    if(key == QueryService.UNDEFINED) {
+    if(key == QueryService.UNDEFINED || !(key instanceof Map)) {
       return;
     }
-    assert key instanceof Map;
     if (this.isAllKeys) {
       Iterator<Map.Entry<?, ?>> entries = ((Map)key).entrySet().iterator();
       while (entries.hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fcd03406/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/CompactMapRangeIndex.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/CompactMapRangeIndex.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/CompactMapRangeIndex.java
index 299ca4f..f8c5745 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/CompactMapRangeIndex.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/query/internal/index/CompactMapRangeIndex.java
@@ -90,10 +90,9 @@ public class CompactMapRangeIndex extends AbstractMapIndex
   void saveMapping(Object key, Object value, RegionEntry entry)
       throws IMQException
   {
-    if(key == QueryService.UNDEFINED) {
+    if(key == QueryService.UNDEFINED || !(key instanceof Map)) {
       return;
     }
-    assert key instanceof Map;
     if (this.isAllKeys) {
       Iterator<Map.Entry<?, ?>> entries = ((Map)key).entrySet().iterator();
       while (entries.hasNext()) {
@@ -107,6 +106,7 @@ public class CompactMapRangeIndex extends AbstractMapIndex
       for (Object mapKey : mapKeys) {
         Object indexKey = ((Map)key).get(mapKey);
         if (indexKey != null) {
+          //Do not convert to IndexManager.NULL.  We are only interested in specific keys
           this.saveIndexAddition(mapKey, indexKey, value, entry);
         }
       }
@@ -156,6 +156,9 @@ public class CompactMapRangeIndex extends AbstractMapIndex
   protected void saveIndexAddition(Object mapKey, Object indexKey, Object value,
       RegionEntry entry) throws IMQException
   {
+    if (indexKey == null) {
+      indexKey = IndexManager.NULL;
+    }
     boolean isPr = this.region instanceof BucketRegion;
     // Get RangeIndex for it or create it if absent
     CompactRangeIndex rg = (CompactRangeIndex) this.mapKeyToValueIndex.get(mapKey);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fcd03406/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
index c339ec9..0bb92d3 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/MapRangeIndexMaintenanceJUnitTest.java
@@ -23,11 +23,13 @@ import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.AttributesFactory;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.Scope;
 import com.gemstone.gemfire.cache.query.CacheUtils;
 import com.gemstone.gemfire.cache.query.Index;
 import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
 import com.gemstone.gemfire.cache.query.QueryService;
+import com.gemstone.gemfire.cache.query.SelectResults;
 import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
@@ -301,7 +303,55 @@ public class MapRangeIndexMaintenanceJUnitTest{
     // recreate index to verify they get updated correctly
     keyIndex1 = (IndexProtocol) qs.createIndex(INDEX_NAME, "positions['SUN', 'IBM']", "/portfolio ");
     assertTrue("Index should be a CompactMapRangeIndex ", keyIndex1 instanceof CompactMapRangeIndex);
+  }
+
+  @Test
+  public void testNullMapValuesInIndexOnLocalRegionForCompactMap() throws Exception{
+    region = CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+    keyIndex1 = (IndexProtocol) qs.createIndex(INDEX_NAME, "positions[*]", "/portfolio ");
+
+    Portfolio p = new Portfolio(1, 1);
+    p.positions = new HashMap();
+    region.put(1, p);
+
+    Portfolio p2 = new Portfolio(2, 2);
+    p2.positions = null;
+    region.put(2, p2);
+
+    Portfolio p3 = new Portfolio(3, 3);
+    p3.positions = new HashMap();
+    p3.positions.put("IBM", "something");
+    p3.positions.put("SUN", null);
+    region.put(3, p3);
+    region.put(3, p3);
+    
+    SelectResults result = (SelectResults) qs.newQuery("select * from /portfolio p where p.positions['SUN'] = null").execute();
+    assertEquals(1, result.size());
+  }
+
+  @Test
+  public void testNullMapValuesInIndexOnLocalRegionForMap() throws Exception{
+    IndexManager.TEST_RANGEINDEX_ONLY = true;
+    region = CacheUtils.getCache().createRegionFactory(RegionShortcut.REPLICATE).create("portfolio");
+    qs = CacheUtils.getQueryService();
+    keyIndex1 = (IndexProtocol) qs.createIndex(INDEX_NAME, "positions[*]", "/portfolio ");
+
+    Portfolio p = new Portfolio(1, 1);
+    p.positions = new HashMap();
+    region.put(1, p);
+
+    Portfolio p2 = new Portfolio(2, 2);
+    p2.positions = null;
+    region.put(2, p2);
+
+    Portfolio p3 = new Portfolio(3, 3);
+    p3.positions = new HashMap();
+    p3.positions.put("SUN", null);
+    region.put(3, p3);
     
+    SelectResults result = (SelectResults) qs.newQuery("select * from /portfolio p where p.positions['SUN'] = null").execute();
+    assertEquals(1, result.size());
   }
 
   /**


[49/50] [abbrv] incubator-geode git commit: GEODE-214: improve Azul support

Posted by bs...@apache.org.
GEODE-214: improve Azul support

- isTenured now looks for "GenPauseless Old Gen" for azul.
- Azul jvm version no longer logged as unsupported.
- Fatal log message about jvm version not being supported
  is now a warning since we continue to run.
- Fixed a bug in ReflectionSingleObjectSizer in how it calculated
  the size of a field. It was calling Field.getClass instead of Field.getType.
  Field.getClass always returns an instance of Field.class which the sizer
  always says is an objref size. getType will return a primitive class
  for primitive fields which is what we want.
- Improved ObjectSizerJUnitTest.
- The object header size and reference size are now correctly computed
  for azul.
- Added some java 8 support to the ObjectSizer.
- Fix tests that are unintentionally spawning processes that use the
  default mcast-port. This fixes intermittent failures caused by finding
  another member of the wrong GemFire version.


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

Branch: refs/heads/feature/GEODE-77
Commit: eb7e7b778d8b64741d1e377022c620feaa6483ca
Parents: 2914567
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Tue Jun 2 10:45:30 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Fri Aug 14 11:23:27 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/SharedLibrary.java         | 20 ++++++-
 .../cache/MinimumSystemRequirements.java        |  4 +-
 .../cache/control/HeapMemoryMonitor.java        |  1 +
 .../gemfire/internal/lang/SystemUtils.java      | 35 ++++++++++--
 .../size/ReflectionSingleObjectSizer.java       |  4 +-
 .../LocatorLauncherRemoteFileJUnitTest.java     |  8 +--
 .../LocatorLauncherRemoteJUnitTest.java         | 57 ++++++++------------
 .../ServerLauncherLocalJUnitTest.java           | 47 ++++++++++------
 .../ServerLauncherRemoteFileJUnitTest.java      |  2 -
 .../ServerLauncherWithSpringJUnitTest.java      |  2 +
 .../internal/size/ObjectSizerJUnitTest.java     | 14 ++++-
 11 files changed, 126 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/main/java/com/gemstone/gemfire/internal/SharedLibrary.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/SharedLibrary.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/SharedLibrary.java
index 59ab34e..fd923ad 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/SharedLibrary.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/SharedLibrary.java
@@ -8,6 +8,7 @@
 package com.gemstone.gemfire.internal;
 
 import com.gemstone.gemfire.InternalGemFireError;
+import com.gemstone.gemfire.internal.lang.SystemUtils;
 import com.gemstone.gemfire.pdx.internal.unsafe.UnsafeWrapper;
 
 import java.io.File;
@@ -68,9 +69,19 @@ public class SharedLibrary {
       int scaleIndex = 0;
       int tmpReferenceSize = 0;
       int tmpObjectHeaderSize = 0;
+      if (SystemUtils.isAzulJVM()) {
+        tmpObjectHeaderSize = 8;
+        tmpReferenceSize = 8;
+      } else {
       if (unsafe != null) {
         // Use unsafe to figure out the size of an object reference since we might
         // be using compressed oops.
+        // Note: as of java 8 compressed oops do not imply a compressed object header.
+        // The object header is determined by UseCompressedClassPointers.
+        // UseCompressedClassPointers requires UseCompressedOops
+        // but UseCompressedOops does not require UseCompressedClassPointers.
+        // But it seems unlikely that someone would compress their oops
+        // not their class pointers. 
         scaleIndex = unsafe.arrayScaleIndex(Object[].class);
         if (scaleIndex == 4) {
           // compressed oops
@@ -85,8 +96,12 @@ public class SharedLibrary {
         }
       }
       if (scaleIndex == 0) {
-        // If our heap is > 32G then assume large oops. Otherwise assume compressed oops.
-        if (Runtime.getRuntime().maxMemory() > (32L*1024*1024*1024)) {
+        // If our heap is > 32G (64G on java 8) then assume large oops. Otherwise assume compressed oops.
+        long SMALL_OOP_BOUNDARY = 32L;
+        if (SystemUtils.isJavaVersionAtLeast("1.8")) {
+          SMALL_OOP_BOUNDARY = 64L;
+        }
+        if (Runtime.getRuntime().maxMemory() > (SMALL_OOP_BOUNDARY*1024*1024*1024)) {
           tmpReferenceSize = 8;
           tmpObjectHeaderSize = 16;
         } else {
@@ -94,6 +109,7 @@ public class SharedLibrary {
           tmpObjectHeaderSize = 12;
         }
       }
+      }
       referenceSize = tmpReferenceSize;
       objectHeaderSize = tmpObjectHeaderSize;
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/MinimumSystemRequirements.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/MinimumSystemRequirements.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/MinimumSystemRequirements.java
index d0a346f..12f6177 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/MinimumSystemRequirements.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/MinimumSystemRequirements.java
@@ -58,7 +58,7 @@ public final class MinimumSystemRequirements {
     minimumSystemRequirementsMet &= checkJavaVersion();
 
     if (!minimumSystemRequirementsMet) {
-      logger.fatal(LocalizedMessage.create(LocalizedStrings.MinimumSystemRequirements_NOT_MET));
+      logger.warn(LocalizedMessage.create(LocalizedStrings.MinimumSystemRequirements_NOT_MET));
     }
 
     return minimumSystemRequirementsMet;
@@ -76,7 +76,7 @@ public final class MinimumSystemRequirements {
       return true;
     }
 
-    logger.fatal(LocalizedMessage.create(LocalizedStrings.MinimumSystemRequirements_JAVA_VERSION, JAVA_VERSION));
+    logger.warn(LocalizedMessage.create(LocalizedStrings.MinimumSystemRequirements_JAVA_VERSION, JAVA_VERSION));
     return false;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/HeapMemoryMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/HeapMemoryMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/HeapMemoryMonitor.java
index fda337b..d84a367 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/HeapMemoryMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/HeapMemoryMonitor.java
@@ -181,6 +181,7 @@ public class HeapMemoryMonitor implements NotificationListener, ResourceMonitor
         || name.equals("Old Space")       // BEA JRockit 1.5, 1.6 GC
         || name.equals("Tenured Gen")     // Hitachi 1.5 GC
         || name.equals("Java heap")       // IBM 1.5, 1.6 GC
+        || name.equals("GenPauseless Old Gen") // azul C4/GPGC collector
         
         // Allow an unknown pool name to monitor
         || (HEAP_POOL != null && name.equals(HEAP_POOL));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/main/java/com/gemstone/gemfire/internal/lang/SystemUtils.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/lang/SystemUtils.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/lang/SystemUtils.java
index f4f111a..9912398 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/lang/SystemUtils.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/lang/SystemUtils.java
@@ -36,6 +36,7 @@ public class SystemUtils {
   public static final String APPLE_JVM_VENDOR_NAME = "Apple";
   public static final String IBM_JVM_NAME = "IBM";
   public static final String ORACLE_JVM_VENDOR_NAME = "Oracle";
+  public static final String AZUL_JVM_VENDOR_NAME = "Azul";
 
   // Operating System Names
   public static final String LINUX_OS_NAME = "Linux";
@@ -45,13 +46,30 @@ public class SystemUtils {
   /**
    * Utility method to determine whether the installed Java Runtime Environment (JRE) is minimally at the specified,
    * expected version.  Typically, Java versions are of the form "1.6.0_31"...
+   * In the Azul JVM java.version does not have the "_NN" suffix. Instead it has the azul product version
+   * as the suffix like so "-zing_NN.NN.N.N". So on azul we instead use the "java.specification.version" sys prop
+   * and only compare the major and minor version numbers. All the stuff after the second "." in expectedVersion
+   * is ignored.
    * 
-   * @param expectedVersion an int value specifying the minimum expected version of the Java Runtime.
+   * @param expectedVersion an string value specifying the minimum expected version of the Java Runtime.
    * @return a boolean value indicating if the Java Runtime meets the expected version requirement.
    * @see java.lang.System#getProperty(String) with "java.version".
    */
-  public static boolean isJavaVersionAtLeast(final String expectedVersion) {
-    String actualVersionDigits = StringUtils.getDigitsOnly(System.getProperty("java.version"));
+  public static boolean isJavaVersionAtLeast(String expectedVersion) {
+    String actualVersionDigits;
+    if (isAzulJVM()) {
+      actualVersionDigits = StringUtils.getDigitsOnly(System.getProperty("java.specification.version"));
+      int dotIdx = expectedVersion.indexOf('.');
+      if (dotIdx != -1) {
+        dotIdx = expectedVersion.indexOf('.', dotIdx+1);
+        if (dotIdx != -1) {
+          // strip off everything after the second dot.
+          expectedVersion = expectedVersion.substring(0, dotIdx);
+        }
+      }
+    } else {
+      actualVersionDigits = StringUtils.getDigitsOnly(System.getProperty("java.version"));
+    }
 
     String expectedVersionDigits = StringUtils.padEnding(StringUtils.getDigitsOnly(expectedVersion), '0',
       actualVersionDigits.length());
@@ -86,6 +104,17 @@ public class SystemUtils {
     return isJvmVendor(ORACLE_JVM_VENDOR_NAME);
   }
 
+  /**
+   * Utility method to determine whether the Java application process is executing on the Azul JVM.
+   *
+   * @return a boolean value indicating whether the Java application process is executing and running 
+   * on the Azul JVM.
+   * @see #isJvmVendor(String)
+   */
+  public static boolean isAzulJVM() {
+    return isJvmVendor(AZUL_JVM_VENDOR_NAME);
+  }
+  
   // @see java.lang.System#getProperty(String) with 'java.vm.vendor'.
   private static boolean isJvmVendor(final String expectedJvmVendorName) {
     String jvmVendor = System.getProperty("java.vm.vendor");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/main/java/com/gemstone/gemfire/internal/size/ReflectionSingleObjectSizer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/size/ReflectionSingleObjectSizer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/size/ReflectionSingleObjectSizer.java
index 47b73a8..d3f3b24 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/size/ReflectionSingleObjectSizer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/size/ReflectionSingleObjectSizer.java
@@ -109,7 +109,7 @@ public class ReflectionSingleObjectSizer implements SingleObjectSizer {
       } while (clazz != null);
 
       if (lastField != null) {
-        size = lastFieldOffset + sizeType(lastField.getClass());
+        size = lastFieldOffset + sizeType(lastField.getType());
       } else {
         // class with no fields
         size = OBJECT_SIZE;
@@ -123,7 +123,7 @@ public class ReflectionSingleObjectSizer implements SingleObjectSizer {
         Field[] fields = clazz.getDeclaredFields();
         for(Field field: fields) {
           if(!Modifier.isStatic(field.getModifiers())) {
-            size += sizeType(field.getClass());
+            size += sizeType(field.getType());
           }
         }
         clazz = clazz.getSuperclass();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileJUnitTest.java
index b481461..0be4a8e 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteFileJUnitTest.java
@@ -56,9 +56,7 @@ public class LocatorLauncherRemoteFileJUnitTest extends LocatorLauncherRemoteJUn
    * Override because FileProcessController cannot request status with PID
    */
   public void testStatusUsingPid() throws Throwable {
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -135,9 +133,7 @@ public class LocatorLauncherRemoteFileJUnitTest extends LocatorLauncherRemoteJUn
    * Override because FileProcessController cannot request stop with PID
    */
   public void testStopUsingPid() throws Throwable {
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
index 25aa23c..8977e47 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/LocatorLauncherRemoteJUnitTest.java
@@ -86,8 +86,13 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
     assertTrue(this.temporaryFolder.getRoot().isDirectory() && this.temporaryFolder.getRoot().canWrite());
 
     // launch LocatorLauncherForkingProcess which then launches the GemFire Locator
-    List<String> command = new ArrayList<String>();
+    final List<String> jvmArguments = getJvmArguments();
+    
+    final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
+    for (String jvmArgument : jvmArguments) {
+      command.add(jvmArgument);
+    }
     command.add("-cp");
     command.add(System.getProperty("java.class.path"));
     command.add(LocatorLauncherRemoteDUnitTest.class.getName().concat("$").concat(LocatorLauncherForkingProcess.class.getSimpleName()));
@@ -129,9 +134,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
   @Test
   public void testStartCreatesPidFile() throws Throwable {
     // build and start the locator
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -200,9 +203,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
     assertTrue(this.statusFile.exists());
     
     // build and start the locator
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -264,9 +265,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
     writePid(this.pidFile, Integer.MAX_VALUE);
 
     // build and start the locator
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -323,9 +322,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
     writePid(this.pidFile, otherPid);
 
     // build and start the locator
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -380,9 +377,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
   public void testStartUsingPortInUseFails() throws Throwable {
     this.socket = SocketCreator.getDefaultInstance().createServerSocket(this.locatorPort, 50, null, -1);
     
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -456,10 +451,8 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
     assertFalse(this.socket.isClosed());
     
     // launch locator
-    final List<String> jvmArguments = new ArrayList<String>();
+    final List<String> jvmArguments = getJvmArguments();
     jvmArguments.add("-D" + DistributionLocator.TEST_OVERRIDE_DEFAULT_PORT_PROPERTY + "=" + this.locatorPort);
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -533,9 +526,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
     writePid(this.pidFile, realPid);
     
     // build and start the locator
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -606,9 +597,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
 
   @Test
   public void testStatusUsingPid() throws Throwable {
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -690,9 +679,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
   
   @Test
   public void testStatusUsingWorkingDirectory() throws Throwable {
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -817,9 +804,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
   
   @Test
   public void testStopUsingPid() throws Throwable {
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -893,9 +878,7 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
   
   @Test
   public void testStopUsingWorkingDirectory() throws Throwable {
-    final List<String> jvmArguments = new ArrayList<String>();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
-    jvmArguments.add("-Dgemfire.log-level=config");
+    final List<String> jvmArguments = getJvmArguments();
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -948,6 +931,12 @@ public class LocatorLauncherRemoteJUnitTest extends AbstractLocatorLauncherJUnit
     }
   }
 
+  public static List<String> getJvmArguments() {
+    final List<String> jvmArguments = new ArrayList<String>();
+    jvmArguments.add("-Dgemfire.log-level=config");
+    return jvmArguments;
+  }
+  
   /**
    * Used only by {@link LocatorLauncherRemoteJUnitTest#testRunningLocatorOutlivesForkingProcess}
    */

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
index 1addfc8..44849f8 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherLocalJUnitTest.java
@@ -121,7 +121,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setDisableDefaultServer(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     this.launcher = builder.build();
     assertNotNull(this.launcher);
@@ -173,7 +174,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setDisableDefaultServer(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     assertFalse(builder.getForce());
     this.launcher = builder.build();
@@ -228,7 +230,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setDisableDefaultServer(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     assertFalse(builder.getForce());
     this.launcher = builder.build();
@@ -273,7 +276,9 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setDisableDefaultServer(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
+    
     this.launcher = builder.build();
 
     // wait for server to start
@@ -328,7 +333,9 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setDisableDefaultServer(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
+
     this.launcher = builder.build();
 
     // wait for server to start
@@ -387,7 +394,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setForce(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     assertTrue(builder.getForce());
     this.launcher = builder.build();
@@ -466,7 +474,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
         .setServerPort(freeTCPPorts[1])
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     this.launcher = builder.build();
     this.launcher.start();
@@ -535,7 +544,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
         .setServerPort(this.serverPort)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     this.launcher = builder.build();
     this.launcher.start();
@@ -585,7 +595,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
     final Builder builder = new Builder()
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     this.launcher = builder.build();
     
@@ -663,7 +674,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setDisableDefaultServer(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     assertFalse(builder.getForce());
     this.launcher = builder.build();
@@ -741,7 +753,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
         .setServerPort(freeTCPPort)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     this.launcher = builder.build();
     
@@ -801,7 +814,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setDisableDefaultServer(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
     
     assertFalse(builder.getForce());
     this.launcher = builder.build();
@@ -865,7 +879,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setDisableDefaultServer(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
     
     assertFalse(builder.getForce());
     this.launcher = builder.build();
@@ -930,7 +945,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setDisableDefaultServer(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     assertFalse(builder.getForce());
     this.launcher = builder.build();
@@ -983,7 +999,8 @@ public class ServerLauncherLocalJUnitTest extends AbstractServerLauncherJUnitTes
         .setDisableDefaultServer(true)
         .setMemberName(getUniqueName())
         .setRedirectOutput(true)
-        .set(DistributionConfig.LOG_LEVEL_NAME, "config");
+        .set(DistributionConfig.LOG_LEVEL_NAME, "config")
+        .set(DistributionConfig.MCAST_PORT_NAME, "0");
 
     assertFalse(builder.getForce());
     this.launcher = builder.build();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteFileJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteFileJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteFileJUnitTest.java
index 3b321ed..11cab25 100755
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteFileJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherRemoteFileJUnitTest.java
@@ -57,7 +57,6 @@ public class ServerLauncherRemoteFileJUnitTest extends ServerLauncherRemoteJUnit
    */
   public void testStatusUsingPid() throws Throwable {
     final List<String> jvmArguments = getJvmArguments();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());
@@ -135,7 +134,6 @@ public class ServerLauncherRemoteFileJUnitTest extends ServerLauncherRemoteJUnit
    */
   public void testStopUsingPid() throws Throwable {
     final List<String> jvmArguments = getJvmArguments();
-    jvmArguments.add("-D" + getUniqueName() + "=true");
     
     final List<String> command = new ArrayList<String>();
     command.add(new File(new File(System.getProperty("java.home"), "bin"), "java").getCanonicalPath());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithSpringJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithSpringJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithSpringJUnitTest.java
index dbefdbd..4472ab6 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithSpringJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/distributed/ServerLauncherWithSpringJUnitTest.java
@@ -15,6 +15,7 @@ import org.springframework.data.gemfire.support.SpringContextBootstrappingInitia
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.distributed.AbstractLauncher.Status;
 import com.gemstone.gemfire.distributed.ServerLauncher.Builder;
+import com.gemstone.gemfire.distributed.internal.DistributionConfig;
 import com.gemstone.gemfire.internal.process.ProcessType;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 
@@ -47,6 +48,7 @@ public class ServerLauncherWithSpringJUnitTest extends AbstractServerLauncherJUn
       .setForce(true)
       .setMemberName(getUniqueName())
       .setSpringXmlLocation("spring/spring-gemfire-context.xml")
+      .set(DistributionConfig.MCAST_PORT_NAME, "0")
       .build();
 
     assertNotNull(this.launcher);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/eb7e7b77/gemfire-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectSizerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectSizerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectSizerJUnitTest.java
index 3d6a731..7c63d90 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectSizerJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/size/ObjectSizerJUnitTest.java
@@ -27,8 +27,7 @@ public class ObjectSizerJUnitTest extends TestCase {
   }
 
   public void test() throws IllegalArgumentException, IllegalAccessException {
-    long size = ObjectGraphSizer.size(new Object());
-    assertEquals(OBJECT_SIZE, 8, size);
+    assertEquals(roundup(OBJECT_SIZE), ObjectGraphSizer.size(new Object()));
     
     assertEquals(roundup(OBJECT_SIZE + 4), ObjectGraphSizer.size(new TestObject1()));
     assertEquals(roundup(OBJECT_SIZE + 4), ObjectGraphSizer.size(new TestObject2()));
@@ -37,6 +36,7 @@ public class ObjectSizerJUnitTest extends TestCase {
     assertEquals(roundup(OBJECT_SIZE + REFERENCE_SIZE), ObjectGraphSizer.size(new TestObject4()));
     assertEquals(roundup(OBJECT_SIZE + REFERENCE_SIZE) + roundup(OBJECT_SIZE + 4), ObjectGraphSizer.size(new TestObject5()));
     assertEquals(roundup(OBJECT_SIZE + REFERENCE_SIZE) + roundup(OBJECT_SIZE + REFERENCE_SIZE * 4 + 4) + roundup(OBJECT_SIZE + 4), ObjectGraphSizer.size(new TestObject6()));
+    assertEquals(roundup(OBJECT_SIZE + 7), ObjectGraphSizer.size(new TestObject7()));
   }
   
   private static class TestObject1 {
@@ -67,4 +67,14 @@ public class ObjectSizerJUnitTest extends TestCase {
       array[2] = array[3];
     }
   }
+  
+  private static class TestObject7 {
+    byte b1;
+    byte b2;
+    byte b3;
+    byte b4;
+    byte b5;
+    byte b6;
+    byte b7;
+  }
 }
\ No newline at end of file


[32/50] [abbrv] incubator-geode git commit: Fix for GEODE-109

Posted by bs...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByScoreExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByScoreExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByScoreExecutor.java
index ca6cd82..19356b7 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByScoreExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZRemRangeByScoreExecutor.java
@@ -74,7 +74,7 @@ public class ZRemRangeByScoreExecutor extends SortedSetExecutor {
     try {
       if (start == Double.NEGATIVE_INFINITY && stop == Double.POSITIVE_INFINITY && startInclusive && stopInclusive) {
         numRemoved = keyRegion.size();
-        context.getRegionCache().removeKey(key);
+        context.getRegionProvider().removeKey(key);
       } else {
         removeList = getKeys(context, key, keyRegion, start, stop, startInclusive, stopInclusive);
       }
@@ -93,7 +93,7 @@ public class ZRemRangeByScoreExecutor extends SortedSetExecutor {
         if (oldVal != null)
           numRemoved++;
         if (keyRegion.isEmpty())
-          context.getRegionCache().removeKey(key);
+          context.getRegionProvider().removeKey(key);
       }
     }
     command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), numRemoved));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZScanExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZScanExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZScanExecutor.java
index 097e9be..faa8ffd 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZScanExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/sortedset/ZScanExecutor.java
@@ -2,6 +2,7 @@ package com.gemstone.gemfire.internal.redis.executor.sortedset;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.regex.Pattern;
@@ -31,7 +32,7 @@ public class ZScanExecutor extends AbstractScanExecutor {
     }
 
     ByteArrayWrapper key = command.getKey();
-    Region<ByteArrayWrapper, DoubleWrapper> keyRegion = (Region<ByteArrayWrapper, DoubleWrapper>) context.getRegionCache().getRegion(key);
+    Region<ByteArrayWrapper, DoubleWrapper> keyRegion = (Region<ByteArrayWrapper, DoubleWrapper>) context.getRegionProvider().getRegion(key);
     checkDataType(key, RedisDataType.REDIS_SORTEDSET, context);
     if (keyRegion == null) {
       command.setResponse(Coder.getScanResponse(context.getByteBufAllocator(), new ArrayList<String>()));
@@ -96,7 +97,7 @@ public class ZScanExecutor extends AbstractScanExecutor {
       return;
     }
 
-    List<ByteArrayWrapper> returnList = (List<ByteArrayWrapper>) getIteration(keyRegion.entrySet(), matchPattern, count, cursor);
+    List<ByteArrayWrapper> returnList = (List<ByteArrayWrapper>) getIteration(new HashSet(keyRegion.entrySet()), matchPattern, count, cursor);
 
     command.setResponse(Coder.getScanResponse(context.getByteBufAllocator(), returnList));
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/AppendExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/AppendExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/AppendExecutor.java
index fa83882..fda38c8 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/AppendExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/AppendExecutor.java
@@ -17,7 +17,7 @@ public class AppendExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();;
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();;
 
     if (commandElems.size() < 3) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.APPEND));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitCountExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitCountExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitCountExecutor.java
index a4fbf95..4ac408d 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitCountExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitCountExecutor.java
@@ -18,7 +18,7 @@ public class BitCountExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() != 2 && commandElems.size() != 4) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.BITCOUNT));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitOpExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitOpExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitOpExecutor.java
index 7dd10e9..c678b79 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitOpExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitOpExecutor.java
@@ -17,7 +17,7 @@ public class BitOpExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 4) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.BITOP));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitPosExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitPosExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitPosExecutor.java
index 7a2a492..796c858 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitPosExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/BitPosExecutor.java
@@ -19,7 +19,7 @@ public class BitPosExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 3) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.BITPOS));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/DecrByExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/DecrByExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/DecrByExecutor.java
index 4bcbd8d..9dfdfed 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/DecrByExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/DecrByExecutor.java
@@ -23,7 +23,7 @@ public class DecrByExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 3) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.DECRBY));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/DecrExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/DecrExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/DecrExecutor.java
index 55b7fc5..b66160d 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/DecrExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/DecrExecutor.java
@@ -9,7 +9,7 @@ import com.gemstone.gemfire.internal.redis.Command;
 import com.gemstone.gemfire.internal.redis.ExecutionHandlerContext;
 import com.gemstone.gemfire.internal.redis.RedisConstants.ArityDef;
 import com.gemstone.gemfire.internal.redis.RedisDataType;
-import com.gemstone.gemfire.internal.redis.RegionCache;
+import com.gemstone.gemfire.internal.redis.RegionProvider;
 
 public class DecrExecutor extends StringExecutor {
 
@@ -25,7 +25,7 @@ public class DecrExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    RegionCache rC = context.getRegionCache();
+    RegionProvider rC = context.getRegionProvider();
     Region<ByteArrayWrapper, ByteArrayWrapper> r = rC.getStringsRegion();;
 
     if (commandElems.size() < 2) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetBitExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetBitExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetBitExecutor.java
index 1035d10..928700a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetBitExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetBitExecutor.java
@@ -17,7 +17,7 @@ public class GetBitExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 3) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.GETBIT));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetExecutor.java
index aed0f2a..86d5c75 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetExecutor.java
@@ -12,7 +12,7 @@ public class GetExecutor extends StringExecutor {
 
   @Override
   public void executeCommand(Command command, ExecutionHandlerContext context) {
-    Region<ByteArrayWrapper, ByteArrayWrapper> r= context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r= context.getRegionProvider().getStringsRegion();
 
     if (command.getProcessedCommand().size() < 2) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.GETEXECUTOR));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetRangeExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetRangeExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetRangeExecutor.java
index 20d938d..4d96c5f 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetRangeExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetRangeExecutor.java
@@ -23,7 +23,7 @@ public class GetRangeExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 4) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.GETRANGE));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetSetExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetSetExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetSetExecutor.java
index f599c96..132fdc5 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetSetExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/GetSetExecutor.java
@@ -17,7 +17,7 @@ public class GetSetExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 3) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.GETSET));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrByExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrByExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrByExecutor.java
index 8e9ca4d..02c5b1e 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrByExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrByExecutor.java
@@ -23,7 +23,7 @@ public class IncrByExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 3) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.INCRBY));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrByFloatExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrByFloatExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrByFloatExecutor.java
index 2c3c5f8..c08f709 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrByFloatExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrByFloatExecutor.java
@@ -24,7 +24,7 @@ public class IncrByFloatExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
     if (commandElems.size() < 3) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.INCRBYFLOAT));
       return;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrExecutor.java
index aafe7dd..44feb03 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/IncrExecutor.java
@@ -21,7 +21,7 @@ public class IncrExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 2) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.INCR));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MGetExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MGetExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MGetExecutor.java
index bd002ce..b063288 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MGetExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MGetExecutor.java
@@ -18,7 +18,7 @@ public class MGetExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 2) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.MGET));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MSetExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MSetExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MSetExecutor.java
index d95dc34..531895e 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MSetExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MSetExecutor.java
@@ -20,7 +20,7 @@ public class MSetExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 3 || commandElems.size() % 2 == 0) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.MSET));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MSetNXExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MSetNXExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MSetNXExecutor.java
index c7f52cd..0182f19 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MSetNXExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/MSetNXExecutor.java
@@ -22,7 +22,7 @@ public class MSetNXExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 3 || commandElems.size() % 2 == 0) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.MSETNX));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetBitExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetBitExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetBitExecutor.java
index ef8cd3e..e3e85b3 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetBitExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetBitExecutor.java
@@ -21,7 +21,7 @@ public class SetBitExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 4) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.SETBIT));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetEXExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetEXExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetEXExecutor.java
index 6ad4c90..448f912 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetEXExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetEXExecutor.java
@@ -24,7 +24,7 @@ public class SetEXExecutor extends StringExecutor implements Extendable {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 4) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), getArgsError()));
@@ -54,7 +54,7 @@ public class SetEXExecutor extends StringExecutor implements Extendable {
     checkAndSetDataType(key, context);
     r.put(key, new ByteArrayWrapper(value));
 
-    context.getRegionCache().setExpiration(key, expiration);
+    context.getRegionProvider().setExpiration(key, expiration);
 
     command.setResponse(Coder.getSimpleStringResponse(context.getByteBufAllocator(), SUCCESS));
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetExecutor.java
index c4b9b85..b6dc703 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetExecutor.java
@@ -19,7 +19,7 @@ public class SetExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 3) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.SET));
@@ -90,7 +90,7 @@ public class SetExecutor extends StringExecutor {
     }
 
     if (keyWasSet && expiration > 0L) {
-      context.getRegionCache().setExpiration(key, expiration);
+      context.getRegionProvider().setExpiration(key, expiration);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetNXExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetNXExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetNXExecutor.java
index 0956a6a..0daaeb9 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetNXExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetNXExecutor.java
@@ -21,7 +21,7 @@ public class SetNXExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 3) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.SETNX));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetRangeExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetRangeExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetRangeExecutor.java
index 8f00a83..1656065 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetRangeExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/SetRangeExecutor.java
@@ -19,7 +19,7 @@ public class SetRangeExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 4) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.SETRANGE));
@@ -48,7 +48,7 @@ public class SetRangeExecutor extends StringExecutor {
       int length = wrapper == null ? 0 : wrapper.toBytes().length;
       command.setResponse(Coder.getIntegerResponse(context.getByteBufAllocator(), length));
       if (wrapper == null)
-        context.getRegionCache().removeKey(key);
+        context.getRegionProvider().removeKey(key);
       return;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/StringExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/StringExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/StringExecutor.java
index 1f95ce6..49c81b3 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/StringExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/StringExecutor.java
@@ -9,7 +9,7 @@ import com.gemstone.gemfire.internal.redis.executor.AbstractExecutor;
 public abstract class StringExecutor extends AbstractExecutor {
   
   protected final void checkAndSetDataType(ByteArrayWrapper key, ExecutionHandlerContext context) {
-    Object oldVal = context.getRegionCache().metaPutIfAbsent(key, RedisDataType.REDIS_STRING);
+    Object oldVal = context.getRegionProvider().metaPutIfAbsent(key, RedisDataType.REDIS_STRING);
     if (oldVal == RedisDataType.REDIS_PROTECTED)
       throw new RedisDataTypeMismatchException("The key name \"" + key + "\" is protected");
     if (oldVal != null && oldVal != RedisDataType.REDIS_STRING)
@@ -17,7 +17,7 @@ public abstract class StringExecutor extends AbstractExecutor {
   }
   
   protected void checkDataType(ByteArrayWrapper key, ExecutionHandlerContext context) {
-    RedisDataType currentType = context.getRegionCache().getRedisDataType(key);
+    RedisDataType currentType = context.getRegionProvider().getRedisDataType(key);
     if (currentType == null)
       return;
     if (currentType == RedisDataType.REDIS_PROTECTED)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/StrlenExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/StrlenExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/StrlenExecutor.java
index 8174f1c..389de5a 100755
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/StrlenExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/executor/string/StrlenExecutor.java
@@ -18,7 +18,7 @@ public class StrlenExecutor extends StringExecutor {
   public void executeCommand(Command command, ExecutionHandlerContext context) {
     List<byte[]> commandElems = command.getProcessedCommand();
 
-    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionCache().getStringsRegion();
+    Region<ByteArrayWrapper, ByteArrayWrapper> r = context.getRegionProvider().getStringsRegion();
 
     if (commandElems.size() < 2) {
       command.setResponse(Coder.getErrorResponse(context.getByteBufAllocator(), ArityDef.STRLEN));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/org/apache/hadoop/fs/GlobPattern.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/org/apache/hadoop/fs/GlobPattern.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/org/apache/hadoop/fs/GlobPattern.java
new file mode 100644
index 0000000..64a2b6b
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/redis/org/apache/hadoop/fs/GlobPattern.java
@@ -0,0 +1,164 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.gemstone.gemfire.internal.redis.org.apache.hadoop.fs;
+
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+
+/**
+ * A class for POSIX glob pattern with brace expansions.
+ */
+public class GlobPattern {
+  private static final char BACKSLASH = '\\';
+  private Pattern compiled;
+  private boolean hasWildcard = false;
+
+  /**
+   * Construct the glob pattern object with a glob pattern string
+   * @param globPattern the glob pattern string
+   */
+  public GlobPattern(String globPattern) {
+    set(globPattern);
+  }
+
+  /**
+   * @return the compiled pattern
+   */
+  public Pattern compiled() {
+    return compiled;
+  }
+
+  /**
+   * Compile glob pattern string
+   * @param globPattern the glob pattern
+   * @return the pattern object
+   */
+  public static Pattern compile(String globPattern) {
+    return new GlobPattern(globPattern).compiled();
+  }
+
+  /**
+   * Match input against the compiled glob pattern
+   * @param s input chars
+   * @return true for successful matches
+   */
+  public boolean matches(CharSequence s) {
+    return compiled.matcher(s).matches();
+  }
+
+  /**
+   * Set and compile a glob pattern
+   * @param glob  the glob pattern string
+   */
+  public void set(String glob) {
+    StringBuilder regex = new StringBuilder();
+    int setOpen = 0;
+    int curlyOpen = 0;
+    int len = glob.length();
+    hasWildcard = false;
+
+    for (int i = 0; i < len; i++) {
+      char c = glob.charAt(i);
+
+      switch (c) {
+      case BACKSLASH:
+        if (++i >= len) {
+          error("Missing escaped character", glob, i);
+        }
+        regex.append(c).append(glob.charAt(i));
+        continue;
+      case '.':
+      case '$':
+      case '(':
+      case ')':
+      case '|':
+      case '+':
+        // escape regex special chars that are not glob special chars
+        regex.append(BACKSLASH);
+        break;
+      case '*':
+        regex.append('.');
+        hasWildcard = true;
+        break;
+      case '?':
+        regex.append('.');
+        hasWildcard = true;
+        continue;
+      case '{': // start of a group
+        regex.append("(?:"); // non-capturing
+        curlyOpen++;
+        hasWildcard = true;
+        continue;
+      case ',':
+        regex.append(curlyOpen > 0 ? '|' : c);
+        continue;
+      case '}':
+        if (curlyOpen > 0) {
+          // end of a group
+          curlyOpen--;
+          regex.append(")");
+          continue;
+        }
+        break;
+      case '[':
+        if (setOpen > 0) {
+          error("Unclosed character class", glob, i);
+        }
+        setOpen++;
+        hasWildcard = true;
+        break;
+      case '^': // ^ inside [...] can be unescaped
+        if (setOpen == 0) {
+          regex.append(BACKSLASH);
+        }
+        break;
+      case '!': // [! needs to be translated to [^
+        regex.append(setOpen > 0 && '[' == glob.charAt(i - 1) ? '^' : '!');
+        continue;
+      case ']':
+        // Many set errors like [][] could not be easily detected here,
+        // as []], []-] and [-] are all valid POSIX glob and java regex.
+        // We'll just let the regex compiler do the real work.
+        setOpen = 0;
+        break;
+      default:
+      }
+      regex.append(c);
+    }
+
+    if (setOpen > 0) {
+      error("Unclosed character class", glob, len);
+    }
+    if (curlyOpen > 0) {
+      error("Unclosed group", glob, len);
+    }
+    compiled = Pattern.compile(regex.toString());
+  }
+
+  /**
+   * @return true if this is a wildcard pattern (with special chars)
+   */
+  public boolean hasWildcard() {
+    return hasWildcard;
+  }
+
+  private static void error(String message, String pattern, int pos) {
+    throw new PatternSyntaxException(message, pattern, pos);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/main/java/com/gemstone/gemfire/redis/GemFireRedisServer.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/redis/GemFireRedisServer.java b/gemfire-core/src/main/java/com/gemstone/gemfire/redis/GemFireRedisServer.java
index 662e0a8..821ea08 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/redis/GemFireRedisServer.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/redis/GemFireRedisServer.java
@@ -46,7 +46,7 @@ import com.gemstone.gemfire.internal.redis.ByteToCommandDecoder;
 import com.gemstone.gemfire.internal.redis.Coder;
 import com.gemstone.gemfire.internal.redis.ExecutionHandlerContext;
 import com.gemstone.gemfire.internal.redis.RedisDataType;
-import com.gemstone.gemfire.internal.redis.RegionCache;
+import com.gemstone.gemfire.internal.redis.RegionProvider;
 import com.gemstone.gemfire.internal.redis.executor.hll.HyperLogLogPlus;
 
 /**
@@ -63,6 +63,12 @@ import com.gemstone.gemfire.internal.redis.executor.hll.HyperLogLogPlus;
  * or {@link GemFireRedisServer#STRING_REGION}. The default Region type is 
  * {@link RegionShortcut#PARTITION} although this can be changed by specifying the
  * SystemProperty {@value #DEFAULT_REGION_SYS_PROP_NAME} to a type defined by {@link RegionShortcut}.
+ * If the {@link GemFireRedisServer#NUM_THREADS_SYS_PROP_NAME} system property is set to 0,
+ * one thread per client will be created. Otherwise a worker thread pool of specified size is
+ * used or a default size of 4 * {@link Runtime#availableProcessors()} if the property is not set.
+ * <p>
+ * Setting the AUTH password requires setting the property "redis-password" just as "redis-port"
+ * would be in xml or through GFSH.
  * <p>
  * The supported commands are as follows:
  * <p>
@@ -182,7 +188,7 @@ public class GemFireRedisServer {
    */
   private LogWriter logger;
 
-  private RegionCache regionCache;
+  private RegionProvider regionCache;
 
   private final MetaCacheListener metaListener;
 
@@ -216,12 +222,6 @@ public class GemFireRedisServer {
   public static final String REDIS_META_DATA_REGION = "__ReDiS_MeTa_DaTa";
 
   /**
-   * The field that defines the name of the {@link Region} which holds all of
-   * the lists meta data. The current value of this field is {@value #LISTS_META_DATA_REGION}.
-   */
-  public static final String LISTS_META_DATA_REGION = "__LiStS_MeTa_DaTa";
-
-  /**
    * The system property name used to set the default {@link Region} creation
    * type. The property name is {@value #DEFAULT_REGION_SYS_PROP_NAME} and the
    * acceptable values are types defined by {@link RegionShortcut}, 
@@ -236,28 +236,23 @@ public class GemFireRedisServer {
   public static final String NUM_THREADS_SYS_PROP_NAME = "gemfireredis.numthreads";
 
   /**
-   * This String holds the String representation of the {@link RegionShortcut}
-   * that will be used as the default Region type for all region creation. This
-   * is set by the system property {@value #DEFAULT_REGION_SYS_PROP_NAME} and the
-   * default type is {@link RegionShortcut#PARTITION}.
-   */
-  private static final String regionType = System.getProperty(DEFAULT_REGION_SYS_PROP_NAME, "PARTITION");
-
-  /**
    * The actual {@link RegionShortcut} type specified by the system property
    * {@value #DEFAULT_REGION_SYS_PROP_NAME}.
    */
-  public static final RegionShortcut DEFAULT_REGION_TYPE = setRegion(regionType);
+  public final RegionShortcut DEFAULT_REGION_TYPE;
+
+  private boolean shutdown;
+  private boolean started;
 
   /**
    * Determine the {@link RegionShortcut} type from a String value.
    * If the String value doesn't map to a RegionShortcut type then 
    * {@link RegionShortcut#PARTITION} will be used by default.
    * 
-   * @param regionType The String representation of a {@link RegionShortcut}
    * @return {@link RegionShortcut}
    */
-  private static RegionShortcut setRegion(String regionType) {
+  private static RegionShortcut setRegionType() {
+    String regionType = System.getProperty(DEFAULT_REGION_SYS_PROP_NAME, "PARTITION");
     RegionShortcut type;
     try {
       type = RegionShortcut.valueOf(regionType);
@@ -312,9 +307,9 @@ public class GemFireRedisServer {
 
   /**
    * Constructor for {@link GemFireRedisServer} that will start the
-   * server and bind to the given address and port. Additionally the 
-   * number of threads used by the server to handle clients is specified
-   * as well as the logging level to be used by GemFire
+   * server and bind to the given address and port. Keep in mind that the
+   * log level configuration will only be set if a {@link Cache} does not already
+   * exist, if one already exists then setting that property will have no effect.
    * 
    * @param bindAddress The address to which the server will attempt to bind to
    * @param port The port the server will bind to, will use {@value #DEFAULT_REDIS_SERVER_PORT} by default if argument is less than or equal to 0
@@ -344,6 +339,9 @@ public class GemFireRedisServer {
       }
 
     });
+    this.DEFAULT_REGION_TYPE = setRegionType();
+    this.shutdown = false;
+    this.started = false;
   }
 
   /**
@@ -362,15 +360,18 @@ public class GemFireRedisServer {
    * This is function to call on a {@link GemFireRedisServer} instance
    * to start it running
    */
-  public void start() {
-    try {
-      startGemFire();
-      initializeRedis();
-      startRedisServer();
-    } catch (IOException e) {
-      throw new RuntimeException("Could not start Server", e);
-    } catch (InterruptedException e) {
-      throw new RuntimeException("Could not start Server", e);
+  public synchronized void start() {
+    if (!started) {
+      try {
+        startGemFire();
+        initializeRedis();
+        startRedisServer();
+      } catch (IOException e) {
+        throw new RuntimeException("Could not start Server", e);
+      } catch (InterruptedException e) {
+        throw new RuntimeException("Could not start Server", e);
+      }
+      started = true;
     }
   }
 
@@ -383,20 +384,24 @@ public class GemFireRedisServer {
   private void startGemFire() {
     Cache c = GemFireCacheImpl.getInstance();
     if (c == null) {
-      CacheFactory cacheFactory = new CacheFactory();
-      if (logLevel != null)
-        cacheFactory.set("log-level", logLevel);
-      this.cache = cacheFactory.create();
-    } else
-      this.cache = c;
-    this.logger = this.cache.getLogger();
+      synchronized (GemFireRedisServer.class) {
+        c = GemFireCacheImpl.getInstance();
+        if (c == null) {
+          CacheFactory cacheFactory = new CacheFactory();
+          if (logLevel != null)
+            cacheFactory.set("log-level", logLevel);
+          c = cacheFactory.create();
+        }
+      }
+    }
+    this.cache = c;
+    this.logger = c.getLogger();
   }
 
   private void initializeRedis() {
     synchronized (this.cache) {
       RegionFactory<String, RedisDataType> rfMeta = cache.createRegionFactory(RegionShortcut.REPLICATE);
       rfMeta.addCacheListener(this.metaListener);
-      RegionFactory<String, Integer> rfList = cache.createRegionFactory(RegionShortcut.REPLICATE);
       RegionFactory<ByteArrayWrapper, ByteArrayWrapper> rfString = cache.createRegionFactory(DEFAULT_REGION_TYPE);
       RegionFactory<ByteArrayWrapper, HyperLogLogPlus> rfHLL = cache.createRegionFactory(DEFAULT_REGION_TYPE);
       Region<ByteArrayWrapper, ByteArrayWrapper> stringsRegion;
@@ -408,14 +413,10 @@ public class GemFireRedisServer {
       Region<String, RedisDataType> redisMetaData;
       if ((redisMetaData = this.cache.getRegion(REDIS_META_DATA_REGION)) == null)
         redisMetaData = rfMeta.create(REDIS_META_DATA_REGION);
-      Region<String, Integer> listsMetaData;
-      if ((listsMetaData = this.cache.getRegion(LISTS_META_DATA_REGION)) == null)
-        listsMetaData = rfList.create(LISTS_META_DATA_REGION);
-      this.regionCache = new RegionCache(stringsRegion, hLLRegion, redisMetaData, listsMetaData, expirationFutures, expirationExecutor);
+      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);
       redisMetaData.put(STRING_REGION, RedisDataType.REDIS_PROTECTED);
-      redisMetaData.put(LISTS_META_DATA_REGION, RedisDataType.REDIS_PROTECTED);
     }
     checkForRegions();
   }
@@ -542,9 +543,9 @@ public class GemFireRedisServer {
       final String key = (String) event.getKey();
       final RedisDataType value = event.getOldValue();
       if (value != null && value != RedisDataType.REDIS_STRING && value != RedisDataType.REDIS_HLL && value != RedisDataType.REDIS_PROTECTED) {
-        Region<?, ?> r = this.regionCache.getRegion(Coder.stringToByteArrayWrapper(key));
+        ByteArrayWrapper kW = Coder.stringToByteArrayWrapper(key);
+        Region<?, ?> r = this.regionCache.getRegion(kW);
         if (r != null) { 
-          ByteArrayWrapper kW = Coder.stringToByteArrayWrapper(key);
           this.regionCache.removeRegionReferenceLocally(kW, value);
         }
       }
@@ -579,23 +580,26 @@ public class GemFireRedisServer {
    * Shutdown method for {@link GemFireRedisServer}. This closes the {@link Cache},
    * interrupts all execution and forcefully closes all connections.
    */
-  public void shutdown() {
-    if (logger.infoEnabled())
-      logger.info("GemFireRedisServer shutting down");
-    ChannelFuture closeFuture = this.serverChannel.closeFuture();
-    this.serverChannel.close();
-    Future<?> c = workerGroup.shutdownGracefully();
-    Future<?> c2 = bossGroup.shutdownGracefully();
-    c.syncUninterruptibly();
-    c2.syncUninterruptibly();
-    this.regionCache.close();
-    if (mainThread != null)
-      mainThread.interrupt();
-    for (ScheduledFuture<?> f : this.expirationFutures.values())
-      f.cancel(true);
-    this.expirationFutures.clear();
-    this.expirationExecutor.shutdownNow();
-    closeFuture.syncUninterruptibly();
+  public synchronized void shutdown() {
+    if (!shutdown) {
+      if (logger.infoEnabled())
+        logger.info("GemFireRedisServer shutting down");
+      ChannelFuture closeFuture = this.serverChannel.closeFuture();
+      Future<?> c = workerGroup.shutdownGracefully();
+      Future<?> c2 = bossGroup.shutdownGracefully();
+      this.serverChannel.close();
+      c.syncUninterruptibly();
+      c2.syncUninterruptibly();
+      this.regionCache.close();
+      if (mainThread != null)
+        mainThread.interrupt();
+      for (ScheduledFuture<?> f : this.expirationFutures.values())
+        f.cancel(true);
+      this.expirationFutures.clear();
+      this.expirationExecutor.shutdownNow();
+      closeFuture.syncUninterruptibly();
+      shutdown = true;
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java
new file mode 100644
index 0000000..618aca0
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/redis/ConcurrentStartTest.java
@@ -0,0 +1,58 @@
+package com.gemstone.gemfire.redis;
+
+import static org.junit.Assert.assertFalse;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class ConcurrentStartTest {
+
+  int numServers = 10;
+  @Test
+  public void testCachelessStart() throws InterruptedException {
+    runNServers(numServers);
+    GemFireCacheImpl.getInstance().close();
+  }
+  @Test
+  public void testCachefulStart() throws InterruptedException {
+    CacheFactory cf = new CacheFactory();
+    cf.set("mcast-port", "0");
+    cf.set("locators", "");
+    Cache c = cf.create();
+    runNServers(numServers);
+    c.close();
+  }
+  
+  private void runNServers(int n) throws InterruptedException {
+    final int[] ports = AvailablePortHelper.getRandomAvailableTCPPorts(numServers);
+    final Thread[] threads = new Thread[n];
+    for (int i = 0; i < n; i++) {
+      final int j = i;
+      Runnable r = new Runnable() {
+
+        @Override
+        public void run() {
+          GemFireRedisServer s = new GemFireRedisServer(ports[j]);
+          s.start();
+          s.shutdown();
+        }
+      };
+      
+      Thread t = new Thread(r);
+      t.setDaemon(true);
+      t.start();
+      threads[i] = t;
+    }
+    for (Thread t : threads)
+      t.join();
+    Cache c = GemFireCacheImpl.getInstance();
+    assertFalse(c.isClosed());
+  }
+}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/test/java/com/gemstone/gemfire/redis/RedisDistDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/redis/RedisDistDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/redis/RedisDistDUnitTest.java
new file mode 100644
index 0000000..858321d
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/redis/RedisDistDUnitTest.java
@@ -0,0 +1,231 @@
+package com.gemstone.gemfire.redis;
+
+import java.util.Random;
+
+import redis.clients.jedis.Jedis;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache30.CacheTestCase;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+
+import dunit.AsyncInvocation;
+import dunit.Host;
+import dunit.SerializableCallable;
+import dunit.VM;
+
+public class RedisDistDUnitTest extends CacheTestCase {
+
+  public static final String TEST_KEY = "key";
+  public static int pushes = 200;
+  int redisPort = 6379;
+  private Host host;
+  private VM server1;
+  private VM server2;
+  private VM client1;
+  private VM client2;
+
+  private int server1Port;
+  private int server2Port;
+
+  private abstract class ClientTestBase extends SerializableCallable {
+
+    int port;
+    protected ClientTestBase (int port) {
+      this.port = port;
+    }
+
+  }
+
+  public RedisDistDUnitTest() throws Throwable {
+    super("RedisDistTest");  
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    disconnectAllFromDS();
+    host = Host.getHost(0);
+    server1 = host.getVM(0);
+    server2 = host.getVM(1);
+    client1 = host.getVM(2);
+    client2 = host.getVM(3);  
+    final SerializableCallable<Object> startRedisAdapter = new SerializableCallable<Object>() {
+
+      private static final long serialVersionUID = 1978017907725504294L;
+
+      @Override
+      public Object call() throws Exception {
+        int port = AvailablePortHelper.getRandomAvailableTCPPort();
+        CacheFactory cF = new CacheFactory();
+        cF.set("log-level", "info");
+        cF.set("redis-bind-address", "localhost");
+        cF.set("redis-port", ""+port);
+        cF.set("mcast-port", "40404");
+        cF.create();
+        return Integer.valueOf(port);
+      }
+    };
+    AsyncInvocation i = server1.invokeAsync(startRedisAdapter);
+    server2Port = (Integer) server2.invoke(startRedisAdapter);
+    try {
+      server1Port = (Integer) i.getResult();
+    } catch (Throwable e) {
+      throw new Exception(e);
+    }
+  }
+
+  @Override
+  public void tearDown2() throws Exception {
+    super.tearDown2();
+    disconnectAllFromDS();
+  }
+
+  public void testConcListOps() throws Throwable {
+    final Jedis jedis1 = new Jedis("localhost", server1Port, 10000);
+    final Jedis jedis2 = new Jedis("localhost", server2Port, 10000);
+    final int pushes = 20;
+    class ConcListOps extends ClientTestBase {
+      protected ConcListOps(int port) {
+        super(port);
+      }
+
+      @Override
+      public Object call() throws Exception {
+        Jedis jedis = new Jedis("localhost", port, 10000);
+        Random r = new Random();
+        for (int i = 0; i < pushes; i++) {
+          if (r.nextBoolean()) {
+            jedis.lpush(TEST_KEY, randString());
+          } else {
+            jedis.rpush(TEST_KEY, randString());
+          }
+        }
+        return null;
+      }
+    };
+
+    AsyncInvocation i = client1.invokeAsync(new ConcListOps(server1Port));
+    client2.invoke(new ConcListOps(server2Port));
+    i.getResult();
+    long expected = 2 * pushes;
+    long result1 = jedis1.llen(TEST_KEY);
+    long result2 = jedis2.llen(TEST_KEY);
+    assertEquals(expected, result1);
+    assertEquals(result1, result2);
+  }
+
+
+  public void testConcCreateDestroy() throws Throwable {
+    final int ops = 40;
+    final String hKey = TEST_KEY+"hash";
+    final String lKey = TEST_KEY+"list";
+    final String zKey = TEST_KEY+"zset";
+    final String sKey = TEST_KEY+"set";
+
+    class ConcCreateDestroy extends ClientTestBase{
+      protected ConcCreateDestroy(int port) {
+        super(port);
+      }
+
+      @Override
+      public Object call() throws Exception {
+        Jedis jedis = new Jedis("localhost", port, 10000);
+        Random r = new Random();
+        for (int i = 0; i < ops; i++) {
+          int n = r.nextInt(4);
+          if (n == 0) {
+            if (r.nextBoolean()) {
+              jedis.hset(hKey, randString(), randString());
+            } else {
+              jedis.del(hKey);
+            }
+          } else if (n == 1) {
+            if (r.nextBoolean()) {
+              jedis.lpush(lKey, randString());
+            } else {
+              jedis.del(lKey);
+            }
+          } else if (n == 2) {
+            if (r.nextBoolean()) {
+              jedis.zadd(zKey, r.nextDouble(), randString());
+            } else {
+              jedis.del(zKey);
+            }
+          } else {
+            if (r.nextBoolean()) {
+              jedis.sadd(sKey, randString());
+            } else {
+              jedis.del(sKey);
+            }
+          }
+        }
+        return null;
+      }
+    }
+
+    // Expect to run with no exception
+    AsyncInvocation i = client1.invokeAsync(new ConcCreateDestroy(server1Port));
+    client2.invoke(new ConcCreateDestroy(server2Port));
+    i.getResult();
+  }
+
+  /**
+   * Just make sure there are no unexpected server crashes
+   * @throws Throwable 
+   */
+  public void testConcOps() throws Throwable {
+
+    final int ops = 100;
+    final String hKey = TEST_KEY+"hash";
+    final String lKey = TEST_KEY+"list";
+    final String zKey = TEST_KEY+"zset";
+    final String sKey = TEST_KEY+"set";
+
+    class ConcOps extends ClientTestBase {
+
+      protected ConcOps(int port) {
+        super(port);
+      }
+
+      @Override
+      public Object call() throws Exception {
+        Jedis jedis = new Jedis("localhost", port, 10000);
+        Random r = new Random();
+        for (int i = 0; i < ops; i++) {
+          int n = r.nextInt(4);
+          if (n == 0) {
+            jedis.hset(hKey, randString(), randString());
+            jedis.hgetAll(hKey);
+            jedis.hvals(hKey);
+          } else if (n == 1) {
+            jedis.lpush(lKey, randString());
+            jedis.rpush(lKey, randString());
+            jedis.ltrim(lKey, 0, 100);
+            jedis.lrange(lKey, 0, -1);
+          } else if (n == 2) {
+            jedis.zadd(zKey, r.nextDouble(), randString());
+            jedis.zrangeByLex(zKey, "(a", "[z");
+            jedis.zrangeByScoreWithScores(zKey, 0, 1, 0, 100);
+            jedis.zremrangeByScore(zKey, r.nextDouble(), r.nextDouble());
+          } else {
+            jedis.sadd(sKey, randString());
+            jedis.smembers(sKey);
+            jedis.sdiff(sKey, "afd");
+            jedis.sunionstore("dst", sKey, "afds");
+          }
+        }
+        return null;
+      }
+    }
+
+    // Expect to run with no exception
+    AsyncInvocation i = client1.invokeAsync(new ConcOps(server1Port));
+    client2.invoke(new ConcOps(server2Port));
+    i.getResult();
+  }
+
+  private String randString() {
+    return Long.toHexString(Double.doubleToLongBits(Math.random()));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/1a6a0ef5/gemfire-core/src/test/java/com/gemstone/gemfire/redis/SetsJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/redis/SetsJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/redis/SetsJUnitTest.java
new file mode 100755
index 0000000..abc6060
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/redis/SetsJUnitTest.java
@@ -0,0 +1,242 @@
+package com.gemstone.gemfire.redis;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Random;
+import java.util.Set;
+
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import redis.clients.jedis.Jedis;
+
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.GemFireCache;
+import com.gemstone.gemfire.internal.AvailablePortHelper;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class SetsJUnitTest {
+
+  private static Jedis jedis;
+  private static GemFireRedisServer server;
+  private static GemFireCache cache;
+  private static Random rand;
+  private static int port = 6379;
+
+  @BeforeClass
+  public static void setUp() throws IOException {
+    rand = new Random();
+    CacheFactory cf = new CacheFactory();
+    //cf.set("log-file", "redis.log");
+    cf.set("log-level", "error");
+    cf.set("mcast-port", "0");
+    cf.set("locators", "");
+    cache = cf.create();
+    port = AvailablePortHelper.getRandomAvailableTCPPort();
+    server = new GemFireRedisServer("localhost", port);
+
+    server.start();
+    jedis = new Jedis("localhost", port, 10000000);
+  }
+
+  @Test
+  public void testSAddScard() {
+    int elements = 10;
+    Set<String> strings = new HashSet<String>();
+    String key = randString();
+    for (int i = 0; i < elements; i++) {
+      String elem = randString();
+      strings.add(elem);
+    }
+    String[] stringArray = strings.toArray(new String[strings.size()]);
+    Long response = jedis.sadd(key, stringArray);
+    assertEquals(response, new Long(strings.size()));
+
+    assertEquals(jedis.scard(key), new Long(strings.size()));
+  }
+
+  @Test
+  public void testSMembersIsMember() {
+    int elements = 10;
+    Set<String> strings = new HashSet<String>();
+    String key = randString();
+    for (int i = 0; i < elements; i++) {
+      String elem = randString();
+      strings.add(elem);
+    }
+    String[] stringArray = strings.toArray(new String[strings.size()]);
+    jedis.sadd(key, stringArray);
+
+    Set<String> returnedSet = jedis.smembers(key);
+
+    assertEquals(returnedSet, new HashSet<String>(strings));
+
+    for (String entry: strings) {
+      boolean exists = jedis.sismember(key, entry);
+      assertTrue(exists);
+    }
+  }
+
+  @Test
+  public void testSMove() {
+    String source = randString();
+    String dest = randString();
+    String test = randString();
+    int elements = 10;
+    Set<String> strings = new HashSet<String>();
+    for (int i = 0; i < elements; i++) {
+      String elem = randString();
+      strings.add(elem);
+    }
+    String[] stringArray = strings.toArray(new String[strings.size()]);
+    jedis.sadd(source, stringArray);
+
+    long i = 1;
+    for (String entry: strings) {
+      assertTrue(jedis.smove(source, dest, entry) == 1);
+      assertTrue(jedis.sismember(dest, entry));
+      assertTrue(jedis.scard(source) == strings.size() - i);
+      assertTrue(jedis.scard(dest) == i);
+      i++;
+    }
+
+    assertTrue(jedis.smove(test, dest, randString()) == 0);
+  }
+
+  @Test
+  public void testSDiffAndStore() {
+    int numSets = 3;
+    int elements = 10;
+    String[] keys = new String[numSets];
+    ArrayList<Set<String>> sets = new ArrayList<Set<String>>();
+    for (int j = 0; j < numSets; j++) {
+      keys[j] = randString();
+      Set<String> newSet = new HashSet<String>();
+      for (int i = 0; i < elements; i++)
+        newSet.add(randString());
+      sets.add(newSet);
+    }
+    
+    for (int i = 0; i < numSets; i++) {
+      Set<String> s = sets.get(i);
+      String[] stringArray = s.toArray(new String[s.size()]);
+      jedis.sadd(keys[i], stringArray);
+    }
+    
+    Set<String> result = sets.get(0);
+    for (int i = 1; i < numSets; i++)
+      result.removeAll(sets.get(i));
+    
+    assertEquals(result, jedis.sdiff(keys));
+    
+    String destination = randString();
+    
+    jedis.sdiffstore(destination, keys);
+    
+    Set<String> destResult = jedis.smembers(destination);
+    
+    assertEquals(result, destResult);
+    
+  }
+  
+  @Test
+  public void testSUnionAndStore() {
+    int numSets = 3;
+    int elements = 10;
+    String[] keys = new String[numSets];
+    ArrayList<Set<String>> sets = new ArrayList<Set<String>>();
+    for (int j = 0; j < numSets; j++) {
+      keys[j] = randString();
+      Set<String> newSet = new HashSet<String>();
+      for (int i = 0; i < elements; i++)
+        newSet.add(randString());
+      sets.add(newSet);
+    }
+    
+    for (int i = 0; i < numSets; i++) {
+      Set<String> s = sets.get(i);
+      String[] stringArray = s.toArray(new String[s.size()]);
+      jedis.sadd(keys[i], stringArray);
+    }
+    
+    Set<String> result = sets.get(0);
+    for (int i = 1; i < numSets; i++)
+      result.addAll(sets.get(i));
+    
+    assertEquals(result, jedis.sunion(keys));
+    
+    String destination = randString();
+    
+    jedis.sunionstore(destination, keys);
+    
+    Set<String> destResult = jedis.smembers(destination);
+    
+    assertEquals(result, destResult);
+    
+  }
+  
+  @Test
+  public void testSInterAndStore() {
+    int numSets = 3;
+    int elements = 10;
+    String[] keys = new String[numSets];
+    ArrayList<Set<String>> sets = new ArrayList<Set<String>>();
+    for (int j = 0; j < numSets; j++) {
+      keys[j] = randString();
+      Set<String> newSet = new HashSet<String>();
+      for (int i = 0; i < elements; i++)
+        newSet.add(randString());
+      sets.add(newSet);
+    }
+    
+    for (int i = 0; i < numSets; i++) {
+      Set<String> s = sets.get(i);
+      String[] stringArray = s.toArray(new String[s.size()]);
+      jedis.sadd(keys[i], stringArray);
+    }
+    
+    Set<String> result = sets.get(0);
+    for (int i = 1; i < numSets; i++)
+      result.retainAll(sets.get(i));
+    
+    assertEquals(result, jedis.sinter(keys));
+    
+    String destination = randString();
+    
+    jedis.sinterstore(destination, keys);
+    
+    Set<String> destResult = jedis.smembers(destination);
+    
+    assertEquals(result, destResult);
+    
+  }
+
+  private String randString() {
+    int length = rand.nextInt(8) + 5;
+    StringBuilder rString = new StringBuilder();
+    for (int i = 0; i < length; i++)
+      rString.append((char) (rand.nextInt(57) + 65));
+    return rString.toString();
+    //return Long.toHexString(Double.doubleToLongBits(Math.random()));
+  }
+
+  @After
+  public void flushAll() {
+    jedis.flushAll();
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    jedis.close();
+    cache.close();
+    server.shutdown();
+  }
+}


[40/50] [abbrv] incubator-geode git commit: GEODE-193: Enable subset of CopyOnReadIndexJUnitTests

Posted by bs...@apache.org.
GEODE-193: Enable subset of CopyOnReadIndexJUnitTests


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

Branch: refs/heads/feature/GEODE-77
Commit: 96fd856db31f491c2d1977fbcf01b72b1ded25c0
Parents: bd84581
Author: Jason Huynh <hu...@gmail.com>
Authored: Tue Aug 11 10:17:04 2015 -0700
Committer: Jason Huynh <hu...@gmail.com>
Committed: Tue Aug 11 10:17:04 2015 -0700

----------------------------------------------------------------------
 .../cache/query/internal/index/CopyOnReadIndexJUnitTest.java    | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/96fd856d/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexJUnitTest.java
index dd5f980..e7fe997 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexJUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache/query/internal/index/CopyOnReadIndexJUnitTest.java
@@ -35,7 +35,7 @@ import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 public class CopyOnReadIndexJUnitTest {
   
   static int numObjects = 10;
-  static int objectsAndResultsMultiplier = 100;
+  static int objectsAndResultsMultiplier = 10;
   QueryTestUtils utils;
   static String regionName = "portfolios";
   static final String indexName = "testIndex";
@@ -182,7 +182,6 @@ public class CopyOnReadIndexJUnitTest {
   }
   
   //Test copy on read false
- /*
   @Test
   public void testCopyOnReadFalseWithHashIndexWithLocalRegion() throws Exception {
     utils.getCache().setCopyOnRead(false);
@@ -278,7 +277,7 @@ public class CopyOnReadIndexJUnitTest {
     utils.createIndex(indexName, "pv.secId", "/" + regionName + " p, p.positions.values pv");
     helpExecuteQueriesCopyOnReadFalse(queries, expectedResults, numObjects, objectsAndResultsMultiplier, true, true);
   }
- */ 
+
  /**
   * 
   * @param queries


[46/50] [abbrv] incubator-geode git commit: GEODE-212: improve off-heap + ResourceManager performance

Posted by bs...@apache.org.
GEODE-212: improve off-heap + ResourceManager performance

The synchronous code that notifies the async listener now does
a quick check to see if an event might be needed. This is done
with no synchronization. It just does two volatile reads.
The only time the synchronized notify is now needed is if we might
have crossed a threshold boundary or if the listener indicates it
really wants an event.

The old code called System.currentTimeMillis everytime it created
a MemoryEvent. This timestamp was only used by off-heap to reduce
how many notifications were done when the state is not normal.
Heap does notifications in abnormal states every time the amount
of memory changes. Off-heap does it when abnormal at most every second
by using the timestamp.
To get rid of all these currentTimeMillis calls the new code has the
background OffHeapMemoryUsageListener times out its wait call every
second. If it times out it will set a volatile that indicates it
wants the next abnormal event.


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

Branch: refs/heads/feature/GEODE-77
Commit: 587f5896a893d6512b1ae7b0fd6892e451ca0265
Parents: 1eccf91
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Aug 12 13:49:48 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Aug 13 15:29:42 2015 -0700

----------------------------------------------------------------------
 .../internal/cache/control/MemoryEvent.java     |  9 +-
 .../cache/control/OffHeapMemoryMonitor.java     | 88 ++++++++++++++++----
 2 files changed, 72 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/587f5896/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/MemoryEvent.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/MemoryEvent.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/MemoryEvent.java
index bda6518..c6dfd10 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/MemoryEvent.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/MemoryEvent.java
@@ -19,13 +19,12 @@ import com.gemstone.gemfire.internal.cache.control.MemoryThresholds.MemoryState;
  */
 public class MemoryEvent implements ResourceEvent {
   private final ResourceType type;
-  private volatile MemoryState state;
+  private final MemoryState state;
   private final MemoryState previousState;
   private final DistributedMember member;
   private final long bytesUsed;
   private final boolean isLocal;
   private final MemoryThresholds thresholds;
-  private final long eventTime;
  
   public MemoryEvent(final ResourceType type, final MemoryState previousState, final MemoryState state,
       final DistributedMember member, final long bytesUsed, final boolean isLocal, final MemoryThresholds thresholds) {
@@ -36,7 +35,6 @@ public class MemoryEvent implements ResourceEvent {
     this.bytesUsed = bytesUsed;
     this.isLocal = isLocal;
     this.thresholds = thresholds;
-    this.eventTime = System.currentTimeMillis();
   }
 
   @Override
@@ -66,10 +64,6 @@ public class MemoryEvent implements ResourceEvent {
     return this.isLocal;
   }
   
-  public long getEventTime() {
-    return this.eventTime;
-  }
-
   public MemoryThresholds getThresholds() {
     return this.thresholds;
   }
@@ -84,7 +78,6 @@ public class MemoryEvent implements ResourceEvent {
         .append(",state:" + this.state)
         .append(",bytesUsed:" + this.bytesUsed)
         .append(",isLocal:" + this.isLocal)
-        .append(",eventTime:" + this.eventTime)
         .append(",thresholds:" + this.thresholds + "]")
         .toString();
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/587f5896/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/OffHeapMemoryMonitor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/OffHeapMemoryMonitor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/OffHeapMemoryMonitor.java
index a1856e4..0678c01 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/OffHeapMemoryMonitor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/control/OffHeapMemoryMonitor.java
@@ -119,6 +119,9 @@ public class OffHeapMemoryMonitor implements ResourceMonitor, MemoryUsageListene
 
   @Override
   public void updateMemoryUsed(final long bytesUsed) {
+    if (!mightSendEvent(bytesUsed)) {
+      return;
+    }
     synchronized (this.offHeapMemoryUsageListener) {
       this.offHeapMemoryUsageListener.offHeapMemoryUsed = bytesUsed;
       this.offHeapMemoryUsageListener.notifyAll();
@@ -232,33 +235,70 @@ public class OffHeapMemoryMonitor implements ResourceMonitor, MemoryUsageListene
    * Public for testing.
    * 
    * @param bytesUsed
-   *          Number of bytes of heap memory currently used.
+   *          Number of bytes of off-heap memory currently used.
    */
   public void updateStateAndSendEvent(long bytesUsed) {
     synchronized (this) {
-      MemoryState oldState = this.mostRecentEvent.getState();
-      MemoryState newState = this.thresholds.computeNextState(oldState, bytesUsed);
+      final MemoryEvent mre = this.mostRecentEvent;
+      final MemoryState oldState = mre.getState();
+      final MemoryThresholds thresholds = this.thresholds;
+      MemoryState newState = thresholds.computeNextState(oldState, bytesUsed);
       if (oldState != newState) {
         this.currentState = newState;
         
-        MemoryEvent event = new MemoryEvent(ResourceType.OFFHEAP_MEMORY, oldState, newState, this.cache.getMyId(), bytesUsed, true,
-            this.thresholds);
+        MemoryEvent event = new MemoryEvent(ResourceType.OFFHEAP_MEMORY, oldState, newState, this.cache.getMyId(), bytesUsed, true, thresholds);
         this.upcomingEvent.set(event);
 
         processLocalEvent(event);
         updateStatsFromEvent(event);
         
-      // The state didn't change.  However, if the state isn't normal and we've
-      // been in that state for a while, send another event with the updated
-      // memory usage.
-      } else if (!oldState.isNormal() && (System.currentTimeMillis() - this.mostRecentEvent.getEventTime()) > 1000) {
-        MemoryEvent event = new MemoryEvent(ResourceType.OFFHEAP_MEMORY, oldState, newState, this.cache.getMyId(), bytesUsed, true,
-            this.thresholds);
+      } else if (!oldState.isNormal()
+          && bytesUsed != mre.getBytesUsed()
+          && this.deliverNextAbnormalEvent) {
+        this.deliverNextAbnormalEvent = false;
+        MemoryEvent event = new MemoryEvent(ResourceType.OFFHEAP_MEMORY, oldState, newState, this.cache.getMyId(), bytesUsed, true, thresholds);
         this.upcomingEvent.set(event);
         processLocalEvent(event);
       }
     }
   }
+  
+  /**
+   * Return true if the given number of bytes compared to the
+   * current monitor state would generate a new memory event.
+   * 
+   * @param bytesUsed
+   *          Number of bytes of off-heap memory currently used.
+   * @return true if a new event might need to be sent
+   */
+  private boolean mightSendEvent(long bytesUsed) {
+    final MemoryEvent mre = this.mostRecentEvent;
+    final MemoryState oldState = mre.getState();
+    final MemoryThresholds thresholds = this.thresholds;
+    MemoryState newState = thresholds.computeNextState(oldState, bytesUsed);
+    if (oldState != newState) {
+      return true;
+    } else if (!oldState.isNormal()
+        && bytesUsed != mre.getBytesUsed()
+        && this.deliverNextAbnormalEvent) {
+      return true;
+    }
+    return false;
+  }
+  
+  private volatile boolean deliverNextAbnormalEvent = false;
+   
+  /**
+   * Used by the OffHeapMemoryUsageListener to tell us that
+   * the next abnormal event should be delivered even if the
+   * state does not change as long as the memory usage changed.
+   * For some reason, unknown to me, if we stay in an abnormal
+   * state for more than a second then we want to send another
+   * event to update the memory usage.
+   */
+  void deliverNextAbnormalEvent() {
+    this.deliverNextAbnormalEvent = true;
+  }
 
   /**
    * Update resource manager stats based upon the given event.
@@ -387,7 +427,7 @@ public class OffHeapMemoryMonitor implements ResourceMonitor, MemoryUsageListene
   
   class OffHeapMemoryUsageListener implements Runnable {
     volatile boolean stopRequested = false;
-    volatile long offHeapMemoryUsed; // In bytes
+    long offHeapMemoryUsed; // In bytes
     
     OffHeapMemoryUsageListener(final long offHeapMemoryUsed) {
       this.offHeapMemoryUsed = offHeapMemoryUsed;
@@ -396,15 +436,29 @@ public class OffHeapMemoryMonitor implements ResourceMonitor, MemoryUsageListene
     @Override
     public void run() {
       getLogWriter().fine("OffHeapMemoryUsageListener is starting " + this);
-
+      long lastOffHeapMemoryUsed;
+      synchronized (this) {
+        lastOffHeapMemoryUsed = this.offHeapMemoryUsed;
+      }
       while (!this.stopRequested) {
-        final long saveOffHeapMemoryUsed = this.offHeapMemoryUsed;
-        updateStateAndSendEvent(saveOffHeapMemoryUsed);
+        updateStateAndSendEvent(lastOffHeapMemoryUsed);
 
         synchronized (this) {
-          if (saveOffHeapMemoryUsed == this.offHeapMemoryUsed && !this.stopRequested) {
+          if (lastOffHeapMemoryUsed == this.offHeapMemoryUsed && !this.stopRequested) {
             try {
-              this.wait();
+              do {
+                this.wait(1000);
+                if (this.offHeapMemoryUsed == lastOffHeapMemoryUsed) {
+                  // The wait timed out. So tell the OffHeapMemoryMonitor
+                  // that we need an event if the state is not normal.
+                  deliverNextAbnormalEvent();
+                } else {
+                  // we have been notified so exit the inner while loop
+                  // and call updateStateAndSendEvent.
+                  lastOffHeapMemoryUsed = this.offHeapMemoryUsed;
+                  break;
+                }
+              } while (true);
             } catch (InterruptedException iex) {
               getLogWriter().warning("OffHeapMemoryUsageListener was interrupted " + this);
               this.stopRequested = true;


[27/50] [abbrv] incubator-geode git commit: GEODE-188:wrong name for the gateway substitution filter in 'create async-event-queue' command -replaced string "gateway-event-substitution-listener" with "gateway-event-substitution-filter" and "Gateway

Posted by bs...@apache.org.
GEODE-188:wrong name for the gateway substitution filter in 'create async-event-queue' command
  -replaced string "gateway-event-substitution-listener" with "gateway-event-substitution-filter"
     and "GatewayEventSubstitutionListener" with "GatewayEventSubstitutionFilter

     Ran:
      QueueCommandsDUnitTest


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

Branch: refs/heads/feature/GEODE-77
Commit: 82de565d2b3ad923d422843c13d137eb1999af61
Parents: c575983
Author: kbachhav <kbachhav.pivotal.io>
Authored: Thu Aug 6 14:12:30 2015 +0530
Committer: kbachhav <kbachhav.pivotal.io>
Committed: Thu Aug 6 15:16:23 2015 +0530

----------------------------------------------------------------------
 .../management/internal/cli/commands/QueueCommands.java      | 4 ++--
 .../cli/functions/CreateAsyncEventQueueFunction.java         | 8 ++++----
 .../gemfire/management/internal/cli/i18n/CliStrings.java     | 4 ++--
 .../internal/web/controllers/QueueCommandsController.java    | 6 +++---
 4 files changed, 11 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82de565d/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
index 7b298d6..b59f38a 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/QueueCommands.java
@@ -110,8 +110,8 @@ public class QueueCommands implements CommandMarker {
                  help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER__HELP)
       @CliMetaData (valueSeparator = ",") 
       String[] gatewayEventFilters,
-      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_LISTENER, 
-                 help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_LISTENER__HELP)
+      @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER, 
+                 help = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER__HELP)
       String gatewaySubstitutionListener,
       @CliOption(key = CliStrings.CREATE_ASYNC_EVENT_QUEUE__LISTENER,
                  mandatory = true,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82de565d/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
index 547b199..4ee9e5d 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/CreateAsyncEventQueueFunction.java
@@ -67,7 +67,7 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
       final int dispatcherThreads =(Integer) args[9]; 
       final String orderPolicy= (String) args[10];
       final String[] gatewayEventFilters =(String[]) args[11];
-      final String gatewaySubstitutionListener = (String) args[12];
+      final String gatewaySubstitutionFilter = (String) args[12];
       final String listenerClassName = (String) args[13];
       final Properties listenerProperties = (Properties) args[14];
 
@@ -98,9 +98,9 @@ public class CreateAsyncEventQueueFunction extends FunctionAdapter implements In
           asyncEventQueueFactory.addGatewayEventFilter((GatewayEventFilter) newInstance(gatewayEventFilterKlass, CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER));
         }
       }
-      if (gatewaySubstitutionListener != null) {
-        Class<?> gatewayEventSubstitutionListenerKlass = forName(gatewaySubstitutionListener, CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_LISTENER);
-        asyncEventQueueFactory.setGatewayEventSubstitutionListener((GatewayEventSubstitutionFilter<?,?>) newInstance(gatewayEventSubstitutionListenerKlass, CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_LISTENER));
+      if (gatewaySubstitutionFilter != null) {
+        Class<?> gatewayEventSubstitutionFilterKlass = forName(gatewaySubstitutionFilter, CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER);
+        asyncEventQueueFactory.setGatewayEventSubstitutionListener((GatewayEventSubstitutionFilter<?,?>) newInstance(gatewayEventSubstitutionFilterKlass, CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER));
       }
       
       Object listenerInstance;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82de565d/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
index c65212f..678fd93 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/cli/i18n/CliStrings.java
@@ -428,8 +428,8 @@ public class CliStrings {
   public static final String CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY__HELP = "Policy for dispatching events when --dispatcher-threads is > 1. Possible values are 'THREAD', 'KEY', 'PARTITION'.";
   public static final String CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS = "dispatcher-threads";
   public static final String CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS__HELP = "Number of threads to use for sending events.";
-  public static final String CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_LISTENER = "gateway-event-substitution-listener";
-  public static final String CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_LISTENER__HELP = "Fully qualified class name of the GatewayEventSubstitutionListener for this queue.";
+  public static final String CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER = "gateway-event-substitution-filter";
+  public static final String CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER__HELP = "Fully qualified class name of the GatewayEventSubstitutionFilter for this queue.";
   public static final String CREATE_ASYNC_EVENT_QUEUE__LISTENER = "listener";
   public static final String CREATE_ASYNC_EVENT_QUEUE__LISTENER__HELP = "Fully qualified class name of the AsyncEventListener for this queue.";
   public static final String CREATE_ASYNC_EVENT_QUEUE__LISTENER_PARAM_AND_VALUE = "listener-param";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/82de565d/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
index 00cbd1a..a317aca 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/management/internal/web/controllers/QueueCommandsController.java
@@ -54,7 +54,7 @@ public class QueueCommandsController extends AbstractCommandsController {
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__DISPATCHERTHREADS, defaultValue = "1") final Integer dispatcherThreads,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__ORDERPOLICY, defaultValue = "KEY") final String orderPolicy,
                                       @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__GATEWAYEVENTFILTER, required = false) final String[] gatewayEventFilters,
-                                      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_LISTENER, required = false) final String gatewaySubstitutionListener)
+                                      @RequestParam(value = CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER, required = false) final String gatewaySubstitutionFilter)
 
   {
     CommandStringBuilder command = new CommandStringBuilder(CliStrings.CREATE_ASYNC_EVENT_QUEUE);
@@ -108,8 +108,8 @@ public class QueueCommandsController extends AbstractCommandsController {
           gatewayEventFilters, StringUtils.COMMA_DELIMITER));
     }
 
-    if (hasValue(gatewaySubstitutionListener)) {
-      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_LISTENER, gatewaySubstitutionListener);
+    if (hasValue(gatewaySubstitutionFilter)) {
+      command.addOption(CliStrings.CREATE_ASYNC_EVENT_QUEUE__SUBSTITUTION_FILTER, gatewaySubstitutionFilter);
     }
 
     return processCommand(command.toString());


[28/50] [abbrv] incubator-geode git commit: GEODE-185: Fix races in testRegionIdleInvalidate

Posted by bs...@apache.org.
GEODE-185: Fix races in testRegionIdleInvalidate

The test code is now careful to wait for the expiration clock to advance.
A different assertion will be triggered if the expiration clock goes back in time.
Fixed two places in the product expiration code that did not use the expiration clock.


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

Branch: refs/heads/feature/GEODE-77
Commit: 7d4ae09fc92288788868739451254320617289ca
Parents: 82de565
Author: Darrel Schneider <ds...@pivotal.io>
Authored: Wed Aug 5 14:17:19 2015 -0700
Committer: Darrel Schneider <ds...@pivotal.io>
Committed: Thu Aug 6 09:46:12 2015 -0700

----------------------------------------------------------------------
 .../gemfire/internal/cache/ExpiryTask.java       |  2 +-
 .../internal/cache/RegionIdleExpiryTask.java     |  3 ++-
 .../gemstone/gemfire/cache30/RegionTestCase.java | 15 ++++++++++-----
 .../src/test/java/dunit/DistributedTestCase.java | 19 +++++++++++++++----
 4 files changed, 28 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d4ae09f/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
index 95cd3a8..d5dc5ee 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/ExpiryTask.java
@@ -437,7 +437,7 @@ public abstract class ExpiryTask extends SystemTimer.SystemTimerTask {
     return super.toString() + " for " + getLocalRegion()
       + ", ttl expiration time: " + expTtl
       + ", idle expiration time: " + expIdle +
-      ("[now:" + System.currentTimeMillis() + "]");
+      ("[now:" + calculateNow() + "]");
   }
 
   ////// Abstract methods ///////

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d4ae09f/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionIdleExpiryTask.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionIdleExpiryTask.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionIdleExpiryTask.java
index fbcb12c..52975a2 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionIdleExpiryTask.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionIdleExpiryTask.java
@@ -39,7 +39,8 @@ class RegionIdleExpiryTask extends RegionExpiryTask {
         if (!getLocalRegion().EXPIRY_UNITS_MS) {
           timeout *= 1000;
         }
-        return  timeout + System.currentTimeMillis();
+        // Expiration should always use the DSClock instead of the System clock.
+        return  timeout + getLocalRegion().cacheTimeMillis();
       }
     }
     // otherwise, expire at timeout plus last accessed time

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d4ae09f/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
index 69eeec0..c063a55 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/cache30/RegionTestCase.java
@@ -3853,29 +3853,34 @@ public abstract class RegionTestCase extends CacheTestCase {
         // expiration time to be extended.
         // For this phase we don't worry about actually expiring but just
         // making sure the expiration time gets extended.
-        region.getAttributesMutator().setRegionIdleTimeout(new ExpirationAttributes(9000/*ms*/, ExpirationAction.INVALIDATE));
+        final int EXPIRATION_MS = 9000;
+        region.getAttributesMutator().setRegionIdleTimeout(new ExpirationAttributes(EXPIRATION_MS, ExpirationAction.INVALIDATE));
         
         LocalRegion lr = (LocalRegion) region;
         {
           ExpiryTask expiryTask = lr.getRegionIdleExpiryTask();
           region.put(key, value);
           long createExpiry = expiryTask.getExpirationTime();
-          waitForExpiryClockToChange(lr);
+          long changeTime = waitForExpiryClockToChange(lr, createExpiry-EXPIRATION_MS);
           region.put(key, "VALUE2");
           long putExpiry = expiryTask.getExpirationTime();
+          assertTrue("CLOCK went back in time! Expected putBaseExpiry=" + (putExpiry-EXPIRATION_MS) + " to be >= than changeTime=" + changeTime, (putExpiry-EXPIRATION_MS - changeTime) >= 0);
           assertTrue("expected putExpiry=" + putExpiry + " to be > than createExpiry=" + createExpiry, (putExpiry - createExpiry) > 0);
-          waitForExpiryClockToChange(lr);
+          changeTime = waitForExpiryClockToChange(lr, putExpiry-EXPIRATION_MS);
           region.get(key);
           long getExpiry = expiryTask.getExpirationTime();
+          assertTrue("CLOCK went back in time! Expected getBaseExpiry=" + (getExpiry-EXPIRATION_MS) + " to be >= than changeTime=" + changeTime, (getExpiry-EXPIRATION_MS - changeTime) >= 0);
           assertTrue("expected getExpiry=" + getExpiry + " to be > than putExpiry=" + putExpiry, (getExpiry - putExpiry) > 0);
         
-          waitForExpiryClockToChange(lr);
+          changeTime = waitForExpiryClockToChange(lr, getExpiry-EXPIRATION_MS);
           sub.put(key, value);
           long subPutExpiry = expiryTask.getExpirationTime();
+          assertTrue("CLOCK went back in time! Expected subPutBaseExpiry=" + (subPutExpiry-EXPIRATION_MS) + " to be >= than changeTime=" + changeTime, (subPutExpiry-EXPIRATION_MS - changeTime) >= 0);
           assertTrue("expected subPutExpiry=" + subPutExpiry + " to be > than getExpiry=" + getExpiry, (subPutExpiry - getExpiry) > 0);
-          waitForExpiryClockToChange(lr);
+          changeTime = waitForExpiryClockToChange(lr, subPutExpiry-EXPIRATION_MS);
           sub.get(key);
           long subGetExpiry = expiryTask.getExpirationTime();
+          assertTrue("CLOCK went back in time! Expected subGetBaseExpiry=" + (subGetExpiry-EXPIRATION_MS) + " to be >= than changeTime=" + changeTime, (subGetExpiry-EXPIRATION_MS - changeTime) >= 0);
           assertTrue("expected subGetExpiry=" + subGetExpiry + " to be > than subPutExpiry=" + subPutExpiry, (subGetExpiry - subPutExpiry) > 0);
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7d4ae09f/gemfire-core/src/test/java/dunit/DistributedTestCase.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/dunit/DistributedTestCase.java b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
index 564e7ef..8aa8b6d 100755
--- a/gemfire-core/src/test/java/dunit/DistributedTestCase.java
+++ b/gemfire-core/src/test/java/dunit/DistributedTestCase.java
@@ -976,13 +976,24 @@ public abstract class DistributedTestCase extends TestCase implements java.io.Se
   }
   
   /**
-   * Blocks until the clock used for expiration on the given region changes.
+   * Blocks until the clock used for expiration moves forward.
+   * @return the last time stamp observed
    */
-  public static final void waitForExpiryClockToChange(LocalRegion lr) {
-    long startTime = lr.cacheTimeMillis();
+  public static final long waitForExpiryClockToChange(LocalRegion lr) {
+    return waitForExpiryClockToChange(lr, lr.cacheTimeMillis());
+  }
+  /**
+   * Blocks until the clock used for expiration moves forward.
+   * @param baseTime the timestamp that the clock must exceed
+   * @return the last time stamp observed
+   */
+  public static final long waitForExpiryClockToChange(LocalRegion lr, final long baseTime) {
+    long nowTime;
     do {
       Thread.yield();
-    } while (startTime == lr.cacheTimeMillis());
+      nowTime = lr.cacheTimeMillis();
+    } while ((nowTime - baseTime) <= 0L);
+    return nowTime;
   }
   
   /** pause for specified ms interval