You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by as...@apache.org on 2015/11/11 22:24:03 UTC

[31/50] [abbrv] incubator-geode git commit: Adding an internal listener that can modify region attributes

Adding an internal listener that can modify region attributes

This listener is attached to the cache and gets a callback before any
region is created. This can be used by extensions to modify region
attributes to create resources need for a region before the region is
created.


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

Branch: refs/heads/develop
Commit: 485ebf9512ba11a19a469078ab136f80a1a85f7b
Parents: 62952c1
Author: Dan Smith <up...@apache.org>
Authored: Wed Oct 14 10:50:45 2015 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Thu Oct 15 10:59:17 2015 -0700

----------------------------------------------------------------------
 .../internal/cache/GemFireCacheImpl.java        | 27 +++++++++++
 .../gemfire/internal/cache/LocalRegion.java     |  8 ++--
 .../gemfire/internal/cache/RegionListener.java  | 30 +++++++++++++
 .../internal/cache/RegionListenerJUnitTest.java | 47 ++++++++++++++++++++
 4 files changed, 108 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/485ebf95/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 b0f0f43..e73af49 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
@@ -198,6 +198,7 @@ import com.gemstone.gemfire.internal.cache.wan.parallel.ParallelGatewaySenderQue
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlParser;
 import com.gemstone.gemfire.internal.cache.xmlcache.CacheXmlPropertyResolver;
 import com.gemstone.gemfire.internal.cache.xmlcache.PropertyResolver;
+import com.gemstone.gemfire.internal.concurrent.ConcurrentHashSet;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.jndi.JNDIInvoker;
 import com.gemstone.gemfire.internal.jta.TransactionManagerImpl;
@@ -589,6 +590,8 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
   
   private final CqService cqService;
   
+  private final Set<RegionListener> regionListeners = new ConcurrentHashSet<RegionListener>();
+  
   public static final int DEFAULT_CLIENT_FUNCTION_TIMEOUT = 0;
 
   private static int clientFunctionTimeout;
@@ -3023,6 +3026,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     stopper.checkCancelInProgress(null);
     LocalRegion.validateRegionName(name);
     RegionAttributes<K, V> attrs = p_attrs;
+    attrs = invokeRegionBefore(null, name, attrs, internalRegionArgs);
     if (attrs == null) {
       throw new IllegalArgumentException(LocalizedStrings.GemFireCache_ATTRIBUTES_MUST_NOT_BE_NULL.toLocalizedString());
     }
@@ -3158,6 +3162,7 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
       throw ex;
     }
 
+    invokeRegionAfter(rgn);
     /**
      * Added for M&M . Putting the callback here to avoid creating RegionMBean in case of Exception
      **/
@@ -3168,6 +3173,20 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
     return rgn;
   }
 
+  public RegionAttributes invokeRegionBefore(LocalRegion parent,
+      String name, RegionAttributes attrs, InternalRegionArguments internalRegionArgs) {
+    for(RegionListener listener : regionListeners) {
+      attrs = listener.beforeCreate(parent, name, attrs, internalRegionArgs);
+    }
+    return attrs;
+  }
+  
+  public void invokeRegionAfter(LocalRegion region) {
+    for(RegionListener listener : regionListeners) {
+      listener.afterCreate(region);
+    }
+  }
+
   /**
    * turn on eviction by default for HDFS regions
    */
@@ -3713,6 +3732,14 @@ public class GemFireCacheImpl implements InternalCache, ClientCache, HasCachePer
       return cacheLifecycleListeners.remove(l);
     }
   }
+  
+  public void addRegionListener(RegionListener l ) {
+    this.regionListeners.add(l);
+  }
+  
+  public void removeRegionListener(RegionListener l ) {
+    this.regionListeners.remove(l);
+  }
 
   /**
    * Creates the single instance of the Transation Manager for this cache. Returns the existing one upon request.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/485ebf95/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 9e5bcd2..22d42a9 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
@@ -8,7 +8,7 @@
 
 package com.gemstone.gemfire.internal.cache;
 
-import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.ENTRY_EVENT_NEW_VALUE;
+import static com.gemstone.gemfire.internal.offheap.annotations.OffHeapIdentifier.*;
 
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
@@ -114,8 +114,8 @@ import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HDFSRegionDirector;
 import com.gemstone.gemfire.cache.hdfs.internal.hoplog.HDFSRegionDirector.HdfsRegionManager;
 import com.gemstone.gemfire.cache.partition.PartitionRegionHelper;
 import com.gemstone.gemfire.cache.query.FunctionDomainException;
-import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
 import com.gemstone.gemfire.cache.query.Index;
+import com.gemstone.gemfire.cache.query.IndexMaintenanceException;
 import com.gemstone.gemfire.cache.query.IndexType;
 import com.gemstone.gemfire.cache.query.MultiIndexCreationException;
 import com.gemstone.gemfire.cache.query.NameResolutionException;
@@ -198,14 +198,12 @@ import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl;
 import com.gemstone.gemfire.internal.offheap.SimpleMemoryAllocatorImpl.Chunk;
 import com.gemstone.gemfire.internal.offheap.StoredObject;
-import com.gemstone.gemfire.internal.offheap.annotations.Released;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.offheap.annotations.Unretained;
 import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
 import com.gemstone.gemfire.internal.util.concurrent.FutureResult;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableCountDownLatch;
 import com.gemstone.gemfire.internal.util.concurrent.StoppableReadWriteLock;
-import com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantReadWriteLock.StoppableWriteLock;
 import com.gemstone.org.jgroups.util.StringId;
 
 /**
@@ -941,6 +939,7 @@ public class LocalRegion extends AbstractRegion
     checkReadiness();
     LocalRegion newRegion = null;
     RegionAttributes regionAttributes = attrs;
+    attrs = cache.invokeRegionBefore(this, subregionName, attrs, internalRegionArgs);
     final InputStream snapshotInputStream = internalRegionArgs
         .getSnapshotInputStream();
     final boolean getDestroyLock = internalRegionArgs.getDestroyLockFlag();
@@ -1110,6 +1109,7 @@ public class LocalRegion extends AbstractRegion
       }
     }
 
+    cache.invokeRegionAfter(newRegion);
     return newRegion;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/485ebf95/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionListener.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionListener.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionListener.java
new file mode 100644
index 0000000..88b913c
--- /dev/null
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/RegionListener.java
@@ -0,0 +1,30 @@
+package com.gemstone.gemfire.internal.cache;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+
+/**
+ * Callback on a cache that receives notifications about region creates.
+ * 
+ * If there are multiple listeners added to a cache they are invoked in a random
+ * order. See {@link GemFireCacheImpl#addRegionListener(RegionListener)}
+ * 
+ * If any of these callbacks throw an exception, that exception will get thrown
+ * out to the user creating the region and the region creation will fail.
+ */
+public interface RegionListener {
+  
+  /**
+   * Invoked before a region is created. This callback is allowed to modify the region
+   * attributes before the region is created. Note that it's generally a bad idea to modify
+   * the RegionAttributes in place; a new set of RegionAttributes should be returned that contain
+   * the modifications. InternalRegionArguments *may* be modified, but only if you are sure
+   * the caller is not going to reuse the InternalRegionArguments for something else.
+   */
+  public RegionAttributes beforeCreate(Region parent, String regionName, RegionAttributes attrs, InternalRegionArguments internalRegionArgs);
+
+  /**
+   * Invoked after a region is created.
+   */
+  public void afterCreate(Region region);
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/485ebf95/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RegionListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RegionListenerJUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RegionListenerJUnitTest.java
new file mode 100644
index 0000000..1fc7e59
--- /dev/null
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/RegionListenerJUnitTest.java
@@ -0,0 +1,47 @@
+package com.gemstone.gemfire.internal.cache;
+
+import static org.junit.Assert.*;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+
+@Category(IntegrationTest.class)
+public class RegionListenerJUnitTest {
+
+  @Test
+  public void test() {
+    final AtomicBoolean afterCreateInvoked = new AtomicBoolean(); 
+    RegionListener listener = new RegionListener() {
+      
+      @Override
+      public RegionAttributes beforeCreate(Region parent, String regionName,
+          RegionAttributes attrs, InternalRegionArguments internalRegionArgs) {
+        AttributesFactory newAttrsFactory = new AttributesFactory(attrs);
+        newAttrsFactory.setDataPolicy(DataPolicy.EMPTY);
+        return newAttrsFactory.create();
+      }
+
+      @Override
+      public void afterCreate(Region region) {
+        afterCreateInvoked.set(true);
+      }
+    };
+    
+    GemFireCacheImpl cache = (GemFireCacheImpl) new CacheFactory().set("mcast-port", "0").create();
+    cache.addRegionListener(listener);
+    Region region = cache.createRegionFactory(RegionShortcut.REPLICATE).create("region");
+    assertEquals(DataPolicy.EMPTY, region.getAttributes().getDataPolicy());
+    assertTrue(afterCreateInvoked.get());
+  }
+
+}