You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ji...@apache.org on 2016/06/27 21:20:51 UTC

[01/50] [abbrv] incubator-geode git commit: GEODE-1557: Not logging BucketNotFoundException

Repository: incubator-geode
Updated Branches:
  refs/heads/feature/GEODE-1571 756d8216e -> b43d0266d


GEODE-1557: Not logging BucketNotFoundException

Not logging BucketNotFoundException as an error in LuceneEventLister and added
a test shouldIgnoreBucketNotFoundException to verify it

This closes #168


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

Branch: refs/heads/feature/GEODE-1571
Commit: 83c7c661648c9fd7b0dac054dde60e2ae1ef12cd
Parents: 2df8674
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Thu Jun 16 16:07:04 2016 -0700
Committer: nabarun <nn...@pivotal.io>
Committed: Mon Jun 20 16:51:34 2016 -0700

----------------------------------------------------------------------
 .../lucene/internal/LuceneEventListener.java    | 10 +++++++---
 .../internal/LuceneEventListenerJUnitTest.java  | 21 ++++++++++++++++++++
 2 files changed, 28 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/83c7c661/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
index ca8077d..a7150c0 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListener.java
@@ -36,6 +36,7 @@ import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.CacheObserverHolder;
+import com.gemstone.gemfire.internal.cache.partitioned.Bucket;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy.TestHook;
 import com.gemstone.gemfire.internal.logging.LogService;
 
@@ -68,7 +69,7 @@ public class LuceneEventListener implements AsyncEventListener {
         Region region = event.getRegion();
         Object key = event.getKey();
         Object callbackArgument = event.getCallbackArgument();
-        
+
         IndexRepository repository = repositoryManager.getRepository(region, key, callbackArgument);
 
         Operation op = event.getOperation();
@@ -86,12 +87,15 @@ public class LuceneEventListener implements AsyncEventListener {
         }
         affectedRepos.add(repository);
       }
-      
+
       for(IndexRepository repo : affectedRepos) {
         repo.commit();
       }
       return true;
-    } catch(IOException | BucketNotFoundException e) {
+    } catch(BucketNotFoundException e) {
+      logger.debug("Bucket not found while saving to lucene index: " + e.getMessage());
+      return false;
+    } catch(IOException e) {
       logger.error("Unable to save to lucene index", e);
       return false;
     } finally {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/83c7c661/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListenerJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListenerJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListenerJUnitTest.java
index 53c9db1..86ed481 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListenerJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneEventListenerJUnitTest.java
@@ -18,13 +18,17 @@
  */
 package com.gemstone.gemfire.cache.lucene.internal;
 
+import static org.junit.Assert.*;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.*;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
+import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
@@ -34,7 +38,9 @@ import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEvent;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
+import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
+import org.apache.logging.log4j.Logger;
 
 /**
  * Unit test that async event listener dispatched the events
@@ -99,4 +105,19 @@ public class LuceneEventListenerJUnitTest {
     verify(repo1, times(1)).commit();
     verify(repo2, times(1)).commit();
   }
+
+  @Test
+  public void shouldHandleBucketNotFoundExceptionWithoutLoggingError() throws BucketNotFoundException {
+    RepositoryManager manager = Mockito.mock(RepositoryManager.class);
+    Logger log=Mockito.mock(Logger.class);
+    Mockito.when(manager.getRepository(any(), any(), any()))
+      .thenThrow(BucketNotFoundException.class);
+
+    LuceneEventListener listener = new LuceneEventListener(manager);
+    listener.logger = log;
+    AsyncEvent event = Mockito.mock(AsyncEvent.class);
+    boolean result = listener.processEvents(Arrays.asList(new AsyncEvent[] {event}));
+    assertFalse(result);
+    verify(log, never()).error(anyString(), any(Exception.class));
+  }
 }


[15/50] [abbrv] incubator-geode git commit: GEODE-1571: added security for query and destroy region

Posted by ji...@apache.org.
GEODE-1571: added security for query and destroy region


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

Branch: refs/heads/feature/GEODE-1571
Commit: 7ae6110caa86db46fce1ad7b9990507a213125a5
Parents: a4c2dad
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Thu Jun 23 11:34:33 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu Jun 23 11:34:33 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/BaseCommandQuery.java    |  11 +-
 .../tier/sockets/command/CloseConnection.java   |  13 +-
 .../tier/sockets/command/CreateRegion.java      | 122 +++++++-------
 .../tier/sockets/command/DestroyRegion.java     | 167 +++++++++----------
 .../security/IntegratedClientAuthDUnitTest.java |  42 ++++-
 .../internal/security/clientServer.json         |   7 +
 6 files changed, 200 insertions(+), 162 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ae6110c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommandQuery.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommandQuery.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommandQuery.java
index 3a80e25..8a95cf8 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommandQuery.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommandQuery.java
@@ -31,7 +31,6 @@ import com.gemstone.gemfire.cache.query.SelectResults;
 import com.gemstone.gemfire.cache.query.Struct;
 import com.gemstone.gemfire.cache.query.internal.CqEntry;
 import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
-import com.gemstone.gemfire.cache.query.internal.cq.InternalCqQuery;
 import com.gemstone.gemfire.cache.query.internal.cq.ServerCQ;
 import com.gemstone.gemfire.cache.query.internal.types.CollectionTypeImpl;
 import com.gemstone.gemfire.cache.query.internal.types.StructTypeImpl;
@@ -45,6 +44,7 @@ import com.gemstone.gemfire.internal.cache.tier.MessageType;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequestPP;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 public abstract class BaseCommandQuery extends BaseCommand {
 
@@ -109,6 +109,10 @@ public abstract class BaseCommandQuery extends BaseCommand {
     }
     // Process the query request
     try {
+      // integrated security
+      for(Object regionName:regionNames){
+        GeodeSecurityUtil.authorizeRegionRead(regionName.toString());
+      }
 
       // Execute query
       // startTime = GenericStats.getTime();
@@ -150,11 +154,6 @@ public abstract class BaseCommandQuery extends BaseCommand {
         result = queryContext.getQueryResult();
       }
 
-      // endTime = System.currentTimeMillis();
-      // System.out.println("Query executed in: " + (endTime-startTime) + "ms");
-      // GenericStats.endTime0(startTime);
-
-
       if (result instanceof SelectResults) {
         SelectResults selectResults = (SelectResults)result;
         if (logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ae6110c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseConnection.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseConnection.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseConnection.java
index e487592..40bc1e7 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseConnection.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseConnection.java
@@ -19,12 +19,16 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.tier.Command;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-
-import java.io.IOException;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
  
 public class CloseConnection extends BaseCommand {
 
@@ -52,7 +56,6 @@ public class CloseConnection extends BaseCommand {
     }
 
     try {
-      // clientHost = theSocket.getInetAddress().getCanonicalHostName();
       servConn.setClientDisconnectCleanly();
       String clientHost = servConn.getSocketHost();
       int clientPort = servConn.getSocketPort();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ae6110c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CreateRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CreateRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CreateRegion.java
index 36257b7..7889a72 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CreateRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CreateRegion.java
@@ -15,22 +15,26 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.security.NotAuthorizedException;
-import com.gemstone.gemfire.cache.AttributesFactory;
-import com.gemstone.gemfire.cache.Region;
-
-import java.io.IOException;
 
 
 public class CreateRegion extends BaseCommand {
@@ -45,8 +49,7 @@ public class CreateRegion extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException {
     Part regionNamePart = null;
     String regionName = null;
     CachedRegionHelper crHelper = servConn.getCachedRegionHelper();
@@ -54,7 +57,7 @@ public class CreateRegion extends BaseCommand {
     // bserverStats.incLong(readDestroyRequestTimeId,
     // DistributionStats.getStatTime() - start);
     // bserverStats.incInt(destroyRequestsId, 1);
-//    start = DistributionStats.getStatTime();
+    //    start = DistributionStats.getStatTime();
     // Retrieve the data from the message parts
     Part parentRegionNamePart = msg.getPart(0);
     String parentRegionName = parentRegionNamePart.getString();
@@ -63,74 +66,71 @@ public class CreateRegion extends BaseCommand {
     regionName = regionNamePart.getString();
 
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received create region request ({} bytes) from {} for parent region {} region {}", servConn.getName(), msg.getPayloadLength(), servConn.getSocketString(), parentRegionName, regionName);
+      logger.debug("{}: Received create region request ({} bytes) from {} for parent region {} region {}", servConn.getName(), msg
+        .getPayloadLength(), servConn.getSocketString(), parentRegionName, regionName);
     }
 
     // Process the create region request
     if (parentRegionName == null || regionName == null) {
       String errMessage = "";
       if (parentRegionName == null) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.CreateRegion_0_THE_INPUT_PARENT_REGION_NAME_FOR_THE_CREATE_REGION_REQUEST_IS_NULL, servConn.getName()));
+        logger.warn(LocalizedMessage.create(LocalizedStrings.CreateRegion_0_THE_INPUT_PARENT_REGION_NAME_FOR_THE_CREATE_REGION_REQUEST_IS_NULL, servConn
+          .getName()));
         errMessage = LocalizedStrings.CreateRegion_THE_INPUT_PARENT_REGION_NAME_FOR_THE_CREATE_REGION_REQUEST_IS_NULL.toLocalizedString();
       }
       if (regionName == null) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.CreateRegion_0_THE_INPUT_REGION_NAME_FOR_THE_CREATE_REGION_REQUEST_IS_NULL, servConn.getName())); 
+        logger.warn(LocalizedMessage.create(LocalizedStrings.CreateRegion_0_THE_INPUT_REGION_NAME_FOR_THE_CREATE_REGION_REQUEST_IS_NULL, servConn
+          .getName()));
         errMessage = LocalizedStrings.CreateRegion_THE_INPUT_REGION_NAME_FOR_THE_CREATE_REGION_REQUEST_IS_NULL.toLocalizedString();
       }
-      writeErrorResponse(msg, MessageType.CREATE_REGION_DATA_ERROR, errMessage,
-          servConn);
+      writeErrorResponse(msg, MessageType.CREATE_REGION_DATA_ERROR, errMessage, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      Region parentRegion = crHelper.getRegion(parentRegionName);
-      if (parentRegion == null) {
-        String reason = LocalizedStrings.CreateRegion__0_WAS_NOT_FOUND_DURING_SUBREGION_CREATION_REQUEST.toLocalizedString(parentRegionName);
-        writeRegionDestroyedEx(msg, parentRegionName, reason, servConn);
+
+    Region parentRegion = crHelper.getRegion(parentRegionName);
+    if (parentRegion == null) {
+      String reason = LocalizedStrings.CreateRegion__0_WAS_NOT_FOUND_DURING_SUBREGION_CREATION_REQUEST.toLocalizedString(parentRegionName);
+      writeRegionDestroyedEx(msg, parentRegionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    GeodeSecurityUtil.authorizeDataManage();
+
+    AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+    if (authzRequest != null) {
+      try {
+        authzRequest.createRegionAuthorize(parentRegionName + '/' + regionName);
+      } catch (NotAuthorizedException ex) {
+        writeException(msg, ex, false, servConn);
         servConn.setAsTrue(RESPONDED);
+        return;
       }
-      else {
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        if (authzRequest != null) {
-          try {
-            authzRequest.createRegionAuthorize(parentRegionName + '/'
-                + regionName);
-          }
-          catch (NotAuthorizedException ex) {
-            writeException(msg, ex, false, servConn);
-            servConn.setAsTrue(RESPONDED);
-            return;
-          }
-        }
-        // Create or get the subregion
-        Region region = parentRegion.getSubregion(regionName);
-        if (region == null) {
-          AttributesFactory factory = new AttributesFactory(parentRegion
-              .getAttributes());
-          region = parentRegion.createSubregion(regionName, factory.create());
-          if (logger.isDebugEnabled()) {
-            logger.debug("{}: Created region {}", servConn.getName(), region);
-          }
-        }
-        else {
-          if (logger.isDebugEnabled()) {
-            logger.debug("{}: Retrieved region {}", servConn.getName(), region);
-          }
-        }
-
-        // Update the statistics and write the reply
-        // start = DistributionStats.getStatTime(); WHY ARE WE GETTING START AND
-        // NOT USING IT
-        // bserverStats.incLong(processDestroyTimeId,
-        // DistributionStats.getStatTime() - start);
-        writeReply(msg, servConn);
-        servConn.setAsTrue(RESPONDED);
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Sent create region response for parent region {} region {}", servConn.getName(), parentRegionName, regionName);
-        }
-        // bserverStats.incLong(writeDestroyResponseTimeId,
-        // DistributionStats.getStatTime() - start);
-        // bserverStats.incInt(destroyResponsesId, 1);
+    }
+    // Create or get the subregion
+    Region region = parentRegion.getSubregion(regionName);
+    if (region == null) {
+      AttributesFactory factory = new AttributesFactory(parentRegion.getAttributes());
+      region = parentRegion.createSubregion(regionName, factory.create());
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: Created region {}", servConn.getName(), region);
       }
+    } else {
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: Retrieved region {}", servConn.getName(), region);
+      }
+    }
+
+    // Update the statistics and write the reply
+    // start = DistributionStats.getStatTime(); WHY ARE WE GETTING START AND
+    // NOT USING IT
+    // bserverStats.incLong(processDestroyTimeId,
+    // DistributionStats.getStatTime() - start);
+    writeReply(msg, servConn);
+    servConn.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Sent create region response for parent region {} region {}", servConn.getName(), parentRegionName, regionName);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ae6110c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/DestroyRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/DestroyRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/DestroyRegion.java
index 9271b99..0622cc2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/DestroyRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/DestroyRegion.java
@@ -15,25 +15,30 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import com.gemstone.gemfire.cache.operations.RegionDestroyOperationContext;
+import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
-import com.gemstone.gemfire.cache.operations.RegionDestroyOperationContext;
-import com.gemstone.gemfire.distributed.DistributedSystemDisconnectedException;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 public class DestroyRegion extends BaseCommand {
 
@@ -47,8 +52,7 @@ public class DestroyRegion extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException, InterruptedException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException, InterruptedException {
     Part regionNamePart = null, callbackArgPart = null;
     String regionName = null;
     Object callbackArg = null;
@@ -66,22 +70,21 @@ public class DestroyRegion extends BaseCommand {
     // Retrieve the data from the message parts
     regionNamePart = msg.getPart(0);
     eventPart = msg.getPart(1);
-//    callbackArgPart = null; (redundant assignment)
+    //    callbackArgPart = null; (redundant assignment)
     if (msg.getNumberOfParts() > 2) {
       callbackArgPart = msg.getPart(2);
       try {
         callbackArg = callbackArgPart.getObject();
-      }
-      catch (DistributedSystemDisconnectedException se) {
+      } catch (DistributedSystemDisconnectedException se) {
         // FIXME this can't happen
         if (logger.isDebugEnabled()) {
-          logger.debug("{} ignoring message of type {} from client {} because shutdown occurred during message processing.", servConn.getName(), MessageType.getString(msg.getMessageType()), servConn.getProxyID());
+          logger.debug("{} ignoring message of type {} from client {} because shutdown occurred during message processing.", servConn
+            .getName(), MessageType.getString(msg.getMessageType()), servConn.getProxyID());
         }
 
         servConn.setFlagProcessMessagesAsFalse();
         return;
-      }
-      catch (Exception e) {
+      } catch (Exception e) {
         writeException(msg, e, false, servConn);
         servConn.setAsTrue(RESPONDED);
         return;
@@ -89,87 +92,81 @@ public class DestroyRegion extends BaseCommand {
     }
     regionName = regionNamePart.getString();
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received destroy region request ({} bytes) from {} for region {}", servConn.getName(), msg.getPayloadLength(), servConn.getSocketString(), regionName);
+      logger.debug("{}: Received destroy region request ({} bytes) from {} for region {}", servConn.getName(), msg.getPayloadLength(), servConn
+        .getSocketString(), regionName);
     }
 
     // Process the destroy region request
     if (regionName == null) {
-      logger.warn(LocalizedMessage.create(LocalizedStrings.DestroyRegion_0_THE_INPUT_REGION_NAME_FOR_THE_DESTROY_REGION_REQUEST_IS_NULL, servConn.getName()));
-      errMessage
-          .append(LocalizedStrings.DestroyRegion__THE_INPUT_REGION_NAME_FOR_THE_DESTROY_REGION_REQUEST_IS_NULL.toLocalizedString());
+      logger.warn(LocalizedMessage.create(LocalizedStrings.DestroyRegion_0_THE_INPUT_REGION_NAME_FOR_THE_DESTROY_REGION_REQUEST_IS_NULL, servConn
+        .getName()));
+      errMessage.append(LocalizedStrings.DestroyRegion__THE_INPUT_REGION_NAME_FOR_THE_DESTROY_REGION_REQUEST_IS_NULL.toLocalizedString());
 
-      writeErrorResponse(msg, MessageType.DESTROY_REGION_DATA_ERROR, errMessage
-          .toString(), servConn);
+      writeErrorResponse(msg, MessageType.DESTROY_REGION_DATA_ERROR, errMessage.toString(), servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = LocalizedStrings.DestroyRegion_REGION_WAS_NOT_FOUND_DURING_DESTROY_REGION_REQUEST.toLocalizedString();
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      }
-      else {
-        // Destroy the region
-        ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart
-            .getSerializedForm());
-        long threadId = EventID
-            .readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-        long sequenceId = EventID
-            .readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-        EventID eventId = new EventID(servConn.getEventMemberIDByteArray(),
-            threadId, sequenceId);
-
-        try {
-          AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-          if (authzRequest != null) {
-            RegionDestroyOperationContext destroyContext = authzRequest
-                .destroyRegionAuthorize(regionName, callbackArg);
-            callbackArg = destroyContext.getCallbackArg();
-          }
-          // region.destroyRegion(callbackArg);
-          region.basicBridgeDestroyRegion(callbackArg, servConn.getProxyID(),
-              true /* boolean from cache Client */, eventId);
-        }
-        catch (DistributedSystemDisconnectedException e) {
-          // FIXME better exception hierarchy would avoid this check
-          if (servConn.getCachedRegionHelper().getCache().getCancelCriterion().cancelInProgress() != null) {
-            if (logger.isDebugEnabled()) {
-              logger.debug("{} ignoring message of type {} from client {} because shutdown occurred during message processing.", servConn.getName(), MessageType.getString(msg.getMessageType()), servConn.getProxyID());
-            }
-            servConn.setFlagProcessMessagesAsFalse();
-          }
-          else {
-            writeException(msg, e, false, servConn);
-            servConn.setAsTrue(RESPONDED);
-          }
-          return;
-        }
-        catch (Exception e) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, e);
-
-          // Otherwise, write an exception message and continue
-          writeException(msg, e, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
 
-        // Update the statistics and write the reply
-        {
-          long oldStart = start;
-          start = DistributionStats.getStatTime();
-          stats.incProcessDestroyRegionTime(start - oldStart);
-        }
-        writeReply(msg, servConn);
-        servConn.setAsTrue(RESPONDED);
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = LocalizedStrings.DestroyRegion_REGION_WAS_NOT_FOUND_DURING_DESTROY_REGION_REQUEST.toLocalizedString();
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    // user needs to have data:manage on all regions in order to destory a particular region
+    GeodeSecurityUtil.authorizeDataManage();
+
+    // Destroy the region
+    ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
+    long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    EventID eventId = new EventID(servConn.getEventMemberIDByteArray(), threadId, sequenceId);
+
+    try {
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      if (authzRequest != null) {
+        RegionDestroyOperationContext destroyContext = authzRequest.destroyRegionAuthorize(regionName, callbackArg);
+        callbackArg = destroyContext.getCallbackArg();
+      }
+      // region.destroyRegion(callbackArg);
+      region.basicBridgeDestroyRegion(callbackArg, servConn.getProxyID(), true /* boolean from cache Client */, eventId);
+    } catch (DistributedSystemDisconnectedException e) {
+      // FIXME better exception hierarchy would avoid this check
+      if (servConn.getCachedRegionHelper().getCache().getCancelCriterion().cancelInProgress() != null) {
         if (logger.isDebugEnabled()) {
-          logger.debug("{}: Sent destroy region response for region {}", servConn.getName(), regionName);
+          logger.debug("{} ignoring message of type {} from client {} because shutdown occurred during message processing.", servConn
+            .getName(), MessageType.getString(msg.getMessageType()), servConn.getProxyID());
         }
-        stats.incWriteDestroyRegionResponseTime(DistributionStats.getStatTime()
-            - start);
+        servConn.setFlagProcessMessagesAsFalse();
+      } else {
+        writeException(msg, e, false, servConn);
+        servConn.setAsTrue(RESPONDED);
       }
+      return;
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
+
+      // Otherwise, write an exception message and continue
+      writeException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    // Update the statistics and write the reply
+    {
+      long oldStart = start;
+      start = DistributionStats.getStatTime();
+      stats.incProcessDestroyRegionTime(start - oldStart);
+    }
+    writeReply(msg, servConn);
+    servConn.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Sent destroy region response for region {}", servConn.getName(), regionName);
     }
+    stats.incWriteDestroyRegionResponseTime(DistributionStats.getStatTime() - start);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ae6110c/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
index 212c9c9..4b7b573 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
@@ -35,6 +35,8 @@ import org.junit.experimental.categories.Category;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.client.Pool;
+import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.management.internal.security.JSONAuthorization;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.dunit.Host;
@@ -116,11 +118,6 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
 
       Set keySet = region.keySet();
       assertEquals(0, keySet.size());
-
-//      Query query = cache.getQueryService().newQuery("select * from /AuthRegion");
-//      Object result = query.execute();
-
-      cache.close();
     });
 
 
@@ -173,6 +170,41 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
   }
 
   @Test
+  public void testQuery(){
+    client1.invoke(()-> {
+      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+
+      String query = "select * from /AuthRegion";
+      assertNotAuthorized(()->region.query(query), "DATA:READ:AuthRegion");
+
+      Pool pool = PoolManager.find(region);
+      assertNotAuthorized(()->pool.getQueryService().newQuery(query).execute(), "DATA:READ:AuthRegion");
+    });
+  }
+
+  @Test
+  public void testDestroyRegion() throws InterruptedException {
+    client1.invoke(()-> {
+      Cache cache = SecurityTestUtils.createCacheClient("dataWriter", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      assertNotAuthorized(()->region.destroyRegion(), "DATA:MANAGE");
+    });
+
+    client2.invoke(()-> {
+      Cache cache = SecurityTestUtils.createCacheClient("authRegionManager", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      assertNotAuthorized(()->region.destroyRegion(), "DATA:MANAGE");
+    });
+
+    client3.invoke(()-> {
+      Cache cache = SecurityTestUtils.createCacheClient("super-user", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      region.destroyRegion();
+    });
+  }
+
+  @Test
   public void testDestroyInvalidate() throws InterruptedException {
 
     // Delete one key and invalidate another key with an authorized user.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ae6110c/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
index 7fa2ccf..e5313d2 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
@@ -122,6 +122,13 @@
       ]
     },
     {
+      "name": "dataWriter",
+      "password": "1234567",
+      "roles": [
+        "data-write"
+      ]
+    },
+    {
       "name":"authRegionUser",
       "password": "1234567",
       "roles": [


[37/50] [abbrv] incubator-geode git commit: GEODE-1576: Website needs to be upgraded to include WAN and CQ

Posted by ji...@apache.org.
GEODE-1576: Website needs to be upgraded to include WAN and CQ

upgrading font-awesome to v4.6.3.


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

Branch: refs/heads/feature/GEODE-1571
Commit: 30928d923effd8aed8c376ac1cbaf4d8948c7a3a
Parents: 850e143
Author: Swapnil Bawaskar <sb...@pivotal.io>
Authored: Tue Jun 21 11:58:54 2016 -0700
Committer: Swapnil Bawaskar <sb...@pivotal.io>
Committed: Fri Jun 24 16:05:36 2016 -0700

----------------------------------------------------------------------
 .../website/content/css/font-awesome.min.css    | 409 +------------------
 geode-site/website/content/font/FontAwesome.otf | Bin 61896 -> 124988 bytes
 .../content/font/fontawesome-webfont-eot.eot    | Bin 37405 -> 76518 bytes
 .../content/font/fontawesome-webfont-svg.svg    | 366 +++++++++++++++--
 .../content/font/fontawesome-webfont-ttf.ttf    | Bin 79076 -> 152796 bytes
 .../content/font/fontawesome-webfont-woff.woff  | Bin 43572 -> 90412 bytes
 6 files changed, 330 insertions(+), 445 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/30928d92/geode-site/website/content/css/font-awesome.min.css
----------------------------------------------------------------------
diff --git a/geode-site/website/content/css/font-awesome.min.css b/geode-site/website/content/css/font-awesome.min.css
index 7b8ed50..3fdf08a 100644
--- a/geode-site/website/content/css/font-awesome.min.css
+++ b/geode-site/website/content/css/font-awesome.min.css
@@ -1,405 +1,4 @@
-@font-face{font-family:'FontAwesome';src:url('../font/fontawesome-webfont-eot.eot?v=3.2.1');src:url('../font/fontawesome-webfont-eot.eot?#iefix&v=3.2.1') format('embedded-opentype'),url('../font/fontawesome-webfont-woff.woff?v=3.2.1') format('woff'),url('../font/fontawesome-webfont-ttf.ttf?v=3.2.1') format('truetype'),url('../font/fontawesome-webfont-svg.svg#fontawesomeregular?v=3.2.1') format('svg');font-weight:normal;font-style:normal;}[class^="icon-"],[class*=" icon-"]{font-family:FontAwesome;font-weight:normal;font-style:normal;text-decoration:inherit;-webkit-font-smoothing:antialiased;*margin-right:.3em;}
-[class^="icon-"]:before,[class*=" icon-"]:before{text-decoration:inherit;display:inline-block;speak:none;}
-.icon-large:before{vertical-align:-10%;font-size:1.3333333333333333em;}
-a [class^="icon-"],a [class*=" icon-"]{display:inline;}
-[class^="icon-"].icon-fixed-width,[class*=" icon-"].icon-fixed-width{display:inline-block;width:1.1428571428571428em;text-align:right;padding-right:0.2857142857142857em;}[class^="icon-"].icon-fixed-width.icon-large,[class*=" icon-"].icon-fixed-width.icon-large{width:1.4285714285714286em;}
-.icons-ul{margin-left:2.142857142857143em;list-style-type:none;}.icons-ul>li{position:relative;}
-.icons-ul .icon-li{position:absolute;left:-2.142857142857143em;width:2.142857142857143em;text-align:center;line-height:inherit;}
-[class^="icon-"].hide,[class*=" icon-"].hide{display:none;}
-.icon-muted{color:#eeeeee;}
-.icon-light{color:#ffffff;}
-.icon-dark{color:#333333;}
-.icon-border{border:solid 1px #eeeeee;padding:.2em .25em .15em;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;}
-.icon-2x{font-size:2em;}.icon-2x.icon-border{border-width:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}
-.icon-3x{font-size:3em;}.icon-3x.icon-border{border-width:3px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px;}
-.icon-4x{font-size:4em;}.icon-4x.icon-border{border-width:4px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;}
-.icon-5x{font-size:5em;}.icon-5x.icon-border{border-width:5px;-webkit-border-radius:7px;-moz-border-radius:7px;border-radius:7px;}
-.pull-right{float:right;}
-.pull-left{float:left;}
-[class^="icon-"].pull-left,[class*=" icon-"].pull-left{margin-right:.3em;}
-[class^="icon-"].pull-right,[class*=" icon-"].pull-right{margin-left:.3em;}
-[class^="icon-"],[class*=" icon-"]{display:inline;width:auto;height:auto;line-height:normal;vertical-align:baseline;background-image:none;background-position:0% 0%;background-repeat:repeat;margin-top:0;}
-.icon-white,.nav-pills>.active>a>[class^="icon-"],.nav-pills>.active>a>[class*=" icon-"],.nav-list>.active>a>[class^="icon-"],.nav-list>.active>a>[class*=" icon-"],.navbar-inverse .nav>.active>a>[class^="icon-"],.navbar-inverse .nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"],.dropdown-submenu:hover>a>[class^="icon-"],.dropdown-submenu:hover>a>[class*=" icon-"]{background-image:none;}
-.btn [class^="icon-"].icon-large,.nav [class^="icon-"].icon-large,.btn [class*=" icon-"].icon-large,.nav [class*=" icon-"].icon-large{line-height:.9em;}
-.btn [class^="icon-"].icon-spin,.nav [class^="icon-"].icon-spin,.btn [class*=" icon-"].icon-spin,.nav [class*=" icon-"].icon-spin{display:inline-block;}
-.nav-tabs [class^="icon-"],.nav-pills [class^="icon-"],.nav-tabs [class*=" icon-"],.nav-pills [class*=" icon-"],.nav-tabs [class^="icon-"].icon-large,.nav-pills [class^="icon-"].icon-large,.nav-tabs [class*=" icon-"].icon-large,.nav-pills [class*=" icon-"].icon-large{line-height:.9em;}
-.btn [class^="icon-"].pull-left.icon-2x,.btn [class*=" icon-"].pull-left.icon-2x,.btn [class^="icon-"].pull-right.icon-2x,.btn [class*=" icon-"].pull-right.icon-2x{margin-top:.18em;}
-.btn [class^="icon-"].icon-spin.icon-large,.btn [class*=" icon-"].icon-spin.icon-large{line-height:.8em;}
-.btn.btn-small [class^="icon-"].pull-left.icon-2x,.btn.btn-small [class*=" icon-"].pull-left.icon-2x,.btn.btn-small [class^="icon-"].pull-right.icon-2x,.btn.btn-small [class*=" icon-"].pull-right.icon-2x{margin-top:.25em;}
-.btn.btn-large [class^="icon-"],.btn.btn-large [class*=" icon-"]{margin-top:0;}.btn.btn-large [class^="icon-"].pull-left.icon-2x,.btn.btn-large [class*=" icon-"].pull-left.icon-2x,.btn.btn-large [class^="icon-"].pull-right.icon-2x,.btn.btn-large [class*=" icon-"].pull-right.icon-2x{margin-top:.05em;}
-.btn.btn-large [class^="icon-"].pull-left.icon-2x,.btn.btn-large [class*=" icon-"].pull-left.icon-2x{margin-right:.2em;}
-.btn.btn-large [class^="icon-"].pull-right.icon-2x,.btn.btn-large [class*=" icon-"].pull-right.icon-2x{margin-left:.2em;}
-.nav-list [class^="icon-"],.nav-list [class*=" icon-"]{line-height:inherit;}
-.icon-stack{position:relative;display:inline-block;width:2em;height:2em;line-height:2em;vertical-align:-35%;}.icon-stack [class^="icon-"],.icon-stack [class*=" icon-"]{display:block;text-align:center;position:absolute;width:100%;height:100%;font-size:1em;line-height:inherit;*line-height:2em;}
-.icon-stack .icon-stack-base{font-size:2em;*line-height:1em;}
-.icon-spin{display:inline-block;-moz-animation:spin 2s infinite linear;-o-animation:spin 2s infinite linear;-webkit-animation:spin 2s infinite linear;animation:spin 2s infinite linear;}
-a .icon-stack,a .icon-spin{display:inline-block;text-decoration:none;}
-@-moz-keyframes spin{0%{-moz-transform:rotate(0deg);} 100%{-moz-transform:rotate(359deg);}}@-webkit-keyframes spin{0%{-webkit-transform:rotate(0deg);} 100%{-webkit-transform:rotate(359deg);}}@-o-keyframes spin{0%{-o-transform:rotate(0deg);} 100%{-o-transform:rotate(359deg);}}@-ms-keyframes spin{0%{-ms-transform:rotate(0deg);} 100%{-ms-transform:rotate(359deg);}}@keyframes spin{0%{transform:rotate(0deg);} 100%{transform:rotate(359deg);}}.icon-rotate-90:before{-webkit-transform:rotate(90deg);-moz-transform:rotate(90deg);-ms-transform:rotate(90deg);-o-transform:rotate(90deg);transform:rotate(90deg);filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=1);}
-.icon-rotate-180:before{-webkit-transform:rotate(180deg);-moz-transform:rotate(180deg);-ms-transform:rotate(180deg);-o-transform:rotate(180deg);transform:rotate(180deg);filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=2);}
-.icon-rotate-270:before{-webkit-transform:rotate(270deg);-moz-transform:rotate(270deg);-ms-transform:rotate(270deg);-o-transform:rotate(270deg);transform:rotate(270deg);filter:progid:DXImageTransform.Microsoft.BasicImage(rotation=3);}
-.icon-flip-horizontal:before{-webkit-transform:scale(-1, 1);-moz-transform:scale(-1, 1);-ms-transform:scale(-1, 1);-o-transform:scale(-1, 1);transform:scale(-1, 1);}
-.icon-flip-vertical:before{-webkit-transform:scale(1, -1);-moz-transform:scale(1, -1);-ms-transform:scale(1, -1);-o-transform:scale(1, -1);transform:scale(1, -1);}
-a .icon-rotate-90:before,a .icon-rotate-180:before,a .icon-rotate-270:before,a .icon-flip-horizontal:before,a .icon-flip-vertical:before{display:inline-block;}
-.icon-glass:before{content:"\f000";}
-.icon-music:before{content:"\f001";}
-.icon-search:before{content:"\f002";}
-.icon-envelope-alt:before{content:"\f003";}
-.icon-heart:before{content:"\f004";}
-.icon-star:before{content:"\f005";}
-.icon-star-empty:before{content:"\f006";}
-.icon-user:before{content:"\f007";}
-.icon-film:before{content:"\f008";}
-.icon-th-large:before{content:"\f009";}
-.icon-th:before{content:"\f00a";}
-.icon-th-list:before{content:"\f00b";}
-.icon-ok:before{content:"\f00c";}
-.icon-remove:before{content:"\f00d";}
-.icon-zoom-in:before{content:"\f00e";}
-.icon-zoom-out:before{content:"\f010";}
-.icon-power-off:before,.icon-off:before{content:"\f011";}
-.icon-signal:before{content:"\f012";}
-.icon-gear:before,.icon-cog:before{content:"\f013";}
-.icon-trash:before{content:"\f014";}
-.icon-home:before{content:"\f015";}
-.icon-file-alt:before{content:"\f016";}
-.icon-time:before{content:"\f017";}
-.icon-road:before{content:"\f018";}
-.icon-download-alt:before{content:"\f019";}
-.icon-download:before{content:"\f01a";}
-.icon-upload:before{content:"\f01b";}
-.icon-inbox:before{content:"\f01c";}
-.icon-play-circle:before{content:"\f01d";}
-.icon-rotate-right:before,.icon-repeat:before{content:"\f01e";}
-.icon-refresh:before{content:"\f021";}
-.icon-list-alt:before{content:"\f022";}
-.icon-lock:before{content:"\f023";}
-.icon-flag:before{content:"\f024";}
-.icon-headphones:before{content:"\f025";}
-.icon-volume-off:before{content:"\f026";}
-.icon-volume-down:before{content:"\f027";}
-.icon-volume-up:before{content:"\f028";}
-.icon-qrcode:before{content:"\f029";}
-.icon-barcode:before{content:"\f02a";}
-.icon-tag:before{content:"\f02b";}
-.icon-tags:before{content:"\f02c";}
-.icon-book:before{content:"\f02d";}
-.icon-bookmark:before{content:"\f02e";}
-.icon-print:before{content:"\f02f";}
-.icon-camera:before{content:"\f030";}
-.icon-font:before{content:"\f031";}
-.icon-bold:before{content:"\f032";}
-.icon-italic:before{content:"\f033";}
-.icon-text-height:before{content:"\f034";}
-.icon-text-width:before{content:"\f035";}
-.icon-align-left:before{content:"\f036";}
-.icon-align-center:before{content:"\f037";}
-.icon-align-right:before{content:"\f038";}
-.icon-align-justify:before{content:"\f039";}
-.icon-list:before{content:"\f03a";}
-.icon-indent-left:before{content:"\f03b";}
-.icon-indent-right:before{content:"\f03c";}
-.icon-facetime-video:before{content:"\f03d";}
-.icon-picture:before{content:"\f03e";}
-.icon-pencil:before{content:"\f040";}
-.icon-map-marker:before{content:"\f041";}
-.icon-adjust:before{content:"\f042";}
-.icon-tint:before{content:"\f043";}
-.icon-edit:before{content:"\f044";}
-.icon-share:before{content:"\f045";}
-.icon-check:before{content:"\f046";}
-.icon-move:before{content:"\f047";}
-.icon-step-backward:before{content:"\f048";}
-.icon-fast-backward:before{content:"\f049";}
-.icon-backward:before{content:"\f04a";}
-.icon-play:before{content:"\f04b";}
-.icon-pause:before{content:"\f04c";}
-.icon-stop:before{content:"\f04d";}
-.icon-forward:before{content:"\f04e";}
-.icon-fast-forward:before{content:"\f050";}
-.icon-step-forward:before{content:"\f051";}
-.icon-eject:before{content:"\f052";}
-.icon-chevron-left:before{content:"\f053";}
-.icon-chevron-right:before{content:"\f054";}
-.icon-plus-sign:before{content:"\f055";}
-.icon-minus-sign:before{content:"\f056";}
-.icon-remove-sign:before{content:"\f057";}
-.icon-ok-sign:before{content:"\f058";}
-.icon-question-sign:before{content:"\f059";}
-.icon-info-sign:before{content:"\f05a";}
-.icon-screenshot:before{content:"\f05b";}
-.icon-remove-circle:before{content:"\f05c";}
-.icon-ok-circle:before{content:"\f05d";}
-.icon-ban-circle:before{content:"\f05e";}
-.icon-arrow-left:before{content:"\f060";}
-.icon-arrow-right:before{content:"\f061";}
-.icon-arrow-up:before{content:"\f062";}
-.icon-arrow-down:before{content:"\f063";}
-.icon-mail-forward:before,.icon-share-alt:before{content:"\f064";}
-.icon-resize-full:before{content:"\f065";}
-.icon-resize-small:before{content:"\f066";}
-.icon-plus:before{content:"\f067";}
-.icon-minus:before{content:"\f068";}
-.icon-asterisk:before{content:"\f069";}
-.icon-exclamation-sign:before{content:"\f06a";}
-.icon-gift:before{content:"\f06b";}
-.icon-leaf:before{content:"\f06c";}
-.icon-fire:before{content:"\f06d";}
-.icon-eye-open:before{content:"\f06e";}
-.icon-eye-close:before{content:"\f070";}
-.icon-warning-sign:before{content:"\f071";}
-.icon-plane:before{content:"\f072";}
-.icon-calendar:before{content:"\f073";}
-.icon-random:before{content:"\f074";}
-.icon-comment:before{content:"\f075";}
-.icon-magnet:before{content:"\f076";}
-.icon-chevron-up:before{content:"\f077";}
-.icon-chevron-down:before{content:"\f078";}
-.icon-retweet:before{content:"\f079";}
-.icon-shopping-cart:before{content:"\f07a";}
-.icon-folder-close:before{content:"\f07b";}
-.icon-folder-open:before{content:"\f07c";}
-.icon-resize-vertical:before{content:"\f07d";}
-.icon-resize-horizontal:before{content:"\f07e";}
-.icon-bar-chart:before{content:"\f080";}
-.icon-twitter-sign:before{content:"\f081";}
-.icon-facebook-sign:before{content:"\f082";}
-.icon-camera-retro:before{content:"\f083";}
-.icon-key:before{content:"\f084";}
-.icon-gears:before,.icon-cogs:before{content:"\f085";}
-.icon-comments:before{content:"\f086";}
-.icon-thumbs-up-alt:before{content:"\f087";}
-.icon-thumbs-down-alt:before{content:"\f088";}
-.icon-star-half:before{content:"\f089";}
-.icon-heart-empty:before{content:"\f08a";}
-.icon-signout:before{content:"\f08b";}
-.icon-linkedin-sign:before{content:"\f08c";}
-.icon-pushpin:before{content:"\f08d";}
-.icon-external-link:before{content:"\f08e";}
-.icon-signin:before{content:"\f090";}
-.icon-trophy:before{content:"\f091";}
-.icon-github-sign:before{content:"\f092";}
-.icon-upload-alt:before{content:"\f093";}
-.icon-lemon:before{content:"\f094";}
-.icon-phone:before{content:"\f095";}
-.icon-unchecked:before,.icon-check-empty:before{content:"\f096";}
-.icon-bookmark-empty:before{content:"\f097";}
-.icon-phone-sign:before{content:"\f098";}
-.icon-twitter:before{content:"\f099";}
-.icon-facebook:before{content:"\f09a";}
-.icon-github:before{content:"\f09b";}
-.icon-wiki:before{content:"\f109";}
-.icon-releases:before{content:"\f01a";}
-.icon-unlock:before{content:"\f09c";}
-.icon-credit-card:before{content:"\f09d";}
-.icon-rss:before{content:"\f09e";}
-.icon-hdd:before{content:"\f0a0";}
-.icon-bullhorn:before{content:"\f0a1";}
-.icon-bell:before{content:"\f0a2";}
-.icon-certificate:before{content:"\f0a3";}
-.icon-hand-right:before{content:"\f0a4";}
-.icon-hand-left:before{content:"\f0a5";}
-.icon-hand-up:before{content:"\f0a6";}
-.icon-hand-down:before{content:"\f0a7";}
-.icon-circle-arrow-left:before{content:"\f0a8";}
-.icon-circle-arrow-right:before{content:"\f0a9";}
-.icon-circle-arrow-up:before{content:"\f0aa";}
-.icon-circle-arrow-down:before{content:"\f0ab";}
-.icon-globe:before{content:"\f0ac";}
-.icon-wrench:before{content:"\f0ad";}
-.icon-tasks:before{content:"\f0ae";}
-.icon-filter:before{content:"\f0b0";}
-.icon-briefcase:before{content:"\f0b1";}
-.icon-fullscreen:before{content:"\f0b2";}
-.icon-group:before{content:"\f0c0";}
-.icon-link:before{content:"\f0c1";}
-.icon-cloud:before{content:"\f0c2";}
-.icon-beaker:before{content:"\f0c3";}
-.icon-cut:before{content:"\f0c4";}
-.icon-copy:before{content:"\f0c5";}
-.icon-paperclip:before,.icon-paper-clip:before{content:"\f0c6";}
-.icon-save:before{content:"\f0c7";}
-.icon-sign-blank:before{content:"\f0c8";}
-.icon-reorder:before{content:"\f0c9";}
-.icon-list-ul:before{content:"\f0ca";}
-.icon-list-ol:before{content:"\f0cb";}
-.icon-strikethrough:before{content:"\f0cc";}
-.icon-underline:before{content:"\f0cd";}
-.icon-table:before{content:"\f0ce";}
-.icon-magic:before{content:"\f0d0";}
-.icon-truck:before{content:"\f0d1";}
-.icon-pinterest:before{content:"\f0d2";}
-.icon-pinterest-sign:before{content:"\f0d3";}
-.icon-google-plus-sign:before{content:"\f0d4";}
-.icon-google-plus:before{content:"\f0d5";}
-.icon-money:before{content:"\f0d6";}
-.icon-caret-down:before{content:"\f0d7";}
-.icon-caret-up:before{content:"\f0d8";}
-.icon-caret-left:before{content:"\f0d9";}
-.icon-caret-right:before{content:"\f0da";}
-.icon-columns:before{content:"\f0db";}
-.icon-sort:before{content:"\f0dc";}
-.icon-sort-down:before{content:"\f0dd";}
-.icon-sort-up:before{content:"\f0de";}
-.icon-envelope:before{content:"\f0e0";}
-.icon-linkedin:before{content:"\f0e1";}
-.icon-rotate-left:before,.icon-undo:before{content:"\f0e2";}
-.icon-legal:before{content:"\f0e3";}
-.icon-dashboard:before{content:"\f0e4";}
-.icon-comment-alt:before{content:"\f0e5";}
-.icon-comments-alt:before{content:"\f0e6";}
-.icon-bolt:before{content:"\f0e7";}
-.icon-sitemap:before{content:"\f0e8";}
-.icon-umbrella:before{content:"\f0e9";}
-.icon-paste:before{content:"\f0ea";}
-.icon-lightbulb:before{content:"\f0eb";}
-.icon-exchange:before{content:"\f0ec";}
-.icon-cloud-download:before{content:"\f0ed";}
-.icon-cloud-upload:before{content:"\f0ee";}
-.icon-user-md:before{content:"\f0f0";}
-.icon-stethoscope:before{content:"\f0f1";}
-.icon-suitcase:before{content:"\f0f2";}
-.icon-bell-alt:before{content:"\f0f3";}
-.icon-coffee:before{content:"\f0f4";}
-.icon-food:before{content:"\f0f5";}
-.icon-file-text-alt:before{content:"\f0f6";}
-.icon-building:before{content:"\f0f7";}
-.icon-hospital:before{content:"\f0f8";}
-.icon-ambulance:before{content:"\f0f9";}
-.icon-medkit:before{content:"\f0fa";}
-.icon-fighter-jet:before{content:"\f0fb";}
-.icon-beer:before{content:"\f0fc";}
-.icon-h-sign:before{content:"\f0fd";}
-.icon-plus-sign-alt:before{content:"\f0fe";}
-.icon-double-angle-left:before{content:"\f100";}
-.icon-double-angle-right:before{content:"\f101";}
-.icon-double-angle-up:before{content:"\f102";}
-.icon-double-angle-down:before{content:"\f103";}
-.icon-angle-left:before{content:"\f104";}
-.icon-angle-right:before{content:"\f105";}
-.icon-angle-up:before{content:"\f106";}
-.icon-angle-down:before{content:"\f107";}
-.icon-desktop:before{content:"\f108";}
-.icon-laptop:before{content:"\f109";}
-.icon-tablet:before{content:"\f10a";}
-.icon-mobile-phone:before{content:"\f10b";}
-.icon-circle-blank:before{content:"\f10c";}
-.icon-quote-left:before{content:"\f10d";}
-.icon-quote-right:before{content:"\f10e";}
-.icon-spinner:before{content:"\f110";}
-.icon-circle:before{content:"\f111";}
-.icon-mail-reply:before,.icon-reply:before{content:"\f112";}
-.icon-github-alt:before{content:"\f113";}
-.icon-folder-close-alt:before{content:"\f114";}
-.icon-folder-open-alt:before{content:"\f115";}
-.icon-expand-alt:before{content:"\f116";}
-.icon-collapse-alt:before{content:"\f117";}
-.icon-smile:before{content:"\f118";}
-.icon-frown:before{content:"\f119";}
-.icon-meh:before{content:"\f11a";}
-.icon-gamepad:before{content:"\f11b";}
-.icon-keyboard:before{content:"\f11c";}
-.icon-flag-alt:before{content:"\f11d";}
-.icon-flag-checkered:before{content:"\f11e";}
-.icon-terminal:before{content:"\f120";}
-.icon-code:before{content:"\f121";}
-.icon-reply-all:before{content:"\f122";}
-.icon-mail-reply-all:before{content:"\f122";}
-.icon-star-half-full:before,.icon-star-half-empty:before{content:"\f123";}
-.icon-location-arrow:before{content:"\f124";}
-.icon-crop:before{content:"\f125";}
-.icon-code-fork:before{content:"\f126";}
-.icon-unlink:before{content:"\f127";}
-.icon-question:before{content:"\f128";}
-.icon-info:before{content:"\f129";}
-.icon-exclamation:before{content:"\f12a";}
-.icon-superscript:before{content:"\f12b";}
-.icon-subscript:before{content:"\f12c";}
-.icon-eraser:before{content:"\f12d";}
-.icon-puzzle-piece:before{content:"\f12e";}
-.icon-microphone:before{content:"\f130";}
-.icon-microphone-off:before{content:"\f131";}
-.icon-shield:before{content:"\f132";}
-.icon-calendar-empty:before{content:"\f133";}
-.icon-fire-extinguisher:before{content:"\f134";}
-.icon-rocket:before{content:"\f135";}
-.icon-maxcdn:before{content:"\f136";}
-.icon-chevron-sign-left:before{content:"\f137";}
-.icon-chevron-sign-right:before{content:"\f138";}
-.icon-chevron-sign-up:before{content:"\f139";}
-.icon-chevron-sign-down:before{content:"\f13a";}
-.icon-html5:before{content:"\f13b";}
-.icon-css3:before{content:"\f13c";}
-.icon-anchor:before{content:"\f13d";}
-.icon-unlock-alt:before{content:"\f13e";}
-.icon-bullseye:before{content:"\f140";}
-.icon-ellipsis-horizontal:before{content:"\f141";}
-.icon-ellipsis-vertical:before{content:"\f142";}
-.icon-rss-sign:before{content:"\f143";}
-.icon-play-sign:before{content:"\f144";}
-.icon-ticket:before{content:"\f145";}
-.icon-minus-sign-alt:before{content:"\f146";}
-.icon-check-minus:before{content:"\f147";}
-.icon-level-up:before{content:"\f148";}
-.icon-level-down:before{content:"\f149";}
-.icon-check-sign:before{content:"\f14a";}
-.icon-edit-sign:before{content:"\f14b";}
-.icon-external-link-sign:before{content:"\f14c";}
-.icon-share-sign:before{content:"\f14d";}
-.icon-compass:before{content:"\f14e";}
-.icon-collapse:before{content:"\f150";}
-.icon-collapse-top:before{content:"\f151";}
-.icon-expand:before{content:"\f152";}
-.icon-euro:before,.icon-eur:before{content:"\f153";}
-.icon-gbp:before{content:"\f154";}
-.icon-dollar:before,.icon-usd:before{content:"\f155";}
-.icon-rupee:before,.icon-inr:before{content:"\f156";}
-.icon-yen:before,.icon-jpy:before{content:"\f157";}
-.icon-renminbi:before,.icon-cny:before{content:"\f158";}
-.icon-won:before,.icon-krw:before{content:"\f159";}
-.icon-bitcoin:before,.icon-btc:before{content:"\f15a";}
-.icon-file:before{content:"\f15b";}
-.icon-file-text:before{content:"\f15c";}
-.icon-sort-by-alphabet:before{content:"\f15d";}
-.icon-sort-by-alphabet-alt:before{content:"\f15e";}
-.icon-sort-by-attributes:before{content:"\f160";}
-.icon-sort-by-attributes-alt:before{content:"\f161";}
-.icon-sort-by-order:before{content:"\f162";}
-.icon-sort-by-order-alt:before{content:"\f163";}
-.icon-thumbs-up:before{content:"\f164";}
-.icon-thumbs-down:before{content:"\f165";}
-.icon-youtube-sign:before{content:"\f166";}
-.icon-youtube:before{content:"\f167";}
-.icon-xing:before{content:"\f168";}
-.icon-xing-sign:before{content:"\f169";}
-.icon-youtube-play:before{content:"\f16a";}
-.icon-dropbox:before{content:"\f16b";}
-.icon-stackexchange:before{content:"\f16c";}
-.icon-instagram:before{content:"\f16d";}
-.icon-flickr:before{content:"\f16e";}
-.icon-adn:before{content:"\f170";}
-.icon-bitbucket:before{content:"\f171";}
-.icon-bitbucket-sign:before{content:"\f172";}
-.icon-tumblr:before{content:"\f173";}
-.icon-tumblr-sign:before{content:"\f174";}
-.icon-long-arrow-down:before{content:"\f175";}
-.icon-long-arrow-up:before{content:"\f176";}
-.icon-long-arrow-left:before{content:"\f177";}
-.icon-long-arrow-right:before{content:"\f178";}
-.icon-apple:before{content:"\f179";}
-.icon-windows:before{content:"\f17a";}
-.icon-android:before{content:"\f17b";}
-.icon-linux:before{content:"\f17c";}
-.icon-dribbble:before{content:"\f17d";}
-.icon-skype:before{content:"\f17e";}
-.icon-foursquare:before{content:"\f180";}
-.icon-trello:before{content:"\f181";}
-.icon-female:before{content:"\f182";}
-.icon-male:before{content:"\f183";}
-.icon-gittip:before{content:"\f184";}
-.icon-sun:before{content:"\f185";}
-.icon-moon:before{content:"\f186";}
-.icon-archive:before{content:"\f187";}
-.icon-bug:before{content:"\f188";}
-.icon-vk:before{content:"\f189";}
-.icon-weibo:before{content:"\f18a";}
-.icon-renren:before{content:"\f18b";}
+/*!
+ *  Font Awesome 4.6.3 by @davegandy - http://fontawesome.io - @fontawesome
+ *  License - http://fontawesome.io/license (Font: SIL OFL 1.1, CSS: MIT License)
+ */@font-face{font-family:'FontAwesome';src:url('../font/fontawesome-webfont-eot.eot?v=4.6.3');src:url('../font/fontawesome-webfont-eot.eot?#iefix&v=4.6.3') format('embedded-opentype'),url('../font/fontawesome-webfont.woff2?v=4.6.3') format('woff2'),url('../font/fontawesome-webfont-woff.woff?v=4.6.3') format('woff'),url('../font/fontawesome-webfont-ttf.ttf?v=4.6.3') format('truetype'),url('../font/fontawesome-webfont-svg.svg?v=4.6.3#fontawesomeregular') format('svg');font-weight:normal;font-style:normal}.fa{display:inline-block;font:normal normal normal 14px/1 FontAwesome;font-size:inherit;text-rendering:auto;-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale}.fa-lg{font-size:1.33333333em;line-height:.75em;vertical-align:-15%}.fa-2x{font-size:2em}.fa-3x{font-size:3em}.fa-4x{font-size:4em}.fa-5x{font-size:5em}.fa-fw{width:1.28571429em;text-align:center}.fa-ul{padding-left:0;margin-left:2.14285714em;list-style-type:none}.fa-ul>li{position:relative}.fa-li{position:abs
 olute;left:-2.14285714em;width:2.14285714em;top:.14285714em;text-align:center}.fa-li.fa-lg{left:-1.85714286em}.fa-border{padding:.2em .25em .15em;border:solid .08em #eee;border-radius:.1em}.fa-pull-left{float:left}.fa-pull-right{float:right}.fa.fa-pull-left{margin-right:.3em}.fa.fa-pull-right{margin-left:.3em}.pull-right{float:right}.pull-left{float:left}.fa.pull-left{margin-right:.3em}.fa.pull-right{margin-left:.3em}.fa-spin{-webkit-animation:fa-spin 2s infinite linear;animation:fa-spin 2s infinite linear}.fa-pulse{-webkit-animation:fa-spin 1s infinite steps(8);animation:fa-spin 1s infinite steps(8)}@-webkit-keyframes fa-spin{0%{-webkit-transform:rotate(0deg);transform:rotate(0deg)}100%{-webkit-transform:rotate(359deg);transform:rotate(359deg)}}@keyframes fa-spin{0%{-webkit-transform:rotate(0deg);transform:rotate(0deg)}100%{-webkit-transform:rotate(359deg);transform:rotate(359deg)}}.fa-rotate-90{-ms-filter:"progid:DXImageTransform.Microsoft.BasicImage(rotation=1)";-webkit-transform
 :rotate(90deg);-ms-transform:rotate(90deg);transform:rotate(90deg)}.fa-rotate-180{-ms-filter:"progid:DXImageTransform.Microsoft.BasicImage(rotation=2)";-webkit-transform:rotate(180deg);-ms-transform:rotate(180deg);transform:rotate(180deg)}.fa-rotate-270{-ms-filter:"progid:DXImageTransform.Microsoft.BasicImage(rotation=3)";-webkit-transform:rotate(270deg);-ms-transform:rotate(270deg);transform:rotate(270deg)}.fa-flip-horizontal{-ms-filter:"progid:DXImageTransform.Microsoft.BasicImage(rotation=0, mirror=1)";-webkit-transform:scale(-1, 1);-ms-transform:scale(-1, 1);transform:scale(-1, 1)}.fa-flip-vertical{-ms-filter:"progid:DXImageTransform.Microsoft.BasicImage(rotation=2, mirror=1)";-webkit-transform:scale(1, -1);-ms-transform:scale(1, -1);transform:scale(1, -1)}:root .fa-rotate-90,:root .fa-rotate-180,:root .fa-rotate-270,:root .fa-flip-horizontal,:root .fa-flip-vertical{filter:none}.fa-stack{position:relative;display:inline-block;width:2em;height:2em;line-height:2em;vertical-align:m
 iddle}.fa-stack-1x,.fa-stack-2x{position:absolute;left:0;width:100%;text-align:center}.fa-stack-1x{line-height:inherit}.fa-stack-2x{font-size:2em}.fa-inverse{color:#fff}.fa-glass:before{content:"\f000"}.fa-music:before{content:"\f001"}.fa-search:before{content:"\f002"}.fa-envelope-o:before{content:"\f003"}.fa-heart:before{content:"\f004"}.fa-star:before{content:"\f005"}.fa-star-o:before{content:"\f006"}.fa-user:before{content:"\f007"}.fa-film:before{content:"\f008"}.fa-th-large:before{content:"\f009"}.fa-th:before{content:"\f00a"}.fa-th-list:before{content:"\f00b"}.fa-check:before{content:"\f00c"}.fa-remove:before,.fa-close:before,.fa-times:before{content:"\f00d"}.fa-search-plus:before{content:"\f00e"}.fa-search-minus:before{content:"\f010"}.fa-power-off:before{content:"\f011"}.fa-signal:before{content:"\f012"}.fa-gear:before,.fa-cog:before{content:"\f013"}.fa-trash-o:before{content:"\f014"}.fa-home:before{content:"\f015"}.fa-file-o:before{content:"\f016"}.fa-clock-o:before{content:
 "\f017"}.fa-road:before{content:"\f018"}.fa-download:before{content:"\f019"}.fa-arrow-circle-o-down:before{content:"\f01a"}.fa-arrow-circle-o-up:before{content:"\f01b"}.fa-inbox:before{content:"\f01c"}.fa-play-circle-o:before{content:"\f01d"}.fa-rotate-right:before,.fa-repeat:before{content:"\f01e"}.fa-refresh:before{content:"\f021"}.fa-list-alt:before{content:"\f022"}.fa-lock:before{content:"\f023"}.fa-flag:before{content:"\f024"}.fa-headphones:before{content:"\f025"}.fa-volume-off:before{content:"\f026"}.fa-volume-down:before{content:"\f027"}.fa-volume-up:before{content:"\f028"}.fa-qrcode:before{content:"\f029"}.fa-barcode:before{content:"\f02a"}.fa-tag:before{content:"\f02b"}.fa-tags:before{content:"\f02c"}.fa-book:before{content:"\f02d"}.fa-bookmark:before{content:"\f02e"}.fa-print:before{content:"\f02f"}.fa-camera:before{content:"\f030"}.fa-font:before{content:"\f031"}.fa-bold:before{content:"\f032"}.fa-italic:before{content:"\f033"}.fa-text-height:before{content:"\f034"}.fa-te
 xt-width:before{content:"\f035"}.fa-align-left:before{content:"\f036"}.fa-align-center:before{content:"\f037"}.fa-align-right:before{content:"\f038"}.fa-align-justify:before{content:"\f039"}.fa-list:before{content:"\f03a"}.fa-dedent:before,.fa-outdent:before{content:"\f03b"}.fa-indent:before{content:"\f03c"}.fa-video-camera:before{content:"\f03d"}.fa-photo:before,.fa-image:before,.fa-picture-o:before{content:"\f03e"}.fa-pencil:before{content:"\f040"}.fa-map-marker:before{content:"\f041"}.fa-adjust:before{content:"\f042"}.fa-tint:before{content:"\f043"}.fa-edit:before,.fa-pencil-square-o:before{content:"\f044"}.fa-share-square-o:before{content:"\f045"}.fa-check-square-o:before{content:"\f046"}.fa-arrows:before{content:"\f047"}.fa-step-backward:before{content:"\f048"}.fa-fast-backward:before{content:"\f049"}.fa-backward:before{content:"\f04a"}.fa-play:before{content:"\f04b"}.fa-pause:before{content:"\f04c"}.fa-stop:before{content:"\f04d"}.fa-forward:before{content:"\f04e"}.fa-fast-for
 ward:before{content:"\f050"}.fa-step-forward:before{content:"\f051"}.fa-eject:before{content:"\f052"}.fa-chevron-left:before{content:"\f053"}.fa-chevron-right:before{content:"\f054"}.fa-plus-circle:before{content:"\f055"}.fa-minus-circle:before{content:"\f056"}.fa-times-circle:before{content:"\f057"}.fa-check-circle:before{content:"\f058"}.fa-question-circle:before{content:"\f059"}.fa-info-circle:before{content:"\f05a"}.fa-crosshairs:before{content:"\f05b"}.fa-times-circle-o:before{content:"\f05c"}.fa-check-circle-o:before{content:"\f05d"}.fa-ban:before{content:"\f05e"}.fa-arrow-left:before{content:"\f060"}.fa-arrow-right:before{content:"\f061"}.fa-arrow-up:before{content:"\f062"}.fa-arrow-down:before{content:"\f063"}.fa-mail-forward:before,.fa-share:before{content:"\f064"}.fa-expand:before{content:"\f065"}.fa-compress:before{content:"\f066"}.fa-plus:before{content:"\f067"}.fa-minus:before{content:"\f068"}.fa-asterisk:before{content:"\f069"}.fa-exclamation-circle:before{content:"\f0
 6a"}.fa-gift:before{content:"\f06b"}.fa-leaf:before{content:"\f06c"}.fa-fire:before{content:"\f06d"}.fa-eye:before{content:"\f06e"}.fa-eye-slash:before{content:"\f070"}.fa-warning:before,.fa-exclamation-triangle:before{content:"\f071"}.fa-plane:before{content:"\f072"}.fa-calendar:before{content:"\f073"}.fa-random:before{content:"\f074"}.fa-comment:before{content:"\f075"}.fa-magnet:before{content:"\f076"}.fa-chevron-up:before{content:"\f077"}.fa-chevron-down:before{content:"\f078"}.fa-retweet:before{content:"\f079"}.fa-shopping-cart:before{content:"\f07a"}.fa-folder:before{content:"\f07b"}.fa-folder-open:before{content:"\f07c"}.fa-arrows-v:before{content:"\f07d"}.fa-arrows-h:before{content:"\f07e"}.fa-bar-chart-o:before,.fa-bar-chart:before{content:"\f080"}.fa-twitter-square:before{content:"\f081"}.fa-facebook-square:before{content:"\f082"}.fa-camera-retro:before{content:"\f083"}.fa-key:before{content:"\f084"}.fa-gears:before,.fa-cogs:before{content:"\f085"}.fa-comments:before{conten
 t:"\f086"}.fa-thumbs-o-up:before{content:"\f087"}.fa-thumbs-o-down:before{content:"\f088"}.fa-star-half:before{content:"\f089"}.fa-heart-o:before{content:"\f08a"}.fa-sign-out:before{content:"\f08b"}.fa-linkedin-square:before{content:"\f08c"}.fa-thumb-tack:before{content:"\f08d"}.fa-external-link:before{content:"\f08e"}.fa-sign-in:before{content:"\f090"}.fa-trophy:before{content:"\f091"}.fa-github-square:before{content:"\f092"}.fa-upload:before{content:"\f093"}.fa-lemon-o:before{content:"\f094"}.fa-phone:before{content:"\f095"}.fa-square-o:before{content:"\f096"}.fa-bookmark-o:before{content:"\f097"}.fa-phone-square:before{content:"\f098"}.fa-twitter:before{content:"\f099"}.fa-facebook-f:before,.fa-facebook:before{content:"\f09a"}.fa-github:before{content:"\f09b"}.fa-unlock:before{content:"\f09c"}.fa-credit-card:before{content:"\f09d"}.fa-feed:before,.fa-rss:before{content:"\f09e"}.fa-hdd-o:before{content:"\f0a0"}.fa-bullhorn:before{content:"\f0a1"}.fa-bell:before{content:"\f0f3"}.fa
 -certificate:before{content:"\f0a3"}.fa-hand-o-right:before{content:"\f0a4"}.fa-hand-o-left:before{content:"\f0a5"}.fa-hand-o-up:before{content:"\f0a6"}.fa-hand-o-down:before{content:"\f0a7"}.fa-arrow-circle-left:before{content:"\f0a8"}.fa-arrow-circle-right:before{content:"\f0a9"}.fa-arrow-circle-up:before{content:"\f0aa"}.fa-arrow-circle-down:before{content:"\f0ab"}.fa-globe:before{content:"\f0ac"}.fa-wrench:before{content:"\f0ad"}.fa-tasks:before{content:"\f0ae"}.fa-filter:before{content:"\f0b0"}.fa-briefcase:before{content:"\f0b1"}.fa-arrows-alt:before{content:"\f0b2"}.fa-group:before,.fa-users:before{content:"\f0c0"}.fa-chain:before,.fa-link:before{content:"\f0c1"}.fa-cloud:before{content:"\f0c2"}.fa-flask:before{content:"\f0c3"}.fa-cut:before,.fa-scissors:before{content:"\f0c4"}.fa-copy:before,.fa-files-o:before{content:"\f0c5"}.fa-paperclip:before{content:"\f0c6"}.fa-save:before,.fa-floppy-o:before{content:"\f0c7"}.fa-square:before{content:"\f0c8"}.fa-navicon:before,.fa-reord
 er:before,.fa-bars:before{content:"\f0c9"}.fa-list-ul:before{content:"\f0ca"}.fa-list-ol:before{content:"\f0cb"}.fa-strikethrough:before{content:"\f0cc"}.fa-underline:before{content:"\f0cd"}.fa-table:before{content:"\f0ce"}.fa-magic:before{content:"\f0d0"}.fa-truck:before{content:"\f0d1"}.fa-pinterest:before{content:"\f0d2"}.fa-pinterest-square:before{content:"\f0d3"}.fa-google-plus-square:before{content:"\f0d4"}.fa-google-plus:before{content:"\f0d5"}.fa-money:before{content:"\f0d6"}.fa-caret-down:before{content:"\f0d7"}.fa-caret-up:before{content:"\f0d8"}.fa-caret-left:before{content:"\f0d9"}.fa-caret-right:before{content:"\f0da"}.fa-columns:before{content:"\f0db"}.fa-unsorted:before,.fa-sort:before{content:"\f0dc"}.fa-sort-down:before,.fa-sort-desc:before{content:"\f0dd"}.fa-sort-up:before,.fa-sort-asc:before{content:"\f0de"}.fa-envelope:before{content:"\f0e0"}.fa-linkedin:before{content:"\f0e1"}.fa-rotate-left:before,.fa-undo:before{content:"\f0e2"}.fa-legal:before,.fa-gavel:befo
 re{content:"\f0e3"}.fa-dashboard:before,.fa-tachometer:before{content:"\f0e4"}.fa-comment-o:before{content:"\f0e5"}.fa-comments-o:before{content:"\f0e6"}.fa-flash:before,.fa-bolt:before{content:"\f0e7"}.fa-sitemap:before{content:"\f0e8"}.fa-umbrella:before{content:"\f0e9"}.fa-paste:before,.fa-clipboard:before{content:"\f0ea"}.fa-lightbulb-o:before{content:"\f0eb"}.fa-exchange:before{content:"\f0ec"}.fa-cloud-download:before{content:"\f0ed"}.fa-cloud-upload:before{content:"\f0ee"}.fa-user-md:before{content:"\f0f0"}.fa-stethoscope:before{content:"\f0f1"}.fa-suitcase:before{content:"\f0f2"}.fa-bell-o:before{content:"\f0a2"}.fa-coffee:before{content:"\f0f4"}.fa-cutlery:before{content:"\f0f5"}.fa-file-text-o:before{content:"\f0f6"}.fa-building-o:before{content:"\f0f7"}.fa-hospital-o:before{content:"\f0f8"}.fa-ambulance:before{content:"\f0f9"}.fa-medkit:before{content:"\f0fa"}.fa-fighter-jet:before{content:"\f0fb"}.fa-beer:before{content:"\f0fc"}.fa-h-square:before{content:"\f0fd"}.fa-plu
 s-square:before{content:"\f0fe"}.fa-angle-double-left:before{content:"\f100"}.fa-angle-double-right:before{content:"\f101"}.fa-angle-double-up:before{content:"\f102"}.fa-angle-double-down:before{content:"\f103"}.fa-angle-left:before{content:"\f104"}.fa-angle-right:before{content:"\f105"}.fa-angle-up:before{content:"\f106"}.fa-angle-down:before{content:"\f107"}.fa-desktop:before{content:"\f108"}.fa-laptop:before{content:"\f109"}.fa-tablet:before{content:"\f10a"}.fa-mobile-phone:before,.fa-mobile:before{content:"\f10b"}.fa-circle-o:before{content:"\f10c"}.fa-quote-left:before{content:"\f10d"}.fa-quote-right:before{content:"\f10e"}.fa-spinner:before{content:"\f110"}.fa-circle:before{content:"\f111"}.fa-mail-reply:before,.fa-reply:before{content:"\f112"}.fa-github-alt:before{content:"\f113"}.fa-folder-o:before{content:"\f114"}.fa-folder-open-o:before{content:"\f115"}.fa-smile-o:before{content:"\f118"}.fa-frown-o:before{content:"\f119"}.fa-meh-o:before{content:"\f11a"}.fa-gamepad:before{
 content:"\f11b"}.fa-keyboard-o:before{content:"\f11c"}.fa-flag-o:before{content:"\f11d"}.fa-flag-checkered:before{content:"\f11e"}.fa-terminal:before{content:"\f120"}.fa-code:before{content:"\f121"}.fa-mail-reply-all:before,.fa-reply-all:before{content:"\f122"}.fa-star-half-empty:before,.fa-star-half-full:before,.fa-star-half-o:before{content:"\f123"}.fa-location-arrow:before{content:"\f124"}.fa-crop:before{content:"\f125"}.fa-code-fork:before{content:"\f126"}.fa-unlink:before,.fa-chain-broken:before{content:"\f127"}.fa-question:before{content:"\f128"}.fa-info:before{content:"\f129"}.fa-exclamation:before{content:"\f12a"}.fa-superscript:before{content:"\f12b"}.fa-subscript:before{content:"\f12c"}.fa-eraser:before{content:"\f12d"}.fa-puzzle-piece:before{content:"\f12e"}.fa-microphone:before{content:"\f130"}.fa-microphone-slash:before{content:"\f131"}.fa-shield:before{content:"\f132"}.fa-calendar-o:before{content:"\f133"}.fa-fire-extinguisher:before{content:"\f134"}.fa-rocket:before{c
 ontent:"\f135"}.fa-maxcdn:before{content:"\f136"}.fa-chevron-circle-left:before{content:"\f137"}.fa-chevron-circle-right:before{content:"\f138"}.fa-chevron-circle-up:before{content:"\f139"}.fa-chevron-circle-down:before{content:"\f13a"}.fa-html5:before{content:"\f13b"}.fa-css3:before{content:"\f13c"}.fa-anchor:before{content:"\f13d"}.fa-unlock-alt:before{content:"\f13e"}.fa-bullseye:before{content:"\f140"}.fa-ellipsis-h:before{content:"\f141"}.fa-ellipsis-v:before{content:"\f142"}.fa-rss-square:before{content:"\f143"}.fa-play-circle:before{content:"\f144"}.fa-ticket:before{content:"\f145"}.fa-minus-square:before{content:"\f146"}.fa-minus-square-o:before{content:"\f147"}.fa-level-up:before{content:"\f148"}.fa-level-down:before{content:"\f149"}.fa-check-square:before{content:"\f14a"}.fa-pencil-square:before{content:"\f14b"}.fa-external-link-square:before{content:"\f14c"}.fa-share-square:before{content:"\f14d"}.fa-compass:before{content:"\f14e"}.fa-toggle-down:before,.fa-caret-square-o
 -down:before{content:"\f150"}.fa-toggle-up:before,.fa-caret-square-o-up:before{content:"\f151"}.fa-toggle-right:before,.fa-caret-square-o-right:before{content:"\f152"}.fa-euro:before,.fa-eur:before{content:"\f153"}.fa-gbp:before{content:"\f154"}.fa-dollar:before,.fa-usd:before{content:"\f155"}.fa-rupee:before,.fa-inr:before{content:"\f156"}.fa-cny:before,.fa-rmb:before,.fa-yen:before,.fa-jpy:before{content:"\f157"}.fa-ruble:before,.fa-rouble:before,.fa-rub:before{content:"\f158"}.fa-won:before,.fa-krw:before{content:"\f159"}.fa-bitcoin:before,.fa-btc:before{content:"\f15a"}.fa-file:before{content:"\f15b"}.fa-file-text:before{content:"\f15c"}.fa-sort-alpha-asc:before{content:"\f15d"}.fa-sort-alpha-desc:before{content:"\f15e"}.fa-sort-amount-asc:before{content:"\f160"}.fa-sort-amount-desc:before{content:"\f161"}.fa-sort-numeric-asc:before{content:"\f162"}.fa-sort-numeric-desc:before{content:"\f163"}.fa-thumbs-up:before{content:"\f164"}.fa-thumbs-down:before{content:"\f165"}.fa-youtube
 -square:before{content:"\f166"}.fa-youtube:before{content:"\f167"}.fa-xing:before{content:"\f168"}.fa-xing-square:before{content:"\f169"}.fa-youtube-play:before{content:"\f16a"}.fa-dropbox:before{content:"\f16b"}.fa-stack-overflow:before{content:"\f16c"}.fa-instagram:before{content:"\f16d"}.fa-flickr:before{content:"\f16e"}.fa-adn:before{content:"\f170"}.fa-bitbucket:before{content:"\f171"}.fa-bitbucket-square:before{content:"\f172"}.fa-tumblr:before{content:"\f173"}.fa-tumblr-square:before{content:"\f174"}.fa-long-arrow-down:before{content:"\f175"}.fa-long-arrow-up:before{content:"\f176"}.fa-long-arrow-left:before{content:"\f177"}.fa-long-arrow-right:before{content:"\f178"}.fa-apple:before{content:"\f179"}.fa-windows:before{content:"\f17a"}.fa-android:before{content:"\f17b"}.fa-linux:before{content:"\f17c"}.fa-dribbble:before{content:"\f17d"}.fa-skype:before{content:"\f17e"}.fa-foursquare:before{content:"\f180"}.fa-trello:before{content:"\f181"}.fa-female:before{content:"\f182"}.fa
 -male:before{content:"\f183"}.fa-gittip:before,.fa-gratipay:before{content:"\f184"}.fa-sun-o:before{content:"\f185"}.fa-moon-o:before{content:"\f186"}.fa-archive:before{content:"\f187"}.fa-bug:before{content:"\f188"}.fa-vk:before{content:"\f189"}.fa-weibo:before{content:"\f18a"}.fa-renren:before{content:"\f18b"}.fa-pagelines:before{content:"\f18c"}.fa-stack-exchange:before{content:"\f18d"}.fa-arrow-circle-o-right:before{content:"\f18e"}.fa-arrow-circle-o-left:before{content:"\f190"}.fa-toggle-left:before,.fa-caret-square-o-left:before{content:"\f191"}.fa-dot-circle-o:before{content:"\f192"}.fa-wheelchair:before{content:"\f193"}.fa-vimeo-square:before{content:"\f194"}.fa-turkish-lira:before,.fa-try:before{content:"\f195"}.fa-plus-square-o:before{content:"\f196"}.fa-space-shuttle:before{content:"\f197"}.fa-slack:before{content:"\f198"}.fa-envelope-square:before{content:"\f199"}.fa-wordpress:before{content:"\f19a"}.fa-openid:before{content:"\f19b"}.fa-institution:before,.fa-bank:before
 ,.fa-university:before{content:"\f19c"}.fa-mortar-board:before,.fa-graduation-cap:before{content:"\f19d"}.fa-yahoo:before{content:"\f19e"}.fa-google:before{content:"\f1a0"}.fa-reddit:before{content:"\f1a1"}.fa-reddit-square:before{content:"\f1a2"}.fa-stumbleupon-circle:before{content:"\f1a3"}.fa-stumbleupon:before{content:"\f1a4"}.fa-delicious:before{content:"\f1a5"}.fa-digg:before{content:"\f1a6"}.fa-pied-piper-pp:before{content:"\f1a7"}.fa-pied-piper-alt:before{content:"\f1a8"}.fa-drupal:before{content:"\f1a9"}.fa-joomla:before{content:"\f1aa"}.fa-language:before{content:"\f1ab"}.fa-fax:before{content:"\f1ac"}.fa-building:before{content:"\f1ad"}.fa-child:before{content:"\f1ae"}.fa-paw:before{content:"\f1b0"}.fa-spoon:before{content:"\f1b1"}.fa-cube:before{content:"\f1b2"}.fa-cubes:before{content:"\f1b3"}.fa-behance:before{content:"\f1b4"}.fa-behance-square:before{content:"\f1b5"}.fa-steam:before{content:"\f1b6"}.fa-steam-square:before{content:"\f1b7"}.fa-recycle:before{content:"\f
 1b8"}.fa-automobile:before,.fa-car:before{content:"\f1b9"}.fa-cab:before,.fa-taxi:before{content:"\f1ba"}.fa-tree:before{content:"\f1bb"}.fa-spotify:before{content:"\f1bc"}.fa-deviantart:before{content:"\f1bd"}.fa-soundcloud:before{content:"\f1be"}.fa-database:before{content:"\f1c0"}.fa-file-pdf-o:before{content:"\f1c1"}.fa-file-word-o:before{content:"\f1c2"}.fa-file-excel-o:before{content:"\f1c3"}.fa-file-powerpoint-o:before{content:"\f1c4"}.fa-file-photo-o:before,.fa-file-picture-o:before,.fa-file-image-o:before{content:"\f1c5"}.fa-file-zip-o:before,.fa-file-archive-o:before{content:"\f1c6"}.fa-file-sound-o:before,.fa-file-audio-o:before{content:"\f1c7"}.fa-file-movie-o:before,.fa-file-video-o:before{content:"\f1c8"}.fa-file-code-o:before{content:"\f1c9"}.fa-vine:before{content:"\f1ca"}.fa-codepen:before{content:"\f1cb"}.fa-jsfiddle:before{content:"\f1cc"}.fa-life-bouy:before,.fa-life-buoy:before,.fa-life-saver:before,.fa-support:before,.fa-life-ring:before{content:"\f1cd"}.fa-cir
 cle-o-notch:before{content:"\f1ce"}.fa-ra:before,.fa-resistance:before,.fa-rebel:before{content:"\f1d0"}.fa-ge:before,.fa-empire:before{content:"\f1d1"}.fa-git-square:before{content:"\f1d2"}.fa-git:before{content:"\f1d3"}.fa-y-combinator-square:before,.fa-yc-square:before,.fa-hacker-news:before{content:"\f1d4"}.fa-tencent-weibo:before{content:"\f1d5"}.fa-qq:before{content:"\f1d6"}.fa-wechat:before,.fa-weixin:before{content:"\f1d7"}.fa-send:before,.fa-paper-plane:before{content:"\f1d8"}.fa-send-o:before,.fa-paper-plane-o:before{content:"\f1d9"}.fa-history:before{content:"\f1da"}.fa-circle-thin:before{content:"\f1db"}.fa-header:before{content:"\f1dc"}.fa-paragraph:before{content:"\f1dd"}.fa-sliders:before{content:"\f1de"}.fa-share-alt:before{content:"\f1e0"}.fa-share-alt-square:before{content:"\f1e1"}.fa-bomb:before{content:"\f1e2"}.fa-soccer-ball-o:before,.fa-futbol-o:before{content:"\f1e3"}.fa-tty:before{content:"\f1e4"}.fa-binoculars:before{content:"\f1e5"}.fa-plug:before{content:"
 \f1e6"}.fa-slideshare:before{content:"\f1e7"}.fa-twitch:before{content:"\f1e8"}.fa-yelp:before{content:"\f1e9"}.fa-newspaper-o:before{content:"\f1ea"}.fa-wifi:before{content:"\f1eb"}.fa-calculator:before{content:"\f1ec"}.fa-paypal:before{content:"\f1ed"}.fa-google-wallet:before{content:"\f1ee"}.fa-cc-visa:before{content:"\f1f0"}.fa-cc-mastercard:before{content:"\f1f1"}.fa-cc-discover:before{content:"\f1f2"}.fa-cc-amex:before{content:"\f1f3"}.fa-cc-paypal:before{content:"\f1f4"}.fa-cc-stripe:before{content:"\f1f5"}.fa-bell-slash:before{content:"\f1f6"}.fa-bell-slash-o:before{content:"\f1f7"}.fa-trash:before{content:"\f1f8"}.fa-copyright:before{content:"\f1f9"}.fa-at:before{content:"\f1fa"}.fa-eyedropper:before{content:"\f1fb"}.fa-paint-brush:before{content:"\f1fc"}.fa-birthday-cake:before{content:"\f1fd"}.fa-area-chart:before{content:"\f1fe"}.fa-pie-chart:before{content:"\f200"}.fa-line-chart:before{content:"\f201"}.fa-lastfm:before{content:"\f202"}.fa-lastfm-square:before{content:"\
 f203"}.fa-toggle-off:before{content:"\f204"}.fa-toggle-on:before{content:"\f205"}.fa-bicycle:before{content:"\f206"}.fa-bus:before{content:"\f207"}.fa-ioxhost:before{content:"\f208"}.fa-angellist:before{content:"\f209"}.fa-cc:before{content:"\f20a"}.fa-shekel:before,.fa-sheqel:before,.fa-ils:before{content:"\f20b"}.fa-meanpath:before{content:"\f20c"}.fa-buysellads:before{content:"\f20d"}.fa-connectdevelop:before{content:"\f20e"}.fa-dashcube:before{content:"\f210"}.fa-forumbee:before{content:"\f211"}.fa-leanpub:before{content:"\f212"}.fa-sellsy:before{content:"\f213"}.fa-shirtsinbulk:before{content:"\f214"}.fa-simplybuilt:before{content:"\f215"}.fa-skyatlas:before{content:"\f216"}.fa-cart-plus:before{content:"\f217"}.fa-cart-arrow-down:before{content:"\f218"}.fa-diamond:before{content:"\f219"}.fa-ship:before{content:"\f21a"}.fa-user-secret:before{content:"\f21b"}.fa-motorcycle:before{content:"\f21c"}.fa-street-view:before{content:"\f21d"}.fa-heartbeat:before{content:"\f21e"}.fa-venus
 :before{content:"\f221"}.fa-mars:before{content:"\f222"}.fa-mercury:before{content:"\f223"}.fa-intersex:before,.fa-transgender:before{content:"\f224"}.fa-transgender-alt:before{content:"\f225"}.fa-venus-double:before{content:"\f226"}.fa-mars-double:before{content:"\f227"}.fa-venus-mars:before{content:"\f228"}.fa-mars-stroke:before{content:"\f229"}.fa-mars-stroke-v:before{content:"\f22a"}.fa-mars-stroke-h:before{content:"\f22b"}.fa-neuter:before{content:"\f22c"}.fa-genderless:before{content:"\f22d"}.fa-facebook-official:before{content:"\f230"}.fa-pinterest-p:before{content:"\f231"}.fa-whatsapp:before{content:"\f232"}.fa-server:before{content:"\f233"}.fa-user-plus:before{content:"\f234"}.fa-user-times:before{content:"\f235"}.fa-hotel:before,.fa-bed:before{content:"\f236"}.fa-viacoin:before{content:"\f237"}.fa-train:before{content:"\f238"}.fa-subway:before{content:"\f239"}.fa-medium:before{content:"\f23a"}.fa-yc:before,.fa-y-combinator:before{content:"\f23b"}.fa-optin-monster:before{co
 ntent:"\f23c"}.fa-opencart:before{content:"\f23d"}.fa-expeditedssl:before{content:"\f23e"}.fa-battery-4:before,.fa-battery-full:before{content:"\f240"}.fa-battery-3:before,.fa-battery-three-quarters:before{content:"\f241"}.fa-battery-2:before,.fa-battery-half:before{content:"\f242"}.fa-battery-1:before,.fa-battery-quarter:before{content:"\f243"}.fa-battery-0:before,.fa-battery-empty:before{content:"\f244"}.fa-mouse-pointer:before{content:"\f245"}.fa-i-cursor:before{content:"\f246"}.fa-object-group:before{content:"\f247"}.fa-object-ungroup:before{content:"\f248"}.fa-sticky-note:before{content:"\f249"}.fa-sticky-note-o:before{content:"\f24a"}.fa-cc-jcb:before{content:"\f24b"}.fa-cc-diners-club:before{content:"\f24c"}.fa-clone:before{content:"\f24d"}.fa-balance-scale:before{content:"\f24e"}.fa-hourglass-o:before{content:"\f250"}.fa-hourglass-1:before,.fa-hourglass-start:before{content:"\f251"}.fa-hourglass-2:before,.fa-hourglass-half:before{content:"\f252"}.fa-hourglass-3:before,.fa-ho
 urglass-end:before{content:"\f253"}.fa-hourglass:before{content:"\f254"}.fa-hand-grab-o:before,.fa-hand-rock-o:before{content:"\f255"}.fa-hand-stop-o:before,.fa-hand-paper-o:before{content:"\f256"}.fa-hand-scissors-o:before{content:"\f257"}.fa-hand-lizard-o:before{content:"\f258"}.fa-hand-spock-o:before{content:"\f259"}.fa-hand-pointer-o:before{content:"\f25a"}.fa-hand-peace-o:before{content:"\f25b"}.fa-trademark:before{content:"\f25c"}.fa-registered:before{content:"\f25d"}.fa-creative-commons:before{content:"\f25e"}.fa-gg:before{content:"\f260"}.fa-gg-circle:before{content:"\f261"}.fa-tripadvisor:before{content:"\f262"}.fa-odnoklassniki:before{content:"\f263"}.fa-odnoklassniki-square:before{content:"\f264"}.fa-get-pocket:before{content:"\f265"}.fa-wikipedia-w:before{content:"\f266"}.fa-safari:before{content:"\f267"}.fa-chrome:before{content:"\f268"}.fa-firefox:before{content:"\f269"}.fa-opera:before{content:"\f26a"}.fa-internet-explorer:before{content:"\f26b"}.fa-tv:before,.fa-tele
 vision:before{content:"\f26c"}.fa-contao:before{content:"\f26d"}.fa-500px:before{content:"\f26e"}.fa-amazon:before{content:"\f270"}.fa-calendar-plus-o:before{content:"\f271"}.fa-calendar-minus-o:before{content:"\f272"}.fa-calendar-times-o:before{content:"\f273"}.fa-calendar-check-o:before{content:"\f274"}.fa-industry:before{content:"\f275"}.fa-map-pin:before{content:"\f276"}.fa-map-signs:before{content:"\f277"}.fa-map-o:before{content:"\f278"}.fa-map:before{content:"\f279"}.fa-commenting:before{content:"\f27a"}.fa-commenting-o:before{content:"\f27b"}.fa-houzz:before{content:"\f27c"}.fa-vimeo:before{content:"\f27d"}.fa-black-tie:before{content:"\f27e"}.fa-fonticons:before{content:"\f280"}.fa-reddit-alien:before{content:"\f281"}.fa-edge:before{content:"\f282"}.fa-credit-card-alt:before{content:"\f283"}.fa-codiepie:before{content:"\f284"}.fa-modx:before{content:"\f285"}.fa-fort-awesome:before{content:"\f286"}.fa-usb:before{content:"\f287"}.fa-product-hunt:before{content:"\f288"}.fa-mix
 cloud:before{content:"\f289"}.fa-scribd:before{content:"\f28a"}.fa-pause-circle:before{content:"\f28b"}.fa-pause-circle-o:before{content:"\f28c"}.fa-stop-circle:before{content:"\f28d"}.fa-stop-circle-o:before{content:"\f28e"}.fa-shopping-bag:before{content:"\f290"}.fa-shopping-basket:before{content:"\f291"}.fa-hashtag:before{content:"\f292"}.fa-bluetooth:before{content:"\f293"}.fa-bluetooth-b:before{content:"\f294"}.fa-percent:before{content:"\f295"}.fa-gitlab:before{content:"\f296"}.fa-wpbeginner:before{content:"\f297"}.fa-wpforms:before{content:"\f298"}.fa-envira:before{content:"\f299"}.fa-universal-access:before{content:"\f29a"}.fa-wheelchair-alt:before{content:"\f29b"}.fa-question-circle-o:before{content:"\f29c"}.fa-blind:before{content:"\f29d"}.fa-audio-description:before{content:"\f29e"}.fa-volume-control-phone:before{content:"\f2a0"}.fa-braille:before{content:"\f2a1"}.fa-assistive-listening-systems:before{content:"\f2a2"}.fa-asl-interpreting:before,.fa-american-sign-language-
 interpreting:before{content:"\f2a3"}.fa-deafness:before,.fa-hard-of-hearing:before,.fa-deaf:before{content:"\f2a4"}.fa-glide:before{content:"\f2a5"}.fa-glide-g:before{content:"\f2a6"}.fa-signing:before,.fa-sign-language:before{content:"\f2a7"}.fa-low-vision:before{content:"\f2a8"}.fa-viadeo:before{content:"\f2a9"}.fa-viadeo-square:before{content:"\f2aa"}.fa-snapchat:before{content:"\f2ab"}.fa-snapchat-ghost:before{content:"\f2ac"}.fa-snapchat-square:before{content:"\f2ad"}.fa-pied-piper:before{content:"\f2ae"}.fa-first-order:before{content:"\f2b0"}.fa-yoast:before{content:"\f2b1"}.fa-themeisle:before{content:"\f2b2"}.fa-google-plus-circle:before,.fa-google-plus-official:before{content:"\f2b3"}.fa-fa:before,.fa-font-awesome:before{content:"\f2b4"}.sr-only{position:absolute;width:1px;height:1px;padding:0;margin:-1px;overflow:hidden;clip:rect(0, 0, 0, 0);border:0}.sr-only-focusable:active,.sr-only-focusable:focus{position:static;width:auto;height:auto;margin:0;overflow:visible;clip:aut
 o}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/30928d92/geode-site/website/content/font/FontAwesome.otf
----------------------------------------------------------------------
diff --git a/geode-site/website/content/font/FontAwesome.otf b/geode-site/website/content/font/FontAwesome.otf
old mode 100755
new mode 100644
index 7012545..d4de13e
Binary files a/geode-site/website/content/font/FontAwesome.otf and b/geode-site/website/content/font/FontAwesome.otf differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/30928d92/geode-site/website/content/font/fontawesome-webfont-eot.eot
----------------------------------------------------------------------
diff --git a/geode-site/website/content/font/fontawesome-webfont-eot.eot b/geode-site/website/content/font/fontawesome-webfont-eot.eot
old mode 100755
new mode 100644
index 0662cb9..c7b00d2
Binary files a/geode-site/website/content/font/fontawesome-webfont-eot.eot and b/geode-site/website/content/font/fontawesome-webfont-eot.eot differ


[12/50] [abbrv] incubator-geode git commit: GEODE-1571: rearrange test and clean up log messages

Posted by ji...@apache.org.
GEODE-1571: rearrange test and clean up log messages


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

Branch: refs/heads/feature/GEODE-1571
Commit: e9d914488b4bafab0244217b1f9c863659d0e938
Parents: 2e1b381
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Wed Jun 22 11:06:22 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Wed Jun 22 11:06:22 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/ServerConnection.java    |  2 -
 .../internal/security/GeodeSecurityUtil.java    |  1 +
 .../security/IntegratedClientAuthDUnitTest.java | 80 +++++++-------------
 .../gemfire/security/SecurityTestUtils.java     |  4 +-
 4 files changed, 29 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e9d91448/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
index 27cb62a..7c650b9 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ServerConnection.java
@@ -786,7 +786,6 @@ public class ServerConnection implements Runnable {
           Subject subject = this.clientUserAuths.getSubject(uniqueId);
           if(subject!=null) {
             threadState = GeodeSecurityUtil.bindSubject(subject);
-            logger.info( command + " with " + uniqueId + ", binding " + subject.getPrincipal() + " to the current thread executing ");
           }
         }
 
@@ -1052,7 +1051,6 @@ public class ServerConnection implements Runnable {
       if(principal instanceof Subject){
         Subject subject = (Subject)principal;
         uniqueId = this.clientUserAuths.putSubject(subject);
-        logger.info("Put subject in Map: "+uniqueId+" for "+ subject.getPrincipal());
         logger.info(this.clientUserAuths);
       }
       else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e9d91448/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
index e1b29bc..0ae3f7d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
@@ -269,6 +269,7 @@ public class GeodeSecurityUtil {
     }
     catch (ShiroException e) {
       String msg = currentUser.getPrincipal() + " not authorized for " + context;
+      logger.info(msg);
       throw new NotAuthorizedException(msg, e);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e9d91448/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
index 121c916..212c9c9 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
@@ -17,7 +17,7 @@
 
 package com.gemstone.gemfire.security;
 
-import static com.gemstone.gemfire.cache.query.CacheUtils.*;
+import static com.gemstone.gemfire.security.SecurityTestUtils.*;
 import static org.assertj.core.api.Assertions.*;
 import static org.junit.Assert.*;
 
@@ -28,6 +28,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.assertj.core.api.ThrowableAssert.ThrowingCallable;
+import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -44,24 +45,36 @@ import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 @Category(DistributedTest.class)
 public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
-
-  private VM server1 = null;
   private VM client1 = null;
   private VM client2 = null;
   private VM client3 = null;
   private int serverPort;
 
   @Before
-  public void before(){
+  public void before() throws Exception{
     final Host host = Host.getHost(0);
-    server1 = host.getVM(0);
     client1 = host.getVM(1);
     client2 = host.getVM(2);
     client3 = host.getVM(3);
-    serverPort = server1.invoke(() -> {
-      JSONAuthorization.setUpWithJsonFile("clientServer.json");
-      return SecurityTestUtils.createCacheServer(JSONAuthorization.class.getName()+".create");
-    });
+
+    JSONAuthorization.setUpWithJsonFile("clientServer.json");
+    serverPort =  SecurityTestUtils.createCacheServer(JSONAuthorization.class.getName()+".create");
+    Region region = getCache().getRegion(SecurityTestUtils.REGION_NAME);
+    assertEquals(0, region.size());
+    for (int i = 0; i < 5; i++) {
+      String key = "key" + i;
+      String value = "value" + i;
+      region.put(key, value);
+    }
+    assertEquals(5, region.size());
+  }
+
+  @After
+  public void after(){
+    client1.invoke(() -> closeCache());
+    client2.invoke(() -> closeCache());
+    client3.invoke(() -> closeCache());
+    closeCache();
   }
 
   @Test
@@ -78,7 +91,6 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
 
   @Test
   public void testGetPutAuthorization() throws InterruptedException {
-    int port = serverPort;
     Map<String, String> allValues = new HashMap<String, String>();
     allValues.put("key1", "value1");
     allValues.put("key2", "value2");
@@ -87,17 +99,9 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
     keys.add("key1");
     keys.add("key2");
 
-    // have one client log in as authorized user to put some data in the regions first.
-    client2.invoke(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", port, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
-      region.putAll(allValues);
-      cache.close();
-    });
-
     // client1 connects to server as a user not authorized to do any operations
     AsyncInvocation ai1 =  client1.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", port, SecurityTestUtils.NO_EXCEPTION);
+      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 
       assertNotAuthorized(()->region.put("key3", "value3"), "DATA:WRITE:AuthRegion:key3");
@@ -122,7 +126,7 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
 
     // client2 connects to user as a user authorized to use AuthRegion region
     AsyncInvocation ai2 =  client2.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", port, SecurityTestUtils.NO_EXCEPTION);
+      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 
       region.put("key3", "value3");
@@ -138,13 +142,11 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
       // keyset
       Set keySet = region.keySet();
       assertEquals(3, keySet.size());
-
-      cache.close();
     });
 
     // client3 connects to user as a user authorized to use key1 in AuthRegion region
     AsyncInvocation ai3 =  client3.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", port, SecurityTestUtils.NO_EXCEPTION);
+      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 
       assertNotAuthorized(()->region.put("key2", "value1"), "DATA:WRITE:AuthRegion:key2");
@@ -159,8 +161,6 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
       // keyset
       Set keySet = region.keySet();
       assertEquals(1, keySet.size());
-
-      cache.close();
     });
 
     ai1.join();
@@ -175,9 +175,6 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
   @Test
   public void testDestroyInvalidate() throws InterruptedException {
 
-    // First, load up 5 keys to work with
-    server1.invoke(generate5Keys());
-
     // Delete one key and invalidate another key with an authorized user.
     AsyncInvocation ai1 = client1.invokeAsync(() -> {
       Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
@@ -194,7 +191,7 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
       assertNotNull("Value of key2 should not be null", region.get("key2"));
       region.invalidate("key2");
       assertNull("Value of key2 should have been null", region.get("key2"));
-      cache.close();
+
     });
 
     // Delete one key and invalidate another key with an unauthorized user.
@@ -225,31 +222,8 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
     assertThatThrownBy(shouldRaiseThrowable).hasMessageContaining(permString);
   }
 
-  private static SerializableRunnable generate5Keys() {
-    SerializableRunnable putKeys = new SerializableRunnable() {
-      @Override
-      public void run() {
-        Cache cache = getCache();
-        Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
-        assertNotNull(region);
-        region.clear();
-        for (int i = 0; i < 5; i++) {
-          String key = "key" + i;
-          String value = "value" + i;
-          region.put(key, value);
-        }
-        assertEquals(5, region.size());
-      }
-    };
-    return putKeys;
-  }
-
   @Test
   public void testRegionClear() throws InterruptedException {
-
-    // First, load up 5 keys to work with
-    server1.invoke(generate5Keys());
-
     // Verify that an unauthorized user can't clear the region
     SerializableRunnable clearUnauthorized = new SerializableRunnable() {
       @Override
@@ -257,7 +231,6 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
         Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
         final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
         assertNotAuthorized(() -> region.clear(), "DATA:WRITE:AuthRegion");
-        cache.close();
       }
     };
     client1.invoke(clearUnauthorized);
@@ -269,7 +242,6 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
         Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
         final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
         region.clear();
-        cache.close();
       }
     };
     client2.invoke(clearAuthorized);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e9d91448/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java b/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
index 208e84a..aeffe7f 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/SecurityTestUtils.java
@@ -282,8 +282,7 @@ public final class SecurityTestUtils {
 
     RegionAttributes attrs = factory.create();
 
-    cache.createRegion(REGION_NAME, attrs);
-
+    Region region = cache.createRegion(REGION_NAME, attrs);
     int port = serverPort <= 0 ? 0 : serverPort;
 
     CacheServer server1 = cache.addCacheServer();
@@ -305,6 +304,7 @@ public final class SecurityTestUtils {
     catch (Exception ex) {
       fail("Got unexpected exception when starting server", ex);
     }
+
     return server1.getPort();
   }
 


[08/50] [abbrv] incubator-geode git commit: Clear Region

Posted by ji...@apache.org.
Clear Region


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

Branch: refs/heads/feature/GEODE-1571
Commit: dd1b04b0a07e2d2ad61504498acc69f9133b1f72
Parents: 8076567
Author: Kevin J. Duling <kd...@pivotal.io>
Authored: Tue Jun 21 16:30:09 2016 -0700
Committer: Kevin J. Duling <kd...@pivotal.io>
Committed: Tue Jun 21 16:30:09 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/command/ClearRegion.java | 139 +++++++++----------
 .../security/IntegratedClientAuthDUnitTest.java |  69 ++++++---
 2 files changed, 120 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dd1b04b0/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ClearRegion.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ClearRegion.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ClearRegion.java
index f9531e7..093a322 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ClearRegion.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ClearRegion.java
@@ -15,40 +15,45 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import com.gemstone.gemfire.cache.operations.RegionClearOperationContext;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
-import com.gemstone.gemfire.cache.operations.RegionClearOperationContext;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-import java.io.IOException;
-import java.nio.ByteBuffer;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 
 public class ClearRegion extends BaseCommand {
 
   private final static ClearRegion singleton = new ClearRegion();
 
-  public static Command getCommand() {
-    return singleton;
+  private ClearRegion() {
   }
 
-  private ClearRegion() {
+  public static Command getCommand() {
+    return singleton;
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException, InterruptedException {
-    Part regionNamePart = null,  callbackArgPart = null;
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException, InterruptedException {
+    Part regionNamePart = null, callbackArgPart = null;
     String regionName = null;
     Object callbackArg = null;
     Part eventPart = null;
@@ -64,13 +69,12 @@ public class ClearRegion extends BaseCommand {
     // Retrieve the data from the message parts
     regionNamePart = msg.getPart(0);
     eventPart = msg.getPart(1);
-//    callbackArgPart = null; (redundant assignment)
+    //    callbackArgPart = null; (redundant assignment)
     if (msg.getNumberOfParts() > 2) {
       callbackArgPart = msg.getPart(2);
       try {
         callbackArg = callbackArgPart.getObject();
-      }
-      catch (Exception e) {
+      } catch (Exception e) {
         writeException(msg, e, false, servConn);
         servConn.setAsTrue(RESPONDED);
         return;
@@ -78,73 +82,66 @@ public class ClearRegion extends BaseCommand {
     }
     regionName = regionNamePart.getString();
     if (logger.isDebugEnabled()) {
-      logger.debug(servConn.getName() + ": Received clear region request (" + msg.getPayloadLength() + " bytes) from " + servConn.getSocketString() + " for region " + regionName);
+      logger.debug(servConn.getName() + ": Received clear region request (" + msg.getPayloadLength() + " bytes) from " + servConn
+        .getSocketString() + " for region " + regionName);
     }
 
     // Process the clear region request
     if (regionName == null) {
-      logger.warn(LocalizedMessage.create(LocalizedStrings.ClearRegion_0_THE_INPUT_REGION_NAME_FOR_THE_CLEAR_REGION_REQUEST_IS_NULL, servConn.getName()));
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ClearRegion_0_THE_INPUT_REGION_NAME_FOR_THE_CLEAR_REGION_REQUEST_IS_NULL, servConn
+        .getName()));
       String errMessage = LocalizedStrings.ClearRegion_THE_INPUT_REGION_NAME_FOR_THE_CLEAR_REGION_REQUEST_IS_NULL.toLocalizedString();
 
-      writeErrorResponse(msg, MessageType.CLEAR_REGION_DATA_ERROR, errMessage,
-          servConn);
+      writeErrorResponse(msg, MessageType.CLEAR_REGION_DATA_ERROR, errMessage, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = LocalizedStrings.ClearRegion_WAS_NOT_FOUND_DURING_CLEAR_REGION_REGUEST.toLocalizedString();
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      }
-      else {
-        // Clear the region
-
-        ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart
-            .getSerializedForm());
-        long threadId = EventID
-            .readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-        long sequenceId = EventID
-            .readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-        EventID eventId = new EventID(servConn.getEventMemberIDByteArray(),
-            threadId, sequenceId);
-
-        try {
-          AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-          if (authzRequest != null) {
-            RegionClearOperationContext clearContext = authzRequest
-                .clearAuthorize(regionName, callbackArg);
-            callbackArg = clearContext.getCallbackArg();
-          }
-          region.basicBridgeClear(callbackArg, servConn.getProxyID(),
-              true /* boolean from cache Client */, eventId);
-        }
-        catch (Exception e) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, e);
-
-          // If an exception occurs during the clear, preserve the connection
-          writeException(msg, e, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
-
-        // Update the statistics and write the reply
-        {
-          long oldStart = start;
-          start = DistributionStats.getStatTime();
-          stats.incProcessClearRegionTime(start - oldStart);
-        }
-        writeReply(msg, servConn);
-        servConn.setAsTrue(RESPONDED);
-        if (logger.isDebugEnabled()) {
-          logger.debug(servConn.getName() + ": Sent clear region response for region " + regionName);
-        }
-        stats.incWriteClearRegionResponseTime(DistributionStats.getStatTime()
-            - start);
+
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = LocalizedStrings.ClearRegion_WAS_NOT_FOUND_DURING_CLEAR_REGION_REGUEST.toLocalizedString();
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+    // Clear the region
+    GeodeSecurityUtil.authorizeRegionWrite(regionName);
+
+    ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
+    long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    EventID eventId = new EventID(servConn.getEventMemberIDByteArray(), threadId, sequenceId);
+
+    try {
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      if (authzRequest != null) {
+        RegionClearOperationContext clearContext = authzRequest.clearAuthorize(regionName, callbackArg);
+        callbackArg = clearContext.getCallbackArg();
       }
+      region.basicBridgeClear(callbackArg, servConn.getProxyID(), true /* boolean from cache Client */, eventId);
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
+
+      // If an exception occurs during the clear, preserve the connection
+      writeException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
     }
 
+    // Update the statistics and write the reply
+    {
+      long oldStart = start;
+      start = DistributionStats.getStatTime();
+      stats.incProcessClearRegionTime(start - oldStart);
+    }
+    writeReply(msg, servConn);
+    servConn.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug(servConn.getName() + ": Sent clear region response for region " + regionName);
+    }
+    stats.incWriteClearRegionResponseTime(DistributionStats.getStatTime() - start);
   }
 
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/dd1b04b0/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
index 4e1f84c..a408c81 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
@@ -176,22 +176,7 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
   public void testDestroyInvalidate() throws InterruptedException {
 
     // First, load up 5 keys to work with
-    SerializableRunnable putKeys = new SerializableRunnable() {
-      @Override
-      public void run() {
-        Cache cache = getCache();
-        Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
-        assertNotNull(region);
-        region.clear();
-        for (int i = 0; i < 5; i++) {
-          String key = "key" + i;
-          String value = "value" + i;
-          region.put(key, value);
-        }
-        assertEquals(5, region.size());
-      }
-    };
-    server1.invoke(putKeys);
+    server1.invoke(generate5Keys());
 
     // Delete one key and invalidate another key with an authorized user.
     AsyncInvocation ai1 = client1.invokeAsync(() -> {
@@ -236,8 +221,58 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
     ai2.checkException();
   }
 
-  public static void assertNotAuthorized(ThrowingCallable shouldRaiseThrowable, String permString){
+  public static void assertNotAuthorized(ThrowingCallable shouldRaiseThrowable, String permString) {
     assertThatThrownBy(shouldRaiseThrowable).hasMessageContaining(permString);
   }
 
+  private static SerializableRunnable generate5Keys() {
+    SerializableRunnable putKeys = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = getCache();
+        Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+        assertNotNull(region);
+        region.clear();
+        for (int i = 0; i < 5; i++) {
+          String key = "key" + i;
+          String value = "value" + i;
+          region.put(key, value);
+        }
+        assertEquals(5, region.size());
+      }
+    };
+    return putKeys;
+  }
+
+  @Test
+  public void testRegionClear() throws InterruptedException {
+
+    // First, load up 5 keys to work with
+    server1.invoke(generate5Keys());
+
+    // Verify that an unauthorized user can't clear the region
+    SerializableRunnable clearUnauthorized = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+        final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+        assertNotAuthorized(() -> region.clear(), "DATA:WRITE:AuthRegion");
+        cache.close();
+      }
+    };
+    client1.invoke(clearUnauthorized);
+
+    // Verify that an authorized user can clear the region
+    SerializableRunnable clearAuthorized = new SerializableRunnable() {
+      @Override
+      public void run() {
+        Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+        final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+        assertNotAuthorized(() -> region.clear(), "DATA:WRITE:AuthRegion");
+        cache.close();
+      }
+    };
+    client1.invoke(clearUnauthorized);
+  }
+
 }


[35/50] [abbrv] incubator-geode git commit: GEODE-1576: Website needs to be upgraded to include WAN and CQ

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/30928d92/geode-site/website/content/font/fontawesome-webfont-ttf.ttf
----------------------------------------------------------------------
diff --git a/geode-site/website/content/font/fontawesome-webfont-ttf.ttf b/geode-site/website/content/font/fontawesome-webfont-ttf.ttf
old mode 100755
new mode 100644
index d365924..f221e50
Binary files a/geode-site/website/content/font/fontawesome-webfont-ttf.ttf and b/geode-site/website/content/font/fontawesome-webfont-ttf.ttf differ

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/30928d92/geode-site/website/content/font/fontawesome-webfont-woff.woff
----------------------------------------------------------------------
diff --git a/geode-site/website/content/font/fontawesome-webfont-woff.woff b/geode-site/website/content/font/fontawesome-webfont-woff.woff
old mode 100755
new mode 100644
index b9bd17e..6e7483c
Binary files a/geode-site/website/content/font/fontawesome-webfont-woff.woff and b/geode-site/website/content/font/fontawesome-webfont-woff.woff differ


[30/50] [abbrv] incubator-geode git commit: GEODE-11: Parameterized EntryScore and TopEntries

Posted by ji...@apache.org.
GEODE-11: Parameterized EntryScore and TopEntries

Added a parameter for the type of the key for EntryScore and TopEntries.


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

Branch: refs/heads/feature/GEODE-1571
Commit: 001015665c15ec99895c0ae236b56bc0e2b28986
Parents: 59d471c
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Mon Jun 20 12:15:46 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri Jun 24 14:41:42 2016 -0700

----------------------------------------------------------------------
 .../cache/lucene/internal/LuceneQueryImpl.java  | 14 ++++----
 .../PageableLuceneQueryResultsImpl.java         | 14 ++++----
 .../lucene/internal/distributed/EntryScore.java | 10 +++---
 .../lucene/internal/distributed/TopEntries.java | 16 +++++----
 ...PageableLuceneQueryResultsImplJUnitTest.java |  6 ++--
 .../DistributedScoringJUnitTest.java            |  8 ++---
 .../distributed/EntryScoreJUnitTest.java        |  4 +--
 .../distributed/LuceneFunctionJUnitTest.java    | 11 +++----
 .../TopEntriesCollectorJUnitTest.java           | 16 ++++-----
 .../TopEntriesFunctionCollectorJUnitTest.java   | 13 +++++---
 .../distributed/TopEntriesJUnitTest.java        | 34 ++++++++++----------
 11 files changed, 76 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
index b910f19..9de16dc 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
@@ -67,11 +67,11 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
 
   @Override
   public Collection<K> findKeys() throws LuceneQueryException {
-    TopEntries entries = findTopEntries();
-    final List<EntryScore> hits = entries.getHits();
+    TopEntries<K> entries = findTopEntries();
+    final List<EntryScore<K>> hits = entries.getHits();
 
     return hits.stream()
-      .map(hit -> (K) hit.getKey())
+      .map(hit -> hit.getKey())
       .collect(Collectors.toList());
   }
 
@@ -95,25 +95,25 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
 
   private PageableLuceneQueryResults<K, V> findPages(int pageSize) throws LuceneQueryException {
 
-    TopEntries entries = findTopEntries();
+    TopEntries<K> entries = findTopEntries();
 
     return new PageableLuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
   }
 
 
 
-  private TopEntries findTopEntries() throws LuceneQueryException {
+  private TopEntries<K> findTopEntries() throws LuceneQueryException {
     TopEntriesCollectorManager manager = new TopEntriesCollectorManager(null, limit);
     LuceneFunctionContext<TopEntriesCollector> context = new LuceneFunctionContext<>(query, indexName, manager, limit);
     TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(context);
 
-    ResultCollector<TopEntriesCollector, TopEntries> rc = (ResultCollector<TopEntriesCollector, TopEntries>) onRegion()
+    ResultCollector<TopEntriesCollector, TopEntries<K>> rc = (ResultCollector<TopEntriesCollector, TopEntries<K>>) onRegion()
         .withArgs(context)
         .withCollector(collector)
         .execute(LuceneFunction.ID);
 
     //TODO provide a timeout to the user?
-    TopEntries entries;
+    TopEntries<K> entries;
     try {
       entries = rc.getResult();
     } catch(FunctionException e) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
index dc2d340..487b740 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
@@ -40,7 +40,7 @@ public class PageableLuceneQueryResultsImpl<K,V> implements PageableLuceneQueryR
   /**
    *  list of docs matching search query
    */
-  private final List<EntryScore> hits;
+  private final List<EntryScore<K>> hits;
   
   /**
    * The maximum score. Lazily evaluated
@@ -62,7 +62,7 @@ public class PageableLuceneQueryResultsImpl<K,V> implements PageableLuceneQueryR
    */
   private int pageSize;
   
-  public PageableLuceneQueryResultsImpl(List<EntryScore> hits, Region<K,V> userRegion, int pageSize) {
+  public PageableLuceneQueryResultsImpl(List<EntryScore<K>> hits, Region<K,V> userRegion, int pageSize) {
     this.hits = hits;
     this.userRegion = userRegion;
     this.pageSize = pageSize == 0 ? Integer.MAX_VALUE : pageSize;
@@ -76,17 +76,17 @@ public class PageableLuceneQueryResultsImpl<K,V> implements PageableLuceneQueryR
     
     int end = currentHit + pageSize;
     end = end > hits.size() ? hits.size() : end;
-    List<EntryScore> scores = hits.subList(currentHit, end);
+    List<EntryScore<K>> scores = hits.subList(currentHit, end);
     
     ArrayList<K> keys = new ArrayList<K>(hits.size());
-    for(EntryScore score : scores) {
-      keys.add((K) score.getKey());
+    for(EntryScore<K> score : scores) {
+      keys.add(score.getKey());
     }
     
     Map<K,V> values = userRegion.getAll(keys);
     
     ArrayList<LuceneResultStruct<K,V>> results = new ArrayList<LuceneResultStruct<K,V>>(hits.size());
-    for(EntryScore score : scores) {
+    for(EntryScore<K> score : scores) {
       V value = values.get(score.getKey());
       results.add(new LuceneResultStructImpl(score.getKey(), value, score.getScore()));
     }
@@ -110,7 +110,7 @@ public class PageableLuceneQueryResultsImpl<K,V> implements PageableLuceneQueryR
   @Override
   public float getMaxScore() {
     if(maxScore == Float.MIN_VALUE) {
-      for(EntryScore score : hits) {
+      for(EntryScore<K> score : hits) {
         maxScore = Math.max(maxScore, score.getScore());
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScore.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScore.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScore.java
index 6690342..e891156 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScore.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScore.java
@@ -29,10 +29,12 @@ import com.gemstone.gemfire.internal.Version;
 
 /**
  * Holds one entry matching search query and its metadata
+ *
+ * @param <K> the type of the key
  */
-public class EntryScore implements DataSerializableFixedID {
+public class EntryScore<K> implements DataSerializableFixedID {
   // Key of the entry matching search query
-  private Object key;
+  private K key;
 
   // The score of this document for the query.
   private float score;
@@ -40,12 +42,12 @@ public class EntryScore implements DataSerializableFixedID {
   public EntryScore() {
   }
 
-  public EntryScore(Object key, float score) {
+  public EntryScore(K key, float score) {
     this.key = key;
     this.score = score;
   }
   
-  public Object getKey() {
+  public K getKey() {
     return key;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntries.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntries.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntries.java
index 5813d75..48a2ca9 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntries.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntries.java
@@ -33,16 +33,18 @@ import com.gemstone.gemfire.internal.Version;
 
 /**
  * Holds a ordered collection of entries matching a search query.
+ *
+ * @param <K> the type of key
  */
-public class TopEntries implements DataSerializableFixedID {
+public class TopEntries<K> implements DataSerializableFixedID {
   // ordered collection of entries
-  private List<EntryScore> hits = new ArrayList<>();
+  private List<EntryScore<K>> hits = new ArrayList<>();
 
   // the maximum number of entries stored in this
   private int limit;
 
   // comparator to order entryScore instances
-  final Comparator<EntryScore> comparator = new EntryScoreComparator();
+  final Comparator<EntryScore<K>> comparator = new EntryScoreComparator();
 
   public TopEntries() {
     this(LuceneQueryFactory.DEFAULT_LIMIT);
@@ -61,7 +63,7 @@ public class TopEntries implements DataSerializableFixedID {
    * 
    * @param entry
    */
-  public void addHit(EntryScore entry) {
+  public void addHit(EntryScore<K> entry) {
     if (hits.size() > 0) {
       EntryScore lastEntry = hits.get(hits.size() - 1);
       if (comparator.compare(lastEntry, entry) < 0) {
@@ -86,7 +88,7 @@ public class TopEntries implements DataSerializableFixedID {
   /**
    * @return The entries collection managed by this instance
    */
-  public List<EntryScore> getHits() {
+  public List<EntryScore<K>> getHits() {
     return hits;
   }
 
@@ -101,9 +103,9 @@ public class TopEntries implements DataSerializableFixedID {
    * Compares scores of two entries using natural ordering. I.e. it returns -1 if the first entry's score is less than
    * the second one.
    */
-  class EntryScoreComparator implements Comparator<EntryScore> {
+  class EntryScoreComparator implements Comparator<EntryScore<K>> {
     @Override
-    public int compare(EntryScore o1, EntryScore o2) {
+    public int compare(EntryScore<K> o1, EntryScore<K> o2) {
       return Float.compare(o1.getScore(), o2.getScore());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
index 5047d0d..7703a74 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
@@ -41,13 +41,13 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class PageableLuceneQueryResultsImplJUnitTest {
 
-  private List<EntryScore> hits;
+  private List<EntryScore<String>> hits;
   private List<LuceneResultStruct> expected = new ArrayList<LuceneResultStruct>();
   private Region<String, String> userRegion;
   
   @Before
   public void setUp() {
-    hits = new ArrayList<EntryScore>();
+    hits = new ArrayList<EntryScore<String>>();
     
     for(int i =0; i < 23; i++) {
       hits.add(new EntryScore("key_" + i, i));
@@ -73,7 +73,7 @@ public class PageableLuceneQueryResultsImplJUnitTest {
   
   @Test
   public void testMaxStore() {
-    hits.set(5, new EntryScore("key_5", 502));
+    hits.set(5, new EntryScore<String>("key_5", 502));
     
     PageableLuceneQueryResultsImpl<String, String> results = new PageableLuceneQueryResultsImpl<String, String>(hits, null, 5);
     

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
index 4abc97c..1f1d2c9 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/DistributedScoringJUnitTest.java
@@ -94,7 +94,7 @@ public class DistributedScoringJUnitTest {
 
     TopEntriesCollector collector = new TopEntriesCollector();
     singleIndexRepo.query(query, 100, collector);
-    List<EntryScore> singleResult = collector.getEntries().getHits();
+    List<EntryScore<String>> singleResult = collector.getEntries().getHits();
 
     IndexRepositoryImpl distIR1 = createIndexRepo();
     populateIndex(testStrings, distIR1, 0, testStrings.length / 3);
@@ -120,14 +120,14 @@ public class DistributedScoringJUnitTest {
     distIR3.query(query, 100, collector3);
     collectors.add(collector3);
 
-    List<EntryScore> distResult = manager.reduce(collectors).getEntries().getHits();
+    List<EntryScore<String>> distResult = manager.reduce(collectors).getEntries().getHits();
     
     Assert.assertEquals(singleResult.size(), distResult.size());
     Assert.assertTrue(singleResult.size() > 0);
     
     for (Iterator single = distResult.iterator(), dist = singleResult.iterator(); single.hasNext() && dist.hasNext();) {
-      EntryScore singleScore = (EntryScore) single.next();
-      EntryScore distScore = (EntryScore) dist.next();
+      EntryScore<String> singleScore = (EntryScore<String>) single.next();
+      EntryScore<String> distScore = (EntryScore<String>) dist.next();
       Assert.assertEquals(singleScore.getKey(), distScore.getKey());
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScoreJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScoreJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScoreJUnitTest.java
index bcb34e2..5cb45ef 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScoreJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/EntryScoreJUnitTest.java
@@ -33,8 +33,8 @@ public class EntryScoreJUnitTest {
   @Test
   public void testSerialization() {
     LuceneServiceImpl.registerDataSerializables();
-    EntryScore entry = new EntryScore("entry", .1f);
-    EntryScore copy = CopyHelper.deepCopy(entry);
+    EntryScore<String> entry = new EntryScore<String>("entry", .1f);
+    EntryScore<String> copy = CopyHelper.deepCopy(entry);
     assertEquals("entry", copy.getKey());
     assertEquals(.1f, copy.getScore(), 0f);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
index 26c4d62..dc1cc01 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
@@ -40,7 +40,6 @@ import com.gemstone.gemfire.cache.lucene.internal.StringQueryProvider;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexRepository;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector;
 import com.gemstone.gemfire.cache.lucene.internal.repository.RepositoryManager;
-import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.internal.cache.BucketNotFoundException;
 import com.gemstone.gemfire.internal.cache.InternalCache;
 import com.gemstone.gemfire.internal.cache.execute.InternalRegionFunctionContext;
@@ -60,11 +59,11 @@ public class LuceneFunctionJUnitTest {
 
   String regionPath = "/region";
   String indexName = "index";
-  final EntryScore r1_1 = new EntryScore("key-1-1", .5f);
-  final EntryScore r1_2 = new EntryScore("key-1-2", .4f);
-  final EntryScore r1_3 = new EntryScore("key-1-3", .3f);
-  final EntryScore r2_1 = new EntryScore("key-2-1", .45f);
-  final EntryScore r2_2 = new EntryScore("key-2-2", .35f);
+  final EntryScore<String> r1_1 = new EntryScore<String>("key-1-1", .5f);
+  final EntryScore<String> r1_2 = new EntryScore<String>("key-1-2", .4f);
+  final EntryScore<String> r1_3 = new EntryScore<String>("key-1-3", .3f);
+  final EntryScore<String> r2_1 = new EntryScore<String>("key-2-1", .45f);
+  final EntryScore<String> r2_2 = new EntryScore<String>("key-2-2", .35f);
 
   InternalRegionFunctionContext mockContext;
   ResultSender<TopEntriesCollector> mockResultSender;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
index 9e2f310..342f7b0 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesCollectorJUnitTest.java
@@ -35,16 +35,16 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 @Category(UnitTest.class)
 public class TopEntriesCollectorJUnitTest {
 
-  private EntryScore r1_1 = new EntryScore("1-1", .9f);
-  private EntryScore r1_2 = new EntryScore("1-2", .7f);
-  private EntryScore r1_3 = new EntryScore("1-3", .5f);
+  private EntryScore<String> r1_1 = new EntryScore<String>("1-1", .9f);
+  private EntryScore<String> r1_2 = new EntryScore<String>("1-2", .7f);
+  private EntryScore<String> r1_3 = new EntryScore<String>("1-3", .5f);
 
-  private EntryScore r2_1 = new EntryScore("2-1", .85f);
-  private EntryScore r2_2 = new EntryScore("2-2", .65f);
+  private EntryScore<String> r2_1 = new EntryScore<String>("2-1", .85f);
+  private EntryScore<String> r2_2 = new EntryScore<String>("2-2", .65f);
 
-  private EntryScore r3_1 = new EntryScore("3-1", .8f);
-  private EntryScore r3_2 = new EntryScore("3-2", .6f);
-  private EntryScore r3_3 = new EntryScore("3-3", .4f);
+  private EntryScore<String> r3_1 = new EntryScore<String>("3-1", .8f);
+  private EntryScore<String> r3_2 = new EntryScore<String>("3-2", .6f);
+  private EntryScore<String> r3_3 = new EntryScore<String>("3-3", .4f);
 
   private TopEntriesCollectorManager manager;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
index 372c5a3..9e8be3c 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesFunctionCollectorJUnitTest.java
@@ -42,15 +42,18 @@ import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class TopEntriesFunctionCollectorJUnitTest {
-  EntryScore r1_1, r1_2, r2_1, r2_2;
+  EntryScore<String> r1_1;
+  EntryScore<String> r1_2;
+  EntryScore<String> r2_1;
+  EntryScore<String> r2_2;
   TopEntriesCollector result1, result2;
 
   @Before
   public void initializeCommonObjects() {
-    r1_1 = new EntryScore("3", .9f);
-    r1_2 = new EntryScore("1", .8f);
-    r2_1 = new EntryScore("2", 0.85f);
-    r2_2 = new EntryScore("4", 0.1f);
+    r1_1 = new EntryScore<String>("3", .9f);
+    r1_2 = new EntryScore<String>("1", .8f);
+    r2_1 = new EntryScore<String>("2", 0.85f);
+    r2_2 = new EntryScore<String>("4", 0.1f);
 
     result1 = new TopEntriesCollector(null);
     result1.collect(r1_1);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/00101566/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
index 410d0ef..feeaa21 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/TopEntriesJUnitTest.java
@@ -41,14 +41,14 @@ public class TopEntriesJUnitTest {
 
   private Mockery mockContext;
 
-  private EntryScore r1_1 = new EntryScore("3", .9f);
-  private EntryScore r1_2 = new EntryScore("1", .8f);
-  private EntryScore r2_1 = new EntryScore("2", 0.85f);
-  private EntryScore r2_2 = new EntryScore("4", 0.1f);
+  private EntryScore<String> r1_1 = new EntryScore("3", .9f);
+  private EntryScore<String> r1_2 = new EntryScore("1", .8f);
+  private EntryScore<String> r2_1 = new EntryScore("2", 0.85f);
+  private EntryScore<String> r2_2 = new EntryScore("4", 0.1f);
 
   @Test
   public void testPopulateTopEntries() {
-    TopEntries hits = new TopEntries();
+    TopEntries<String> hits = new TopEntries<String>();
     hits.addHit(r1_1);
     hits.addHit(r2_1);
     hits.addHit(r1_2);
@@ -60,9 +60,9 @@ public class TopEntriesJUnitTest {
 
   @Test
   public void putSameScoreEntries() {
-    TopEntries hits = new TopEntries();
-    EntryScore r1 = new EntryScore("1", .8f);
-    EntryScore r2 = new EntryScore("2", .8f);
+    TopEntries<String> hits = new TopEntries<String>();
+    EntryScore<String> r1 = new EntryScore<String>("1", .8f);
+    EntryScore<String> r2 = new EntryScore<String>("2", .8f);
     hits.addHit(r1);
     hits.addHit(r2);
     
@@ -72,21 +72,21 @@ public class TopEntriesJUnitTest {
   
   @Test
   public void testInitialization() {
-    TopEntries hits = new TopEntries();
+    TopEntries<String> hits = new TopEntries<String>();
     assertEquals(LuceneQueryFactory.DEFAULT_LIMIT, hits.getLimit());
 
-    hits = new TopEntries(123);
+    hits = new TopEntries<String>(123);
     assertEquals(123, hits.getLimit());
   }
 
   @Test(expected = IllegalArgumentException.class)
   public void testInvalidLimit() {
-    new TopEntries(-1);
+    new TopEntries<String>(-1);
   }
   
   @Test
   public void enforceLimit() throws Exception {
-    TopEntries hits = new TopEntries(3);
+    TopEntries<String> hits = new TopEntries<String>(3);
     hits.addHit(r1_1);
     hits.addHit(r2_1);
     hits.addHit(r1_2);
@@ -99,13 +99,13 @@ public class TopEntriesJUnitTest {
   @Test
   public void testSerialization() {
     LuceneServiceImpl.registerDataSerializables();
-    TopEntries hits = new TopEntries(3);
+    TopEntries<String> hits = new TopEntries<String>(3);
     
-    TopEntries copy = CopyHelper.deepCopy(hits);
+    TopEntries<String> copy = CopyHelper.deepCopy(hits);
     assertEquals(3, copy.getLimit());
     assertEquals(0, copy.getHits().size());
     
-    hits = new TopEntries(3);
+    hits = new TopEntries<String>(3);
     hits.addHit(r1_1);
     hits.addHit(r2_1);
     hits.addHit(r1_2);
@@ -116,8 +116,8 @@ public class TopEntriesJUnitTest {
   }
 
   // TODO: extract to lucene test util class
-  public static void verifyResultOrder(Collection<EntryScore> list, EntryScore... expectedEntries) {
-    Iterator<EntryScore> iter = list.iterator();
+  public static void verifyResultOrder(Collection<EntryScore<String>> list, EntryScore<String>... expectedEntries) {
+    Iterator<EntryScore<String>> iter = list.iterator();
     for (EntryScore expectedEntry : expectedEntries) {
       if (!iter.hasNext()) {
         fail();


[38/50] [abbrv] incubator-geode git commit: GEODE-1576: Website needs to be upgraded to include WAN and CQ

Posted by ji...@apache.org.
GEODE-1576: Website needs to be upgraded to include WAN and CQ

adding WAN, CQ, native client and Adapters to the list of features


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

Branch: refs/heads/feature/GEODE-1571
Commit: 95eff4811bbdb7464f7d10fa24e15465015bb86d
Parents: e987e5d
Author: Swapnil Bawaskar <sb...@pivotal.io>
Authored: Fri Jun 24 15:07:26 2016 -0700
Committer: Swapnil Bawaskar <sb...@pivotal.io>
Committed: Fri Jun 24 16:05:36 2016 -0700

----------------------------------------------------------------------
 geode-site/website/content/index.html | 49 ++++++++++++++++++++++++------
 1 file changed, 40 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/95eff481/geode-site/website/content/index.html
----------------------------------------------------------------------
diff --git a/geode-site/website/content/index.html b/geode-site/website/content/index.html
index 8b8f61a..e47b763 100644
--- a/geode-site/website/content/index.html
+++ b/geode-site/website/content/index.html
@@ -49,56 +49,87 @@ under the License. -->
     <div class="container">
         <div class="row">
             <div class="col-md-4">
-                <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-sitemap"></i></div>
+                <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="fa fa-sitemap"></i></div>
                 <h3>Replication and Partitioning</h3>
                 <p>Data can easily be partitioned (sharded) or replicated between nodes allowing performance to scale as needed. Durability is ensured through redundant in-memory copies and disk-based persistence.</p>
             </div>
             <div class="col-md-4">
-                <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-hdd"></i></div>
+                <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="fa fa-hdd-o"></i></div>
                 <h3>Persistence</h3>
                 <p>Super fast write-ahead-logging (WAL) persistence with a shared-nothing architecture that is optimized for fast parallel recovery of nodes or an entire cluster.</p>
             </div>
             <div class="col-md-4">
-                <div class="bf-icon-wrap"><i aria-hidden="true" class="icon-rocket"></i></div>
+                <div class="bf-icon-wrap"><i aria-hidden="true" class="fa fa-rocket"></i></div>
                 <h3>Performance</h3>
                 <p>Linear-scaling low latency for transactions, reads, writes and query processing of indexed or unindexed data.</p>
             </div>
         </div>
         <div class="row">
             <div class="col-md-4">
-                <div class="bf-icon-wrap" style="font-size:40px; vertical-align: 15px;"><i aria-hidden="true" class="icon-fast-forward"></i><i aria-hidden="true" class="icon-dashboard"></i></div>
+                <div class="bf-icon-wrap" style="font-size:40px; vertical-align: 15px;"><i aria-hidden="true" class="fa fa-fast-forward"></i><i aria-hidden="true" class="fa fa-dashboard"></i></div>
                 <h3>In-Memory Storage</h3>
                 <p>Blazing fast in-memory storage optimized for large heaps, with the option of using off-heap storage, compression and features such as disk-overflow, eviction and expiration of data.</p>
             </div>
             <div class="col-md-4">
-                <div class="bf-icon-wrap"><span style="font-size:60px" aria-hidden="true" class="icon-cogs"></span></div>
+                <div class="bf-icon-wrap"><span style="font-size:60px" aria-hidden="true" class="fa fa-cogs"></span></div>
                 <h3>Functions</h3>
                 <p>Distributed location-aware user functions can be deployed and executed by the same nodes storing relevant sharded data for fast parallel processing. Failed operations can be retried on replicant nodes.</p>
             </div>
             <div class="col-md-4">
-                <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-credit-card"></i></div>
+                <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="fa fa-credit-card"></i></div>
                 <h3>Transactions</h3>
                 <p>ACID distributed transactions support efficient and safe coordinated operations on colocated data. Transactions can be initiated or suspended by either a client or a server.</p>
             </div>
         </div>
         <div class="row">
           <div class="col-md-4">
-              <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-table"></i></div>
+              <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="fa fa-table"></i></div>
               <h3>OQL and Indexes</h3>
               <p>Object Query Language allows distributed query execution on hot and cold data, with SQL-like capabilities, including joins.<br/>
               Multiple kinds of indexes can be defined and consistently maintained across the cluster.</p>
           </div>
           <div class="col-md-4">
-              <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-bolt"></i></div>
+              <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="fa fa-bolt"></i></div>
               <h3>Events</h3>
               <p>Clients can be notified about server-side data events, and servers can react synchronously or asynchronously with guaranteed delivery of ordered events.</p>
           </div>
           <div class="col-md-4">
-              <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="icon-cloud"></i></div>
+              <div class="bf-icon-wrap"><i style="font-size:65px; vertical-align: -5px;" aria-hidden="true" class="fa fa-cloud"></i></div>
               <h3>Clustering</h3>
               <p>Highly scalable, robust advanced clustering technology with failure detection, dynamic scaling, and network-partition detection algorithms.</p>
           </div>
         </div>
+        <div class="row">
+          <div class="col-md-4">
+              <div class="bf-icon-wrap" style="font-size:30px; vertical-align: 15px;" aria-hidden="true"><i class="fa fa-cloud"></i><i style="font-size:15px; vertical-align: 15px;" class="fa fa-exchange fa-8x"></i><i class="fa fa-cloud"></i></div>
+              <h3>Multi-Cluster</h3>
+              <p>Geode clusters can be replicated over WAN in various topologies: active-active, active-passive, ring, hub-spoke, star, etc.</p>
+          </div>
+          <div class="col-md-4">
+              <div class="bf-icon-wrap" style="font-size:30px; vertical-align: -5px;" aria-hidden="true"><i class="fa fa-bullhorn"></i><i class="fa fa-laptop"></i></div>
+              <h3>Continuous Query</h3>
+              <p>Clients can stay up to date by registering OQL queries with the Geode servers, making event-driven applications possible.</p>
+          </div>
+          <div class="col-md-4">
+              <div class="bf-icon-wrap" style="font-size:30px; vertical-align: -5px;" aria-hidden="true"><i class="fa fa-desktop"></i><i class="fa fa-laptop"></i></div>
+              <h3>Clients</h3>
+              <p>Clients are available for Java. (C++, C# .NET and Node.js coming soon.) A REST API is available for all other languages.</p>
+          </div>
+        </div>
+      <div class="row">
+          <div class="col-md-4">
+              <div></div>
+          </div>
+          <div class="col-md-4">
+              <div class="bf-icon-wrap" style="font-size:65px; vertical-align: -5px;" aria-hidden="true"><i class="fa fa-plug"></i></div>
+              <h3>Adapters</h3>
+              <p>Geode can be used as a drop-in replacement for Redis and memcached, allowing users of these caches to use Geode's server-side features like WAN.</p>
+          </div>
+          <div class="col-md-4">
+              <div></div>
+          </div>
+        </div>
+
     </div>
 
     </div>


[42/50] [abbrv] incubator-geode git commit: GEODE-1751: putting security checks in all applicable client-server commands.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
index 9cea674..56100f2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
@@ -52,8 +52,7 @@ public class GetAll651 extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException, InterruptedException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException, InterruptedException {
     Part regionNamePart = null, keysPart = null;
     String regionName = null;
     Object[] keys = null;
@@ -69,8 +68,7 @@ public class GetAll651 extends BaseCommand {
     keysPart = msg.getPart(1);
     try {
       keys = (Object[]) keysPart.getObject();
-    }
-    catch (Exception e) {
+    } catch (Exception e) {
       writeChunkedException(msg, e, false, servConn);
       servConn.setAsTrue(RESPONDED);
       return;
@@ -78,21 +76,19 @@ public class GetAll651 extends BaseCommand {
 
     if (logger.isDebugEnabled()) {
       StringBuffer buffer = new StringBuffer();
-      buffer
-        .append(servConn.getName())
-        .append(": Received getAll request (")
-        .append(msg.getPayloadLength())
-        .append(" bytes) from ")
-        .append(servConn.getSocketString())
-        .append(" for region ")
-        .append(regionName)
-        .append(" keys ");
+      buffer.append(servConn.getName())
+            .append(": Received getAll request (")
+            .append(msg.getPayloadLength())
+            .append(" bytes) from ")
+            .append(servConn.getSocketString())
+            .append(" for region ")
+            .append(regionName)
+            .append(" keys ");
       if (keys != null) {
-        for (int i=0; i<keys.length; i++) {
+        for (int i = 0; i < keys.length; i++) {
           buffer.append(keys[i]).append(" ");
         }
-      }
-      else {
+      } else {
         buffer.append("NULL");
       }
       logger.debug(buffer.toString());
@@ -101,50 +97,48 @@ public class GetAll651 extends BaseCommand {
     // Process the getAll request
     if (regionName == null) {
       String message = null;
-//      if (regionName == null) (can only be null) 
+      //      if (regionName == null) (can only be null)
       {
         message = LocalizedStrings.GetAll_THE_INPUT_REGION_NAME_FOR_THE_GETALL_REQUEST_IS_NULL.toLocalizedString();
       }
       logger.warn("{}: {}", servConn.getName(), message);
-      writeChunkedErrorResponse(msg, MessageType.GET_ALL_DATA_ERROR, message,
-          servConn);
+      writeChunkedErrorResponse(msg, MessageType.GET_ALL_DATA_ERROR, message, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = " was not found during getAll request";
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      }
-      else {
-        // Send header
-        ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
 
-        // Send chunk response
-        try {
-          fillAndSendGetAllResponseChunks(region, regionName, keys, servConn);
-          servConn.setAsTrue(RESPONDED);
-        }
-        catch (Exception e) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, e);
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = " was not found during getAll request";
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+    // Send header
+    ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
+    chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
+    chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+    chunkedResponseMsg.sendHeader();
 
-          // Otherwise, write an exception message and continue
-          writeChunkedException(msg, e, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
-      }
+    // Send chunk response
+    try {
+      fillAndSendGetAllResponseChunks(region, regionName, keys, servConn);
+      servConn.setAsTrue(RESPONDED);
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
+
+      // Otherwise, write an exception message and continue
+      writeChunkedException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
     }
   }
 
   private void fillAndSendGetAllResponseChunks(Region region,
-      String regionName, Object[] keys, ServerConnection servConn)
-      throws IOException {
+                                               String regionName,
+                                               Object[] keys,
+                                               ServerConnection servConn) throws IOException {
 
     // Interpret null keys object as a request to get all key,value entry pairs
     // of the region; otherwise iterate each key and perform the get behavior.
@@ -153,8 +147,7 @@ public class GetAll651 extends BaseCommand {
     if (keys != null) {
       allKeysIter = null;
       numKeys = keys.length;
-    }
-    else {
+    } else {
       Set allKeys = region.keySet();
       allKeysIter = allKeys.iterator();
       numKeys = allKeys.size();
@@ -165,7 +158,7 @@ public class GetAll651 extends BaseCommand {
     Request request = (Request) Request.getCommand();
     Object[] valueAndIsObject = new Object[3];
     final boolean isDebugEnabled = logger.isDebugEnabled();
-    for (int i=0; i<numKeys; i++) {
+    for (int i = 0; i < numKeys; i++) {
       // Send the intermediate chunk if necessary
       if (values.size() == maximumChunkSize) {
         // Send the chunk and clear the list
@@ -177,8 +170,7 @@ public class GetAll651 extends BaseCommand {
       boolean keyNotPresent = false;
       if (keys != null) {
         key = keys[i];
-      }
-      else {
+      } else {
         key = allKeysIter.next();
       }
       if (isDebugEnabled) {
@@ -192,47 +184,58 @@ public class GetAll651 extends BaseCommand {
           if (isDebugEnabled) {
             logger.debug("{}: Passed GET pre-authorization for key={}", servConn.getName(), key);
           }
-        }
-        catch (NotAuthorizedException ex) {
-          logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {servConn.getName(), key}), ex);
+        } catch (NotAuthorizedException ex) {
+          logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {
+            servConn.getName(),
+            key
+          }), ex);
           values.addExceptionPart(key, ex);
           continue;
         }
       }
 
-      GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+      try {
+        GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+      } catch (NotAuthorizedException ex) {
+        logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {
+          servConn.getName(),
+          key
+        }), ex);
+        values.addExceptionPart(key, ex);
+        continue;
+      }
 
       // Get the value and update the statistics. Do not deserialize
       // the value if it is a byte[].
       // Getting a value in serialized form is pretty nasty. I split this out
       // so the logic can be re-used by the CacheClientProxy.
-      request.getValueAndIsObject(region, key,
-          null, servConn, valueAndIsObject);
+      request.getValueAndIsObject(region, key, null, servConn, valueAndIsObject);
       Object value = valueAndIsObject[0];
       boolean isObject = ((Boolean) valueAndIsObject[1]).booleanValue();
-      keyNotPresent = ((Boolean) valueAndIsObject[2]).booleanValue();;
+      keyNotPresent = ((Boolean) valueAndIsObject[2]).booleanValue();
+      ;
       if (isDebugEnabled) {
         logger.debug("{}: Retrieved value for key={}: {}", servConn.getName(), key, value);
       }
 
       if (postAuthzRequest != null) {
         try {
-          getContext = postAuthzRequest.getAuthorize(regionName, key, value,
-              isObject, getContext);
+          getContext = postAuthzRequest.getAuthorize(regionName, key, value, isObject, getContext);
           byte[] serializedValue = getContext.getSerializedValue();
           if (serializedValue == null) {
             value = getContext.getObject();
-          }
-          else {
+          } else {
             value = serializedValue;
           }
           isObject = getContext.isObject();
           if (isDebugEnabled) {
             logger.debug("{}: Passed GET post-authorization for key={}: {}", servConn.getName(), key, value);
           }
-        }
-        catch (NotAuthorizedException ex) {
-          logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {servConn.getName(), key}), ex);
+        } catch (NotAuthorizedException ex) {
+          logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {
+            servConn.getName(),
+            key
+          }), ex);
           values.addExceptionPart(key, ex);
           continue;
         }
@@ -245,23 +248,24 @@ public class GetAll651 extends BaseCommand {
       }
 
       // Add the value to the list of values
-      if(keyNotPresent) {
+      if (keyNotPresent) {
         if (logger.isDebugEnabled()) {
           logger.debug("{}: key={} is not present on server.", servConn.getName(), key);
-        } 
+        }
         values.addObjectPartForAbsentKey(key, value);
-      }else {
-        values.addObjectPart(key, value, isObject, null);  
+      } else {
+        values.addObjectPart(key, value, isObject, null);
       }
     }
 
-  // Send the last chunk even if the list is of zero size.
+    // Send the last chunk even if the list is of zero size.
     sendGetAllResponseChunk(region, values, true, servConn);
     servConn.setAsTrue(RESPONDED);
   }
 
   /**
    * This method is protected so it can be overriden in GetAllForRI.
+   *
    * @param includeKeys if the part list should include the keys
    */
   protected ObjectPartList651 getObjectPartsList(boolean includeKeys) {
@@ -269,15 +273,18 @@ public class GetAll651 extends BaseCommand {
     return values;
   }
 
-  private static void sendGetAllResponseChunk(Region region, ObjectPartList651 list,
-      boolean lastChunk, ServerConnection servConn) throws IOException {
+  private static void sendGetAllResponseChunk(Region region,
+                                              ObjectPartList651 list,
+                                              boolean lastChunk,
+                                              ServerConnection servConn) throws IOException {
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     chunkedResponseMsg.setNumberOfParts(1);
     chunkedResponseMsg.setLastChunk(lastChunk);
     chunkedResponseMsg.addObjPart(list, zipValues);
 
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sending {} getAll response chunk for region={} values={} chunk=<{}>", servConn.getName(), (lastChunk ? " last " : " "), region.getFullPath(), list, chunkedResponseMsg);
+      logger.debug("{}: Sending {} getAll response chunk for region={} values={} chunk=<{}>", servConn.getName(), (lastChunk ? " last " : " "), region
+        .getFullPath(), list, chunkedResponseMsg);
     }
 
     chunkedResponseMsg.sendChunk(servConn);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
index 007d583..d162b19 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
@@ -57,14 +57,12 @@ public class GetAll70 extends BaseCommand {
   /**
    * client wants values to be serialized as byte arrays, not objects
    */
- // private boolean requestSerializedValues;
-
+  // private boolean requestSerializedValues;
   protected GetAll70() {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-          throws IOException, InterruptedException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException, InterruptedException {
     Part regionNamePart = null, keysPart = null;
     String regionName = null;
     Object[] keys = null;
@@ -91,15 +89,14 @@ public class GetAll70 extends BaseCommand {
 
     if (logger.isDebugEnabled()) {
       StringBuffer buffer = new StringBuffer();
-      buffer
-              .append(servConn.getName())
-              .append(": Received getAll request (")
-              .append(msg.getPayloadLength())
-              .append(" bytes) from ")
-              .append(servConn.getSocketString())
-              .append(" for region ")
-              .append(regionName)
-              .append(" keys ");
+      buffer.append(servConn.getName())
+            .append(": Received getAll request (")
+            .append(msg.getPayloadLength())
+            .append(" bytes) from ")
+            .append(servConn.getSocketString())
+            .append(" for region ")
+            .append(regionName)
+            .append(" keys ");
       if (keys != null) {
         for (int i = 0; i < keys.length; i++) {
           buffer.append(keys[i]).append(" ");
@@ -118,42 +115,44 @@ public class GetAll70 extends BaseCommand {
         message = LocalizedStrings.GetAll_THE_INPUT_REGION_NAME_FOR_THE_GETALL_REQUEST_IS_NULL.toLocalizedString();
       }
       logger.warn("{}: {}", servConn.getName(), message);
-      writeChunkedErrorResponse(msg, MessageType.GET_ALL_DATA_ERROR, message,
-              servConn);
+      writeChunkedErrorResponse(msg, MessageType.GET_ALL_DATA_ERROR, message, servConn);
       servConn.setAsTrue(RESPONDED);
-    } else {
-      LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = " was not found during getAll request";
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      } else {
-        // Send header
-        ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
+      return;
+    }
 
-        // Send chunk response
-        try {
-          fillAndSendGetAllResponseChunks(region, regionName, keys, servConn, requestSerializedValues);
-          servConn.setAsTrue(RESPONDED);
-        } catch (Exception e) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, e);
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = " was not found during getAll request";
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+    // Send header
+    ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
+    chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
+    chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+    chunkedResponseMsg.sendHeader();
 
-          // Otherwise, write an exception message and continue
-          writeChunkedException(msg, e, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
-      }
+    // Send chunk response
+    try {
+      fillAndSendGetAllResponseChunks(region, regionName, keys, servConn, requestSerializedValues);
+      servConn.setAsTrue(RESPONDED);
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
+
+      // Otherwise, write an exception message and continue
+      writeChunkedException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
     }
   }
 
   private void fillAndSendGetAllResponseChunks(Region region,
-      String regionName, Object[] keys, ServerConnection servConn, boolean requestSerializedValues)
-      throws IOException {
+                                               String regionName,
+                                               Object[] keys,
+                                               ServerConnection servConn,
+                                               boolean requestSerializedValues) throws IOException {
 
     // Interpret null keys object as a request to get all key,value entry pairs
     // of the region; otherwise iterate each key and perform the get behavior.
@@ -176,133 +175,144 @@ public class GetAll70 extends BaseCommand {
     // So the only reason we would tell the VersionedObjectList that it needs to track keys is if we are running
     // in the old mode (which may be impossible since we only used that mode pre 7.0) in which the client told us
     // to get and return all the keys and values. I think this was used for register interest.
-    VersionedObjectList values = new VersionedObjectList(maximumChunkSize, keys == null, region.getAttributes().getConcurrencyChecksEnabled(), requestSerializedValues);
+    VersionedObjectList values = new VersionedObjectList(maximumChunkSize, keys == null, region.getAttributes()
+                                                                                               .getConcurrencyChecksEnabled(), requestSerializedValues);
     try {
-    AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-    AuthorizeRequestPP postAuthzRequest = servConn.getPostAuthzRequest();
-    Get70 request = (Get70) Get70.getCommand();
-    final boolean isDebugEnabled = logger.isDebugEnabled();
-    for (int i = 0; i < numKeys; i++) {
-      // Send the intermediate chunk if necessary
-      if (values.size() == maximumChunkSize) {
-        // Send the chunk and clear the list
-        values.setKeys(null);
-        sendGetAllResponseChunk(region, values, false, servConn);
-        values.clear();
-      }
-
-      Object key;
-      boolean keyNotPresent = false;
-      if (keys != null) {
-        key = keys[i];
-      } else {
-        key = allKeysIter.next();
-      }
-      if (isDebugEnabled) {
-        logger.debug("{}: Getting value for key={}", servConn.getName(), key);
-      }
-      // Determine if the user authorized to get this key
-      GetOperationContext getContext = null;
-      if (authzRequest != null) {
-        try {
-          getContext = authzRequest.getAuthorize(regionName, key, null);
-          if (isDebugEnabled) {
-            logger.debug("{}: Passed GET pre-authorization for key={}", servConn.getName(), key);
-          }
-        } catch (NotAuthorizedException ex) {
-          logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[]{servConn.getName(), key}), ex);
-          values.addExceptionPart(key, ex);
-          continue;
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      AuthorizeRequestPP postAuthzRequest = servConn.getPostAuthzRequest();
+      Get70 request = (Get70) Get70.getCommand();
+      final boolean isDebugEnabled = logger.isDebugEnabled();
+      for (int i = 0; i < numKeys; i++) {
+        // Send the intermediate chunk if necessary
+        if (values.size() == maximumChunkSize) {
+          // Send the chunk and clear the list
+          values.setKeys(null);
+          sendGetAllResponseChunk(region, values, false, servConn);
+          values.clear();
         }
-      }
-
-      try {
-        GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
-      }
-      catch (NotAuthorizedException ex) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[]{servConn.getName(), key}), ex);
-        values.addExceptionPart(key, ex);
-        continue;
-      }
-
-      // Get the value and update the statistics. Do not deserialize
-      // the value if it is a byte[].
-      // Getting a value in serialized form is pretty nasty. I split this out
-      // so the logic can be re-used by the CacheClientProxy.
-      Get70.Entry entry = request.getEntry(region, key, null, servConn);
-      @Retained final Object originalData = entry.value;
-      Object data = originalData;
-      if (logger.isDebugEnabled()) {
-        logger.debug("retrieved key={} {}", key, entry);
-      }
-      boolean addedToValues = false;
-      try {
-        boolean isObject = entry.isObject;
-        VersionTag versionTag = entry.versionTag;
-        keyNotPresent = entry.keyNotPresent;
 
-        if (postAuthzRequest != null) {
+        Object key;
+        boolean keyNotPresent = false;
+        if (keys != null) {
+          key = keys[i];
+        } else {
+          key = allKeysIter.next();
+        }
+        if (isDebugEnabled) {
+          logger.debug("{}: Getting value for key={}", servConn.getName(), key);
+        }
+        // Determine if the user authorized to get this key
+        GetOperationContext getContext = null;
+        if (authzRequest != null) {
           try {
-            getContext = postAuthzRequest.getAuthorize(regionName, key, data,
-                isObject, getContext);
-            GetOperationContextImpl gci = (GetOperationContextImpl) getContext;
-            Object newData = gci.getRawValue();
-            if (newData != data) {
-              // user changed the value
-              isObject = getContext.isObject();
-              data = newData;
+            getContext = authzRequest.getAuthorize(regionName, key, null);
+            if (isDebugEnabled) {
+              logger.debug("{}: Passed GET pre-authorization for key={}", servConn.getName(), key);
             }
           } catch (NotAuthorizedException ex) {
-            logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1,
-                new Object[]{servConn.getName(), key}), ex);
+            logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {
+              servConn.getName(),
+              key
+            }), ex);
             values.addExceptionPart(key, ex);
             continue;
-          } finally {
-            if (getContext != null) {
-              ((GetOperationContextImpl)getContext).release();
-            }
           }
         }
 
-        data = GeodeSecurityUtil.postProcess(regionName, key, data);
+        try {
+          GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+        } catch (NotAuthorizedException ex) {
+          logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {
+            servConn.getName(),
+            key
+          }), ex);
+          values.addExceptionPart(key, ex);
+          continue;
+        }
 
-        // Add the entry to the list that will be returned to the client
-        if (keyNotPresent) {
-          values.addObjectPartForAbsentKey(key, data, versionTag);
-          addedToValues = true;
-        } else {
-          values.addObjectPart(key, data, isObject, versionTag);
-          addedToValues = true;
+        // Get the value and update the statistics. Do not deserialize
+        // the value if it is a byte[].
+        // Getting a value in serialized form is pretty nasty. I split this out
+        // so the logic can be re-used by the CacheClientProxy.
+        Get70.Entry entry = request.getEntry(region, key, null, servConn);
+        @Retained
+        final Object originalData = entry.value;
+        Object data = originalData;
+        if (logger.isDebugEnabled()) {
+          logger.debug("retrieved key={} {}", key, entry);
         }
-      } finally {
-        if (!addedToValues || data != originalData) {
-          OffHeapHelper.release(originalData);
+        boolean addedToValues = false;
+        try {
+          boolean isObject = entry.isObject;
+          VersionTag versionTag = entry.versionTag;
+          keyNotPresent = entry.keyNotPresent;
+
+          if (postAuthzRequest != null) {
+            try {
+              getContext = postAuthzRequest.getAuthorize(regionName, key, data, isObject, getContext);
+              GetOperationContextImpl gci = (GetOperationContextImpl) getContext;
+              Object newData = gci.getRawValue();
+              if (newData != data) {
+                // user changed the value
+                isObject = getContext.isObject();
+                data = newData;
+              }
+            } catch (NotAuthorizedException ex) {
+              logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {
+                servConn.getName(),
+                key
+              }), ex);
+              values.addExceptionPart(key, ex);
+              continue;
+            } finally {
+              if (getContext != null) {
+                ((GetOperationContextImpl) getContext).release();
+              }
+            }
+          }
+
+          data = GeodeSecurityUtil.postProcess(regionName, key, data);
+
+          // Add the entry to the list that will be returned to the client
+          if (keyNotPresent) {
+            values.addObjectPartForAbsentKey(key, data, versionTag);
+            addedToValues = true;
+          } else {
+            values.addObjectPart(key, data, isObject, versionTag);
+            addedToValues = true;
+          }
+        } finally {
+          if (!addedToValues || data != originalData) {
+            OffHeapHelper.release(originalData);
+          }
         }
       }
-    }
 
-    // Send the last chunk even if the list is of zero size.
-    if (Version.GFE_701.compareTo(servConn.getClientVersion()) <= 0) {
-      // 7.0.1 and later clients do not expect the keys in the response
-      values.setKeys(null);
-    }
-    sendGetAllResponseChunk(region, values, true, servConn);
-    servConn.setAsTrue(RESPONDED);
+      // Send the last chunk even if the list is of zero size.
+      if (Version.GFE_701.compareTo(servConn.getClientVersion()) <= 0) {
+        // 7.0.1 and later clients do not expect the keys in the response
+        values.setKeys(null);
+      }
+      sendGetAllResponseChunk(region, values, true, servConn);
+      servConn.setAsTrue(RESPONDED);
     } finally {
       values.release();
     }
   }
 
 
-  private static void sendGetAllResponseChunk(Region region, ObjectPartList list,
-                                              boolean lastChunk, ServerConnection servConn) throws IOException {
+  private static void sendGetAllResponseChunk(Region region,
+                                              ObjectPartList list,
+                                              boolean lastChunk,
+                                              ServerConnection servConn) throws IOException {
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     chunkedResponseMsg.setNumberOfParts(1);
     chunkedResponseMsg.setLastChunk(lastChunk);
     chunkedResponseMsg.addObjPartNoCopying(list);
 
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sending {} getAll response chunk for region={}{}", servConn.getName(), (lastChunk ? " last " : " "), region.getFullPath(), (logger.isTraceEnabled()? " values=" + list + " chunk=<" + chunkedResponseMsg + ">" : ""));
+      logger.debug("{}: Sending {} getAll response chunk for region={}{}", servConn.getName(), (lastChunk ? " last " : " "), region
+        .getFullPath(), (logger.isTraceEnabled() ? " values=" + list + " chunk=<" + chunkedResponseMsg + ">" : ""));
     }
 
     chunkedResponseMsg.sendChunk(servConn);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAllWithCallback.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAllWithCallback.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAllWithCallback.java
index 630163b..42a871a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAllWithCallback.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAllWithCallback.java
@@ -24,10 +24,7 @@ import org.apache.logging.log4j.Logger;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.operations.GetOperationContext;
 import com.gemstone.gemfire.cache.operations.internal.GetOperationContextImpl;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
@@ -38,7 +35,6 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ObjectPartList;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.cache.tier.sockets.VersionedObjectList;
-import com.gemstone.gemfire.internal.cache.tier.sockets.command.Get70.Entry;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
@@ -47,6 +43,7 @@ import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
 import com.gemstone.gemfire.internal.offheap.annotations.Retained;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
 import com.gemstone.gemfire.internal.security.AuthorizeRequestPP;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.security.NotAuthorizedException;
 
 /**
@@ -134,34 +131,36 @@ public class GetAllWithCallback extends BaseCommand {
       writeChunkedErrorResponse(msg, MessageType.GET_ALL_DATA_ERROR, message,
               servConn);
       servConn.setAsTrue(RESPONDED);
-    } else {
-      LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = " was not found during getAll request";
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      } else {
-        // Send header
-        ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
+      return;
+    }
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = " was not found during getAll request";
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+    // Send header
+    ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
+    chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
+    chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+    chunkedResponseMsg.sendHeader();
 
-        // Send chunk response
-        try {
-          fillAndSendGetAllResponseChunks(region, regionName, keys, servConn, callback);
-          servConn.setAsTrue(RESPONDED);
-        } catch (Exception e) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, e);
+    // Send chunk response
+    try {
+      fillAndSendGetAllResponseChunks(region, regionName, keys, servConn, callback);
+      servConn.setAsTrue(RESPONDED);
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
 
-          // Otherwise, write an exception message and continue
-          writeChunkedException(msg, e, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
-      }
+      // Otherwise, write an exception message and continue
+      writeChunkedException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
     }
+
+
   }
 
   private void fillAndSendGetAllResponseChunks(Region region,
@@ -205,6 +204,17 @@ public class GetAllWithCallback extends BaseCommand {
         }
       }
 
+      try {
+        GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+      } catch (NotAuthorizedException ex) {
+        logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {
+          servConn.getName(),
+          key
+        }), ex);
+        values.addExceptionPart(key, ex);
+        continue;
+      }
+
       // Get the value and update the statistics. Do not deserialize
       // the value if it is a byte[].
       // Getting a value in serialized form is pretty nasty. I split this out

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetFunctionAttribute.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetFunctionAttribute.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetFunctionAttribute.java
index 1ea5919..871a80b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetFunctionAttribute.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetFunctionAttribute.java
@@ -26,6 +26,7 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 public class GetFunctionAttribute extends BaseCommand {
 
@@ -50,24 +51,26 @@ public class GetFunctionAttribute extends BaseCommand {
       sendError(msg, message, servConn);
       return;
     }
-    else {
-      Function function = FunctionService.getFunction(functionId);
-      if (function == null) {
-        String message = null;
-        message = LocalizedStrings.GetFunctionAttribute_THE_FUNCTION_IS_NOT_REGISTERED_FOR_FUNCTION_ID_0
-            .toLocalizedString(functionId);
-        logger.warn("{}: {}", servConn.getName(), message);
-        sendError(msg, message, servConn);
-        return;
-      }
-      else {
-        byte[] functionAttributes = new byte[3];
-        functionAttributes[0] = (byte)(function.hasResult() ? 1 : 0);
-        functionAttributes[1] = (byte)(function.isHA() ? 1 : 0);
-        functionAttributes[2] = (byte)(function.optimizeForWrite() ? 1 : 0);
-        writeResponseWithFunctionAttribute(functionAttributes, msg, servConn);
-      }
+
+    Function function = FunctionService.getFunction(functionId);
+    if (function == null) {
+      String message = null;
+      message = LocalizedStrings.GetFunctionAttribute_THE_FUNCTION_IS_NOT_REGISTERED_FOR_FUNCTION_ID_0
+          .toLocalizedString(functionId);
+      logger.warn("{}: {}", servConn.getName(), message);
+      sendError(msg, message, servConn);
+      return;
     }
+
+    GeodeSecurityUtil.authorizeFunctionRead(functionId);
+
+    byte[] functionAttributes = new byte[3];
+    functionAttributes[0] = (byte)(function.hasResult() ? 1 : 0);
+    functionAttributes[1] = (byte)(function.isHA() ? 1 : 0);
+    functionAttributes[2] = (byte)(function.optimizeForWrite() ? 1 : 0);
+    writeResponseWithFunctionAttribute(functionAttributes, msg, servConn);
+
+
   }
 
   private void sendError(Message msg, String message, ServerConnection servConn)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Invalidate.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Invalidate.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Invalidate.java
index ee64872..44a6f7e 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Invalidate.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Invalidate.java
@@ -114,112 +114,106 @@ public class Invalidate extends BaseCommand {
       }
       writeErrorResponse(msg, MessageType.DESTROY_DATA_ERROR, errMessage.toString(), servConn);
       servConn.setAsTrue(RESPONDED);
-    } else {
-      LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = LocalizedStrings.BaseCommand__0_WAS_NOT_FOUND_DURING_1_REQUEST.toLocalizedString(regionName, "invalidate");
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      } else {
-        // Invalidate the entry
-        ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
-        long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-        long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-        EventID eventId = new EventID(servConn.getEventMemberIDByteArray(), threadId, sequenceId);
-
-        Breadcrumbs.setEventId(eventId);
-
-        VersionTag tag = null;
-
-        // for integrated security
-        GeodeSecurityUtil.authorizeRegionWrite(regionName, key.toString());
-
-        try {
-          /*
-           * 
-           * txtodo: doesn't seem like there is any notion of authzInvalidate
-           */
-          AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-          if (authzRequest != null) {
-            InvalidateOperationContext invalidateContext = authzRequest.invalidateAuthorize(regionName, key, callbackArg);
-            callbackArg = invalidateContext.getCallbackArg();
-          }
-          EventIDHolder clientEvent = new EventIDHolder(eventId);
-
-          // msg.isRetry might be set by v7.0 and later clients
-          if (msg.isRetry()) {
-            //            if (logger.isDebugEnabled()) {
-            //              logger.debug("DEBUG: encountered isRetry in Invalidate");
-            //            }
-            clientEvent.setPossibleDuplicate(true);
-            if (region.getAttributes().getConcurrencyChecksEnabled()) {
-              // recover the version tag from other servers
-              clientEvent.setRegion(region);
-              if (!recoverVersionTagForRetriedOperation(clientEvent)) {
-                clientEvent.setPossibleDuplicate(false); // no-one has seen this event
-              }
-            }
-          }
+      return;
+    }
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = LocalizedStrings.BaseCommand__0_WAS_NOT_FOUND_DURING_1_REQUEST.toLocalizedString(regionName, "invalidate");
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+    // Invalidate the entry
+    ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
+    long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    EventID eventId = new EventID(servConn.getEventMemberIDByteArray(), threadId, sequenceId);
 
-          region.basicBridgeInvalidate(key, callbackArg, servConn.getProxyID(), true, clientEvent);
-          tag = clientEvent.getVersionTag();
-          servConn.setModificationInfo(true, regionName, key);
-        } catch (EntryNotFoundException e) {
-          // Don't send an exception back to the client if this
-          // exception happens. Just log it and continue.
-          logger.info(LocalizedMessage.create(LocalizedStrings.BaseCommand_DURING_0_NO_ENTRY_WAS_FOUND_FOR_KEY_1, new Object[] {
-            "invalidate",
-            key
-          }));
-        } catch (RegionDestroyedException rde) {
-          writeException(msg, rde, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        } catch (Exception e) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, e);
-
-          // If an exception occurs during the destroy, preserve the connection
-          writeException(msg, e, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          if (e instanceof GemFireSecurityException) {
-            // Fine logging for security exceptions since these are already
-            // logged by the security logger
-            if (logger.isDebugEnabled()) {
-              logger.debug("{}: Unexpected Security exception", servConn.getName(), e);
-            }
-          } else {
-            logger.warn(LocalizedMessage.create(LocalizedStrings.BaseCommand_0_UNEXPECTED_EXCEPTION, servConn.getName()), e);
-          }
-          return;
-        }
+    Breadcrumbs.setEventId(eventId);
 
-        // Update the statistics and write the reply
-        {
-          long oldStart = start;
-          start = DistributionStats.getStatTime();
-          stats.incProcessInvalidateTime(start - oldStart);
-        }
-        if (region instanceof PartitionedRegion) {
-          PartitionedRegion pr = (PartitionedRegion) region;
-          if (pr.isNetworkHop() != (byte) 0) {
-            writeReplyWithRefreshMetadata(msg, servConn, pr, pr.isNetworkHop(), tag);
-            pr.setIsNetworkHop((byte) 0);
-            pr.setMetadataVersion(Byte.valueOf((byte) 0));
-          } else {
-            writeReply(msg, servConn, tag);
+    VersionTag tag = null;
+
+    // for integrated security
+    GeodeSecurityUtil.authorizeRegionWrite(regionName, key.toString());
+
+    try {
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      if (authzRequest != null) {
+        InvalidateOperationContext invalidateContext = authzRequest.invalidateAuthorize(regionName, key, callbackArg);
+        callbackArg = invalidateContext.getCallbackArg();
+      }
+      EventIDHolder clientEvent = new EventIDHolder(eventId);
+
+      // msg.isRetry might be set by v7.0 and later clients
+      if (msg.isRetry()) {
+        //            if (logger.isDebugEnabled()) {
+        //              logger.debug("DEBUG: encountered isRetry in Invalidate");
+        //            }
+        clientEvent.setPossibleDuplicate(true);
+        if (region.getAttributes().getConcurrencyChecksEnabled()) {
+          // recover the version tag from other servers
+          clientEvent.setRegion(region);
+          if (!recoverVersionTagForRetriedOperation(clientEvent)) {
+            clientEvent.setPossibleDuplicate(false); // no-one has seen this event
           }
-        } else {
-          writeReply(msg, servConn, tag);
         }
-        servConn.setAsTrue(RESPONDED);
+      }
+
+      region.basicBridgeInvalidate(key, callbackArg, servConn.getProxyID(), true, clientEvent);
+      tag = clientEvent.getVersionTag();
+      servConn.setModificationInfo(true, regionName, key);
+    } catch (EntryNotFoundException e) {
+      // Don't send an exception back to the client if this
+      // exception happens. Just log it and continue.
+      logger.info(LocalizedMessage.create(LocalizedStrings.BaseCommand_DURING_0_NO_ENTRY_WAS_FOUND_FOR_KEY_1, new Object[] {
+        "invalidate", key
+      }));
+    } catch (RegionDestroyedException rde) {
+      writeException(msg, rde, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
+
+      // If an exception occurs during the destroy, preserve the connection
+      writeException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      if (e instanceof GemFireSecurityException) {
+        // Fine logging for security exceptions since these are already
+        // logged by the security logger
         if (logger.isDebugEnabled()) {
-          logger.debug("{}: Sent invalidate response for region {} key {}", servConn.getName(), regionName, key);
+          logger.debug("{}: Unexpected Security exception", servConn.getName(), e);
         }
-        stats.incWriteInvalidateResponseTime(DistributionStats.getStatTime() - start);
+      } else {
+        logger.warn(LocalizedMessage.create(LocalizedStrings.BaseCommand_0_UNEXPECTED_EXCEPTION, servConn.getName()), e);
       }
+      return;
     }
 
+    // Update the statistics and write the reply
+    {
+      long oldStart = start;
+      start = DistributionStats.getStatTime();
+      stats.incProcessInvalidateTime(start - oldStart);
+    }
+    if (region instanceof PartitionedRegion) {
+      PartitionedRegion pr = (PartitionedRegion) region;
+      if (pr.isNetworkHop() != (byte) 0) {
+        writeReplyWithRefreshMetadata(msg, servConn, pr, pr.isNetworkHop(), tag);
+        pr.setIsNetworkHop((byte) 0);
+        pr.setMetadataVersion(Byte.valueOf((byte) 0));
+      } else {
+        writeReply(msg, servConn, tag);
+      }
+    } else {
+      writeReply(msg, servConn, tag);
+    }
+    servConn.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Sent invalidate response for region {} key {}", servConn.getName(), regionName, key);
+    }
+    stats.incWriteInvalidateResponseTime(DistributionStats.getStatTime() - start);
   }
 
   protected void writeReply(Message origMsg, ServerConnection servConn, VersionTag tag) throws IOException {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/KeySet.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/KeySet.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/KeySet.java
index 8da7af9..0163be7 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/KeySet.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/KeySet.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
@@ -56,8 +56,7 @@ public class KeySet extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException, InterruptedException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException, InterruptedException {
     Part regionNamePart = null;
     String regionName = null;
     CachedRegionHelper crHelper = servConn.getCachedRegionHelper();
@@ -70,84 +69,86 @@ public class KeySet extends BaseCommand {
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     final boolean isDebugEnabled = logger.isDebugEnabled();
     if (isDebugEnabled) {
-      logger.debug("{}: Received key set request ({} bytes) from {} for region {}", servConn.getName(), msg.getPayloadLength(), servConn.getSocketString(), regionName);
+      logger.debug("{}: Received key set request ({} bytes) from {} for region {}", servConn.getName(), msg.getPayloadLength(), servConn
+        .getSocketString(), regionName);
     }
 
     // Process the key set request
     if (regionName == null) {
       String message = null;
-//      if (regionName == null) (can only be null) 
+      //      if (regionName == null) (can only be null)
       {
-        message = LocalizedStrings.KeySet_0_THE_INPUT_REGION_NAME_FOR_THE_KEY_SET_REQUEST_IS_NULL.toLocalizedString(servConn.getName());
-        logger.warn(LocalizedMessage.create(LocalizedStrings.KeySet_0_THE_INPUT_REGION_NAME_FOR_THE_KEY_SET_REQUEST_IS_NULL, servConn.getName()));
+        message = LocalizedStrings.KeySet_0_THE_INPUT_REGION_NAME_FOR_THE_KEY_SET_REQUEST_IS_NULL.toLocalizedString(servConn
+          .getName());
+        logger.warn(LocalizedMessage.create(LocalizedStrings.KeySet_0_THE_INPUT_REGION_NAME_FOR_THE_KEY_SET_REQUEST_IS_NULL, servConn
+          .getName()));
       }
-      writeKeySetErrorResponse(msg, MessageType.KEY_SET_DATA_ERROR, message,
-          servConn);
+      writeKeySetErrorResponse(msg, MessageType.KEY_SET_DATA_ERROR, message, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = LocalizedStrings.KeySet__0_WAS_NOT_FOUND_DURING_KEY_SET_REQUEST.toLocalizedString(regionName);
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = LocalizedStrings.KeySet__0_WAS_NOT_FOUND_DURING_KEY_SET_REQUEST.toLocalizedString(regionName);
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
+
+    GeodeSecurityUtil.authorizeRegionRead(regionName);
+
+    KeySetOperationContext keySetContext = null;
+    AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+    if (authzRequest != null) {
+      try {
+        keySetContext = authzRequest.keySetAuthorize(regionName);
+      } catch (NotAuthorizedException ex) {
+        writeChunkedException(msg, ex, false, servConn);
         servConn.setAsTrue(RESPONDED);
-      }
-      else {
-        KeySetOperationContext keySetContext = null;
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        if (authzRequest != null) {
-          try {
-            keySetContext = authzRequest.keySetAuthorize(regionName);
-          }
-          catch (NotAuthorizedException ex) {
-            writeChunkedException(msg, ex, false, servConn);
-            servConn.setAsTrue(RESPONDED);
-            return;
-          }
-        }
-        // Update the statistics and write the reply
-        // bserverStats.incLong(processDestroyTimeId,
-        // DistributionStats.getStatTime() - start);
-        // start = DistributionStats.getStatTime();
-
-        // Send header
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
-
-        // Send chunk response
-        try {
-          fillAndSendKeySetResponseChunks(region, regionName, keySetContext,
-              servConn);
-          servConn.setAsTrue(RESPONDED);
-        }
-        catch (Exception e) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, e);
-
-          // Otherwise, write an exception message and continue
-          writeChunkedException(msg, e, false, servConn, servConn
-              .getChunkedResponseMessage());
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
-
-        if (isDebugEnabled) {
-          // logger.fine(getName() + ": Sent chunk (1 of 1) of register interest
-          // response (" + chunkedResponseMsg.getBufferLength() + " bytes) for
-          // region " + regionName + " key " + key);
-          logger.debug("{}: Sent key set response for the region {}", servConn.getName(), regionName);
-        }
-        // bserverStats.incLong(writeDestroyResponseTimeId,
-        // DistributionStats.getStatTime() - start);
-        // bserverStats.incInt(destroyResponsesId, 1);
+        return;
       }
     }
+    // Update the statistics and write the reply
+    // bserverStats.incLong(processDestroyTimeId,
+    // DistributionStats.getStatTime() - start);
+    // start = DistributionStats.getStatTime();
+
+    // Send header
+    chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
+    chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+    chunkedResponseMsg.sendHeader();
+
+    // Send chunk response
+    try {
+      fillAndSendKeySetResponseChunks(region, regionName, keySetContext, servConn);
+      servConn.setAsTrue(RESPONDED);
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
+
+      // Otherwise, write an exception message and continue
+      writeChunkedException(msg, e, false, servConn, servConn.getChunkedResponseMessage());
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    if (isDebugEnabled) {
+      // logger.fine(getName() + ": Sent chunk (1 of 1) of register interest
+      // response (" + chunkedResponseMsg.getBufferLength() + " bytes) for
+      // region " + regionName + " key " + key);
+      logger.debug("{}: Sent key set response for the region {}", servConn.getName(), regionName);
+    }
+    // bserverStats.incLong(writeDestroyResponseTimeId,
+    // DistributionStats.getStatTime() - start);
+    // bserverStats.incInt(destroyResponsesId, 1);
+
   }
 
   private void fillAndSendKeySetResponseChunks(LocalRegion region,
-      String regionName, KeySetOperationContext context,
-      ServerConnection servConn) throws IOException {
+                                               String regionName,
+                                               KeySetOperationContext context,
+                                               ServerConnection servConn) throws IOException {
 
     // Get the key set
     Set keySet = region.keys();
@@ -156,19 +157,18 @@ public class KeySet extends BaseCommand {
     // Post-operation filtering
     AuthorizeRequestPP postAuthzRequest = servConn.getPostAuthzRequest();
     if (postAuthzRequest != null) {
-      keySetContext = postAuthzRequest.keySetAuthorize(regionName, keySet,
-          keySetContext);
+      keySetContext = postAuthzRequest.keySetAuthorize(regionName, keySet, keySetContext);
       keySet = keySetContext.getKeySet();
     }
 
     List keyList = new ArrayList(maximumChunkSize);
     final boolean isTraceEnabled = logger.isTraceEnabled();
-    for (Iterator it = keySet.iterator(); it.hasNext();) {
+    for (Iterator it = keySet.iterator(); it.hasNext(); ) {
       Object entryKey = it.next();
-      GeodeSecurityUtil.authorizeRegionRead(regionName, entryKey.toString());
       keyList.add(entryKey);
       if (isTraceEnabled) {
-        logger.trace("{}: fillAndSendKeySetResponseKey <{}>; list size was {}; region: {}", servConn.getName(), entryKey, keyList.size(), region.getFullPath());
+        logger.trace("{}: fillAndSendKeySetResponseKey <{}>; list size was {}; region: {}", servConn.getName(), entryKey, keyList
+          .size(), region.getFullPath());
       }
       if (keyList.size() == maximumChunkSize) {
         // Send the chunk and clear the list
@@ -180,8 +180,8 @@ public class KeySet extends BaseCommand {
     sendKeySetResponseChunk(region, keyList, true, servConn);
   }
 
-  private static void sendKeySetResponseChunk(Region region, List list,
-      boolean lastChunk, ServerConnection servConn) throws IOException {
+  private static void sendKeySetResponseChunk(Region region, List list, boolean lastChunk, ServerConnection servConn)
+    throws IOException {
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
 
     chunkedResponseMsg.setNumberOfParts(1);
@@ -189,7 +189,8 @@ public class KeySet extends BaseCommand {
     chunkedResponseMsg.addObjPart(list, zipValues);
 
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sending {} key set response chunk for region={}{}", servConn.getName(), (lastChunk ? " last " : " "), region.getFullPath(), (logger.isTraceEnabled() ? " keys=" + list + " chunk=<" + chunkedResponseMsg + ">" : ""));
+      logger.debug("{}: Sending {} key set response chunk for region={}{}", servConn.getName(), (lastChunk ? " last " : " "), region
+        .getFullPath(), (logger.isTraceEnabled() ? " keys=" + list + " chunk=<" + chunkedResponseMsg + ">" : ""));
     }
 
     chunkedResponseMsg.sendChunk(servConn);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Put61.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Put61.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Put61.java
index 4cadd20..48e2a01 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Put61.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Put61.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
@@ -28,7 +28,6 @@ import com.gemstone.gemfire.cache.RegionDestroyedException;
 import com.gemstone.gemfire.cache.ResourceException;
 import com.gemstone.gemfire.cache.operations.PutOperationContext;
 import com.gemstone.gemfire.distributed.internal.DistributionStats;
-import com.gemstone.gemfire.internal.cache.EntryEventImpl;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.EventIDHolder;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
@@ -44,6 +43,7 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.security.GemFireSecurityException;
 
 /**
@@ -62,7 +62,7 @@ public class Put61 extends BaseCommand {
 
   @Override
   public void cmdExecute(Message msg, ServerConnection servConn, long p_start)
-      throws IOException, InterruptedException {
+    throws IOException, InterruptedException {
     long start = p_start;
     Part regionNamePart = null, keyPart = null, valuePart = null, callbackArgPart = null;
     String regionName = null;
@@ -76,11 +76,9 @@ public class Put61 extends BaseCommand {
       boolean interrupted = Thread.interrupted();
       try {
         Thread.sleep(crHelper.emulateSlowServer());
-      }
-      catch (InterruptedException ugh) {
+      } catch (InterruptedException ugh) {
         interrupted = true;
-      }
-      finally {
+      } finally {
         if (interrupted) {
           Thread.currentThread().interrupt();
         }
@@ -98,9 +96,8 @@ public class Put61 extends BaseCommand {
     regionNamePart = msg.getPart(0);
     keyPart = msg.getPart(1);
     try {
-      isDelta = (Boolean)msg.getPart(2).getObject();
-    }
-    catch (Exception e) {
+      isDelta = (Boolean) msg.getPart(2).getObject();
+    } catch (Exception e) {
       writeException(msg, MessageType.PUT_DELTA_ERROR, e, false, servConn);
       servConn.setAsTrue(RESPONDED);
       // CachePerfStats not available here.
@@ -112,8 +109,7 @@ public class Put61 extends BaseCommand {
       callbackArgPart = msg.getPart(5);
       try {
         callbackArg = callbackArgPart.getObject();
-      }
-      catch (Exception e) {
+      } catch (Exception e) {
         writeException(msg, e, false, servConn);
         servConn.setAsTrue(RESPONDED);
         return;
@@ -123,8 +119,7 @@ public class Put61 extends BaseCommand {
 
     try {
       key = keyPart.getStringOrObject();
-    }
-    catch (Exception e) {
+    } catch (Exception e) {
       writeException(msg, e, false, servConn);
       servConn.setAsTrue(RESPONDED);
       return;
@@ -132,7 +127,8 @@ public class Put61 extends BaseCommand {
 
     final boolean isDebugEnabled = logger.isDebugEnabled();
     if (isDebugEnabled) {
-      logger.debug("{}: Received 6.1{}put request ({} bytes) from {} for region {} key {}", servConn.getName(), (isDelta ? " delta " : " "), msg.getPayloadLength(), servConn.getSocketString(), regionName, key);
+      logger.debug("{}: Received 6.1{}put request ({} bytes) from {} for region {} key {}", servConn.getName(), (isDelta ? " delta " : " "), msg
+        .getPayloadLength(), servConn.getSocketString(), regionName, key);
     }
 
     // Process the put request
@@ -151,169 +147,153 @@ public class Put61 extends BaseCommand {
         }
         errMessage.append(putMsg);
       }
-      writeErrorResponse(msg, MessageType.PUT_DATA_ERROR,
-          errMessage.toString(), servConn);
+      writeErrorResponse(msg, MessageType.PUT_DATA_ERROR, errMessage.toString(), servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = " was not found during 6.1 put request";
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      }
-      else if (valuePart.isNull() && region.containsKey(key)) {
-        // Invalid to 'put' a null value in an existing key
-        String putMsg = " Attempted to 6.1 put a null value for existing key "
-            + key;
-        if (isDebugEnabled) {
-          logger.debug("{}:{}", servConn.getName(), putMsg);
-        }
-        errMessage.append(putMsg);
-        writeErrorResponse(msg, MessageType.PUT_DATA_ERROR, errMessage
-            .toString(), servConn);
-        servConn.setAsTrue(RESPONDED);
+
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = " was not found during 6.1 put request";
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    if (valuePart.isNull() && region.containsKey(key)) {
+      // Invalid to 'put' a null value in an existing key
+      String putMsg = " Attempted to 6.1 put a null value for existing key " + key;
+      if (isDebugEnabled) {
+        logger.debug("{}:{}", servConn.getName(), putMsg);
       }
-      else {
-        // try {
-        // this.eventId = (EventID)eventPart.getObject();
-        ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart
-            .getSerializedForm());
-        long threadId = EventID
-            .readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-        long sequenceId = EventID
-            .readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-        EventID eventId = new EventID(servConn.getEventMemberIDByteArray(),
-            threadId, sequenceId);
+      errMessage.append(putMsg);
+      writeErrorResponse(msg, MessageType.PUT_DATA_ERROR, errMessage.toString(), servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
 
-        try {
-          Object value = null;
-          if (!isDelta) {
-            value = valuePart.getSerializedForm();
-          } 
-          boolean isObject = valuePart.isObject();
-          boolean isMetaRegion = region.isUsedForMetaRegion();
-          msg.setMetaRegion(isMetaRegion);
-          AuthorizeRequest authzRequest = null;
-          if (!isMetaRegion) {
-            authzRequest = servConn.getAuthzRequest();
-          }
-          if (authzRequest != null) {
-            // TODO SW: This is to handle DynamicRegionFactory create
-            // calls. Rework this when the semantics of DynamicRegionFactory are
-            // cleaned up.
-            if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-              authzRequest.createRegionAuthorize((String)key);
-            }
-            // Allow PUT operations on meta regions (bug #38961)
-            else {
-              PutOperationContext putContext = authzRequest.putAuthorize(
-                  regionName, key, value, isObject, callbackArg);
-              value = putContext.getValue();
-              isObject = putContext.isObject();
-              callbackArg = putContext.getCallbackArg();
-            }
-          }
-          // If the value is 1 byte and the byte represents null,
-          // attempt to create the entry. This test needs to be
-          // moved to DataSerializer or DataSerializer.NULL needs
-          // to be publicly accessible.
-          boolean result = false;
-          if (value == null && !isDelta) {
-            // Create the null entry. Since the value is null, the value of the
-            // isObject
-            // the true after null doesn't matter and is not used.
-            result = region.basicBridgeCreate(key, null, true, callbackArg,
-                servConn.getProxyID(), true, new EventIDHolder(eventId), false);
-          }
-          else {
-            // Put the entry
-            byte[] delta = null;
-            if (isDelta) {
-              delta = valuePart.getSerializedForm();              
-            }
-            result = region.basicBridgePut(key, value, delta, isObject,
-                callbackArg, servConn.getProxyID(), true, new EventIDHolder(eventId));
-          }
-          if (result) {
-            servConn.setModificationInfo(true, regionName, key);
-          }
-          else {
-            String message = servConn.getName()
-                + ": Failed to 6.1 put entry for region " + regionName
-                + " key " + key + " value " + valuePart;
-            if (isDebugEnabled) {
-              logger.debug(message);
-            }
-            throw new Exception(message);
-          }
-        }
-        catch (RegionDestroyedException rde) {
-          writeException(msg, rde, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
-        catch (ResourceException re) {
-          writeException(msg, re, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
-        catch (InvalidDeltaException ide) {
-          logger.info(LocalizedMessage.create(LocalizedStrings.UpdateOperation_ERROR_APPLYING_DELTA_FOR_KEY_0_OF_REGION_1, new Object[] { key, regionName }));
-          writeException(msg, MessageType.PUT_DELTA_ERROR, ide, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          region.getCachePerfStats().incDeltaFullValuesRequested();
-          return;
+    GeodeSecurityUtil.authorizeRegionWrite(regionName, key.toString());
 
-        }
-        catch (Exception ce) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, ce);
+    // try {
+    // this.eventId = (EventID)eventPart.getObject();
+    ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
+    long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    EventID eventId = new EventID(servConn.getEventMemberIDByteArray(), threadId, sequenceId);
 
-          // If an exception occurs during the put, preserve the connection
-          writeException(msg, ce, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          if (ce instanceof GemFireSecurityException) {
-            // Fine logging for security exceptions since these are already
-            // logged by the security logger
-            if (isDebugEnabled) {
-              logger.debug("{}: Unexpected Security exception", servConn.getName(), ce);
-            }
-          }
-          else if (isDebugEnabled) {
-            logger.debug("{}: Unexpected Exception", servConn.getName(), ce);
-          }
-          return;
+    try {
+      Object value = null;
+      if (!isDelta) {
+        value = valuePart.getSerializedForm();
+      }
+      boolean isObject = valuePart.isObject();
+      boolean isMetaRegion = region.isUsedForMetaRegion();
+      msg.setMetaRegion(isMetaRegion);
+      AuthorizeRequest authzRequest = null;
+      if (!isMetaRegion) {
+        authzRequest = servConn.getAuthzRequest();
+      }
+      if (authzRequest != null) {
+        // TODO SW: This is to handle DynamicRegionFactory create
+        // calls. Rework this when the semantics of DynamicRegionFactory are
+        // cleaned up.
+        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
+          authzRequest.createRegionAuthorize((String) key);
         }
-        finally {
-          long oldStart = start;
-          start = DistributionStats.getStatTime();
-          stats.incProcessPutTime(start - oldStart);
+        // Allow PUT operations on meta regions (bug #38961)
+        else {
+          PutOperationContext putContext = authzRequest.putAuthorize(regionName, key, value, isObject, callbackArg);
+          value = putContext.getValue();
+          isObject = putContext.isObject();
+          callbackArg = putContext.getCallbackArg();
         }
-
-        // Increment statistics and write the reply
-        if (region instanceof PartitionedRegion) {
-          PartitionedRegion pr = (PartitionedRegion)region;
-          if (pr.isNetworkHop() != (byte)0) {
-            writeReplyWithRefreshMetadata(msg, servConn,pr,pr.isNetworkHop());
-            pr.setIsNetworkHop((byte)0);
-            pr.setMetadataVersion(Byte.valueOf((byte)0));
-          }
-          else {
-            writeReply(msg, servConn);
-          }
+      }
+      // If the value is 1 byte and the byte represents null,
+      // attempt to create the entry. This test needs to be
+      // moved to DataSerializer or DataSerializer.NULL needs
+      // to be publicly accessible.
+      boolean result = false;
+      if (value == null && !isDelta) {
+        // Create the null entry. Since the value is null, the value of the
+        // isObject
+        // the true after null doesn't matter and is not used.
+        result = region.basicBridgeCreate(key, null, true, callbackArg, servConn.getProxyID(), true, new EventIDHolder(eventId), false);
+      } else {
+        // Put the entry
+        byte[] delta = null;
+        if (isDelta) {
+          delta = valuePart.getSerializedForm();
         }
-        else {
-          writeReply(msg, servConn);
+        result = region.basicBridgePut(key, value, delta, isObject, callbackArg, servConn.getProxyID(), true, new EventIDHolder(eventId));
+      }
+      if (result) {
+        servConn.setModificationInfo(true, regionName, key);
+      } else {
+        String message = servConn.getName() + ": Failed to 6.1 put entry for region " + regionName + " key " + key + " value " + valuePart;
+        if (isDebugEnabled) {
+          logger.debug(message);
         }
-        servConn.setAsTrue(RESPONDED);
+        throw new Exception(message);
+      }
+    } catch (RegionDestroyedException rde) {
+      writeException(msg, rde, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    } catch (ResourceException re) {
+      writeException(msg, re, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    } catch (InvalidDeltaException ide) {
+      logger.info(LocalizedMessage.create(LocalizedStrings.UpdateOperation_ERROR_APPLYING_DELTA_FOR_KEY_0_OF_REGION_1, new Object[] {
+        key,
+        regionName
+      }));
+      writeException(msg, MessageType.PUT_DELTA_ERROR, ide, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      region.getCachePerfStats().incDeltaFullValuesRequested();
+      return;
+
+    } catch (Exception ce) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, ce);
+
+      // If an exception occurs during the put, preserve the connection
+      writeException(msg, ce, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      if (ce instanceof GemFireSecurityException) {
+        // Fine logging for security exceptions since these are already
+        // logged by the security logger
         if (isDebugEnabled) {
-          logger.debug("{}: Sent 6.1 put response back to {} for region {} key {} value {}", servConn.getName(), servConn.getSocketString(), regionName, key, valuePart);
+          logger.debug("{}: Unexpected Security exception", servConn.getName(), ce);
         }
-        stats.incWritePutResponseTime(DistributionStats.getStatTime() - start);
+      } else if (isDebugEnabled) {
+        logger.debug("{}: Unexpected Exception", servConn.getName(), ce);
       }
+      return;
+    } finally {
+      long oldStart = start;
+      start = DistributionStats.getStatTime();
+      stats.incProcessPutTime(start - oldStart);
     }
 
+    // Increment statistics and write the reply
+    if (region instanceof PartitionedRegion) {
+      PartitionedRegion pr = (PartitionedRegion) region;
+      if (pr.isNetworkHop() != (byte) 0) {
+        writeReplyWithRefreshMetadata(msg, servConn, pr, pr.isNetworkHop());
+        pr.setIsNetworkHop((byte) 0);
+        pr.setMetadataVersion(Byte.valueOf((byte) 0));
+      } else {
+        writeReply(msg, servConn);
+      }
+    } else {
+      writeReply(msg, servConn);
+    }
+    servConn.setAsTrue(RESPONDED);
+    if (isDebugEnabled) {
+      logger.debug("{}: Sent 6.1 put response back to {} for region {} key {} value {}", servConn.getName(), servConn.getSocketString(), regionName, key, valuePart);
+    }
+    stats.incWritePutResponseTime(DistributionStats.getStatTime() - start);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Put65.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Put65.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Put65.java
index 156fbbc..960958f 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Put65.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Put65.java
@@ -209,6 +209,7 @@ public class Put65 extends BaseCommand {
       return;
     }
 
+    GeodeSecurityUtil.authorizeRegionWrite(regionName, key.toString());
 
     ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
     long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
@@ -269,8 +270,6 @@ public class Put65 extends BaseCommand {
         logger.debug("processing put65 with operation={}", operation);
       }
 
-      GeodeSecurityUtil.authorizeRegionWrite(regionName, key.toString());
-
       // If the value is 1 byte and the byte represents null,
       // attempt to create the entry. This test needs to be
       // moved to DataSerializer or DataSerializer.NULL needs

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll.java
index 1f668e4..8b24f3e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll.java
@@ -19,7 +19,18 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import com.gemstone.gemfire.cache.DynamicRegionFactory;
+import com.gemstone.gemfire.cache.RegionDestroyedException;
+import com.gemstone.gemfire.cache.ResourceException;
+import com.gemstone.gemfire.cache.operations.PutAllOperationContext;
+import com.gemstone.gemfire.cache.operations.internal.UpdateOnlyMap;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
 import com.gemstone.gemfire.internal.cache.CachedDeserializableFactory;
 import com.gemstone.gemfire.internal.cache.EventID;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
@@ -28,22 +39,16 @@ import com.gemstone.gemfire.internal.cache.PutAllPartialResultException;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
-import com.gemstone.gemfire.cache.DynamicRegionFactory;
-import com.gemstone.gemfire.cache.RegionDestroyedException;
-import com.gemstone.gemfire.cache.ResourceException;
-import com.gemstone.gemfire.cache.operations.PutAllOperationContext;
-import com.gemstone.gemfire.cache.operations.internal.UpdateOnlyMap;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.LinkedHashMap;
-import java.util.Map;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 public class PutAll extends BaseCommand {
   
@@ -116,7 +121,9 @@ public class PutAll extends BaseCommand {
         servConn.setAsTrue(RESPONDED);
         return;
       }
-      
+
+      GeodeSecurityUtil.authorizeRegionWrite(regionName);
+
       // part 1: eventID
       eventPart = msg.getPart(1);
       ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart
@@ -194,22 +201,6 @@ public class PutAll extends BaseCommand {
             map = ((UpdateOnlyMap) map).getInternalMap();
           }
         }
-      } else {
-        // no auth, so update the map based on isObjectMap here
-        /*
-         Collection entries = map.entrySet();
-         Iterator iterator = entries.iterator();
-         Map.Entry mapEntry = null;
-         while (iterator.hasNext()) {
-         mapEntry = (Map.Entry)iterator.next();
-         Object currkey = mapEntry.getKey();
-         byte[] serializedValue = (byte[])mapEntry.getValue();
-         boolean isObject = ((Boolean)isObjectMap.get(currkey)).booleanValue();
-         if (isObject) {
-         map.put(currkey, CachedDeserializableFactory.create(serializedValue));
-         }
-         }
-         */
       }
       
       if (logger.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll70.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll70.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll70.java
index 4b13ecd..6e3aec9 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll70.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll70.java
@@ -51,6 +51,7 @@ import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 public class PutAll70 extends BaseCommand {
   
@@ -127,7 +128,9 @@ public class PutAll70 extends BaseCommand {
         servConn.setAsTrue(RESPONDED);
         return;
       }
-      
+
+      GeodeSecurityUtil.authorizeRegionWrite(regionName);
+
       // part 1: eventID
       eventPart = msg.getPart(1);
       ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll80.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll80.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll80.java
index 0c2a86c..06512ac 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll80.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/PutAll80.java
@@ -134,6 +134,7 @@ public class PutAll80 extends BaseCommand {
         servConn.setAsTrue(RESPONDED);
         return;
       }
+
       LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
       if (region == null) {
         String reason = " was not found during putAll request";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest.java
index 3cf4bf4..6381c7d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest.java
@@ -19,22 +19,28 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+
+import com.gemstone.gemfire.cache.DynamicRegionFactory;
+import com.gemstone.gemfire.cache.InterestResultPolicy;
+import com.gemstone.gemfire.cache.operations.RegisterInterestOperationContext;
+import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
+import com.gemstone.gemfire.internal.cache.tier.InterestType;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ChunkedMessage;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
-import com.gemstone.gemfire.cache.DynamicRegionFactory;
-import com.gemstone.gemfire.cache.InterestResultPolicy;
-import com.gemstone.gemfire.cache.operations.RegisterInterestOperationContext;
-import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
-import java.io.IOException;
-
 public class RegisterInterest extends BaseCommand {
 
   private final static RegisterInterest singleton = new RegisterInterest();
@@ -120,19 +126,6 @@ public class RegisterInterest extends BaseCommand {
       logger.debug("{}: Received register interest request ({} bytes) from {} for region {} key {}", servConn.getName(), msg.getPayloadLength(), servConn.getSocketString(), regionName, key);
     }
     
-    /*
-    AcceptorImpl acceptor = servConn.getAcceptor();
-    
-    //  Check if the Server is running in NotifyBySubscription=true mode.
-    if (!acceptor.getCacheClientNotifier().getNotifyBySubscription()) {
-      // This should have been taken care at the client.
-      String err = LocalizedStrings.RegisterInterest_INTEREST_REGISTRATION_IS_SUPPORTED_ONLY_FOR_SERVERS_WITH_NOTIFYBYSUBSCRIPTION_SET_TO_TRUE.toLocalizedString() ;
-      writeChunkedErrorResponse(msg, MessageType.REGISTER_INTEREST_DATA_ERROR,
-          err, servConn);
-      servConn.setAsTrue(RESPONDED);  return;
-    }
-    */
-    
     // Process the register interest request
     if (key == null || regionName == null) {
       StringId message = null;
@@ -149,8 +142,13 @@ public class RegisterInterest extends BaseCommand {
       return;
     }
 
-    // TODO: add security changes here
-    GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+    if(interestType == InterestType.REGULAR_EXPRESSION) {
+      GeodeSecurityUtil.authorizeRegionRead(regionName);
+    }
+    else {
+      GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+    }
+
 
     // input key not null
     LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);


[48/50] [abbrv] incubator-geode git commit: GEODE-1571: putting security checks in CQ related commands and add tests.

Posted by ji...@apache.org.
GEODE-1571: putting security checks in CQ related commands and add tests.

* redo the security used for functions.


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

Branch: refs/heads/feature/GEODE-1571
Commit: fa66c65c66617f20aa3a78ce76e1655d02e88642
Parents: a534931
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon Jun 27 13:26:44 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Jun 27 13:26:44 2016 -0700

----------------------------------------------------------------------
 .../tier/sockets/command/ExecuteFunction.java   |   2 +-
 .../tier/sockets/command/ExecuteFunction65.java |   2 +-
 .../tier/sockets/command/ExecuteFunction66.java |   2 +-
 .../sockets/command/ExecuteRegionFunction.java  |   2 +-
 .../command/ExecuteRegionFunction65.java        |   2 +-
 .../command/ExecuteRegionFunction66.java        |   2 +-
 .../command/ExecuteRegionFunctionSingleHop.java |   2 +-
 .../sockets/command/GetFunctionAttribute.java   |   4 +-
 .../internal/security/GeodeSecurityUtil.java    |   7 --
 .../gemfire/security/GeodePermission.java       |   6 +-
 ...ntegratedClientQueryAuthDistributedTest.java |  45 ---------
 .../cache/query/internal/cq/ClientCQImpl.java   |   3 +-
 .../cache/tier/sockets/command/CloseCQ.java     |  25 +++--
 .../cache/tier/sockets/command/GetCQStats.java  |  42 ++++----
 .../tier/sockets/command/GetDurableCQs.java     |  16 +--
 .../cache/tier/sockets/command/MonitorCQ.java   |  15 +--
 .../cache/tier/sockets/command/StopCQ.java      |  28 +++---
 ...ntegratedClientQueryAuthDistributedTest.java | 100 +++++++++++++++++++
 18 files changed, 173 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction.java
index 0f3bdec..1ad2c42 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction.java
@@ -130,7 +130,7 @@ public class ExecuteFunction extends BaseCommand {
           functionObject = (Function)function;
         }
 
-        GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+        GeodeSecurityUtil.authorizeDataWrite();
 
         FunctionStats stats = FunctionStats.getFunctionStats(functionObject.getId(), null);
         

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction65.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction65.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction65.java
index ff6cdd6..fdd33ac 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction65.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction65.java
@@ -146,7 +146,7 @@ public class ExecuteFunction65 extends BaseCommand {
         functionObject = (Function) function;
       }
 
-      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+      GeodeSecurityUtil.authorizeDataWrite();
 
       FunctionStats stats = FunctionStats.getFunctionStats(functionObject.getId(), null);
       // check if the caller is authorized to do this operation on server

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction66.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction66.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction66.java
index d5f3660..5ed8e00 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction66.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction66.java
@@ -186,7 +186,7 @@ public class ExecuteFunction66 extends BaseCommand {
         functionObject = (Function) function;
       }
 
-      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+      GeodeSecurityUtil.authorizeDataWrite();
 
       FunctionStats stats = FunctionStats.getFunctionStats(functionObject.getId(), null);
       // check if the caller is authorized to do this operation on server

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction.java
index 6889e32..7f37688 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction.java
@@ -152,7 +152,7 @@ public class ExecuteRegionFunction extends BaseCommand {
         functionObject = (Function) function;
       }
 
-      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+      GeodeSecurityUtil.authorizeDataWrite();
 
       // check if the caller is authorized to do this operation on server
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java
index e11787c..652c74e 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java
@@ -182,7 +182,7 @@ public class ExecuteRegionFunction65 extends BaseCommand {
         functionObject = (Function) function;
       }
 
-      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+      GeodeSecurityUtil.authorizeDataWrite();
 
       // check if the caller is authorized to do this operation on server
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction66.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction66.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction66.java
index 36285bf..7c81b8c 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction66.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction66.java
@@ -199,7 +199,7 @@ public class ExecuteRegionFunction66 extends BaseCommand {
         functionObject = (Function) function;
       }
 
-      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+      GeodeSecurityUtil.authorizeDataWrite();
 
       // check if the caller is authorized to do this operation on server
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunctionSingleHop.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunctionSingleHop.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunctionSingleHop.java
index 1bfe7de..fcbe47d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunctionSingleHop.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunctionSingleHop.java
@@ -200,7 +200,7 @@ public class ExecuteRegionFunctionSingleHop extends BaseCommand {
         functionObject = (Function) function;
       }
 
-      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+      GeodeSecurityUtil.authorizeDataWrite();
 
       // check if the caller is authorized to do this operation on server
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetFunctionAttribute.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetFunctionAttribute.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetFunctionAttribute.java
index 871a80b..1cf1ea4 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetFunctionAttribute.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetFunctionAttribute.java
@@ -62,15 +62,13 @@ public class GetFunctionAttribute extends BaseCommand {
       return;
     }
 
-    GeodeSecurityUtil.authorizeFunctionRead(functionId);
+    GeodeSecurityUtil.authorizeClusterRead();
 
     byte[] functionAttributes = new byte[3];
     functionAttributes[0] = (byte)(function.hasResult() ? 1 : 0);
     functionAttributes[1] = (byte)(function.isHA() ? 1 : 0);
     functionAttributes[2] = (byte)(function.optimizeForWrite() ? 1 : 0);
     writeResponseWithFunctionAttribute(functionAttributes, msg, servConn);
-
-
   }
 
   private void sendError(Message msg, String message, ServerConnection servConn)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
index 7cc7dbf..0ae3f7d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
@@ -237,13 +237,6 @@ public class GeodeSecurityUtil {
     authorize("DATA", "READ", regionName, key);
   }
 
-  public static void authorizeFunctionExec(String function){
-    authorize("FUNCTION", "EXEC", function);
-  }
-  public static void authorizeFunctionRead(String function){
-    authorize("FUNCTION", "READ", function);
-  }
-
   public static void authorize(String resource, String operation) {
     authorize(resource, operation, null);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/main/java/com/gemstone/gemfire/security/GeodePermission.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/security/GeodePermission.java b/geode-core/src/main/java/com/gemstone/gemfire/security/GeodePermission.java
index b1aad84..22b53b1 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/security/GeodePermission.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/security/GeodePermission.java
@@ -27,16 +27,14 @@ public class GeodePermission extends WildcardPermission {
   public enum Resource {
     NULL,
     CLUSTER,
-    DATA,
-    FUNCTION
+    DATA
   }
 
   public enum Operation {
     NULL,
     MANAGE,
     WRITE,
-    READ,
-    EXEC
+    READ
   }
 
   public Resource getResource() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
deleted file mode 100644
index 8651a2f..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
+++ /dev/null
@@ -1,45 +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.security;
-
-import com.gemstone.gemfire.cache.Cache;
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.client.Pool;
-import com.gemstone.gemfire.cache.client.PoolManager;
-import com.gemstone.gemfire.test.junit.categories.DistributedTest;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-@Category(DistributedTest.class)
-public class IntegratedClientQueryAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
-
-  @Test
-  public void testQuery(){
-    client1.invoke(()-> {
-      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
-
-      String query = "select * from /AuthRegion";
-      assertNotAuthorized(()->region.query(query), "DATA:READ:AuthRegion");
-
-      Pool pool = PoolManager.find(region);
-      assertNotAuthorized(()->pool.getQueryService().newQuery(query).execute(), "DATA:READ:AuthRegion");
-    });
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-cq/src/main/java/com/gemstone/gemfire/cache/query/internal/cq/ClientCQImpl.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/com/gemstone/gemfire/cache/query/internal/cq/ClientCQImpl.java b/geode-cq/src/main/java/com/gemstone/gemfire/cache/query/internal/cq/ClientCQImpl.java
index 7fbd9d0..f6de4ce 100644
--- a/geode-cq/src/main/java/com/gemstone/gemfire/cache/query/internal/cq/ClientCQImpl.java
+++ b/geode-cq/src/main/java/com/gemstone/gemfire/cache/query/internal/cq/ClientCQImpl.java
@@ -453,8 +453,7 @@ public class ClientCQImpl extends CqQueryImpl implements ClientCQ {
             if (securityLogWriter.warningEnabled()) {
               securityLogWriter.warning(LocalizedStrings.CqQueryImpl_EXCEPTION_WHILE_EXECUTING_CQ_EXCEPTION_0, ex, null);              
             }
-            throw new CqException(
-              LocalizedStrings.CqQueryImpl_GOT_SECURITY_EXCEPTION_WHILE_EXECUTING_CQ_ON_SERVER.toLocalizedString(), ex.getCause());  
+            throw new CqException(ex.getCause().getMessage(), ex.getCause());
           } else if(ex instanceof CqException) {
             throw (CqException)ex;
           } else {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseCQ.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseCQ.java b/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseCQ.java
index 2b25d89..0908783 100644
--- a/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseCQ.java
+++ b/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/CloseCQ.java
@@ -16,23 +16,24 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
-import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
-import com.gemstone.gemfire.internal.cache.tier.Command;
-import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
 import com.gemstone.gemfire.cache.query.CqException;
-import com.gemstone.gemfire.cache.query.CqQuery;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-import com.gemstone.gemfire.cache.query.internal.DefaultQueryService;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.cache.query.internal.cq.InternalCqQuery;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
+import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
+import com.gemstone.gemfire.internal.cache.tier.Command;
+import com.gemstone.gemfire.internal.cache.tier.MessageType;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 
 public class CloseCQ extends BaseCQCommand {
@@ -74,6 +75,8 @@ public class CloseCQ extends BaseCQCommand {
       return;
     }
 
+    GeodeSecurityUtil.authorizeDataManage();
+
     // Process CQ close request
     try {
       // Append Client ID to CQ name

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetCQStats.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetCQStats.java b/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetCQStats.java
index 5c51fd0..6b82913 100644
--- a/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetCQStats.java
+++ b/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetCQStats.java
@@ -17,16 +17,17 @@
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
 
+import java.io.IOException;
+
+import com.gemstone.gemfire.cache.query.internal.cq.CqService;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
-import com.gemstone.gemfire.cache.query.CqException;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-import com.gemstone.gemfire.cache.query.internal.DefaultQueryService;
-import com.gemstone.gemfire.cache.query.internal.cq.CqService;
-
-import java.io.IOException;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 
 public class GetCQStats extends BaseCQCommand {
@@ -68,22 +69,21 @@ public class GetCQStats extends BaseCQCommand {
       sendCqResponse(MessageType.CQDATAERROR_MSG_TYPE, err, msg
           .getTransactionId(), null, servConn);
       return;
+    }
 
+    GeodeSecurityUtil.authorizeClusterRead();
+    // Process the cq request
+    try {
+      // make sure the cqservice has been created
+      // since that is what registers the stats
+      CqService cqService = crHelper.getCache().getCqService();
+      cqService.start();
     }
-    else {
-      // Process the cq request
-      try {
-        // make sure the cqservice has been created
-        // since that is what registers the stats
-        CqService cqService = crHelper.getCache().getCqService();
-        cqService.start();
-      }
-      catch (Exception e) {
-        String err = "Exception while Getting the CQ Statistics. ";
-        sendCqResponse(MessageType.CQ_EXCEPTION_TYPE, err, msg
-            .getTransactionId(), e, servConn);
-        return;
-      }
+    catch (Exception e) {
+      String err = "Exception while Getting the CQ Statistics. ";
+      sendCqResponse(MessageType.CQ_EXCEPTION_TYPE, err, msg
+          .getTransactionId(), e, servConn);
+      return;
     }
     // Send OK to client
     sendCqResponse(MessageType.REPLY, "cq stats sent successfully.", msg

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetDurableCQs.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetDurableCQs.java b/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetDurableCQs.java
index f06d61f..dc1d461 100755
--- a/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetDurableCQs.java
+++ b/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetDurableCQs.java
@@ -20,30 +20,22 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Set;
 
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.operations.GetDurableCQsOperationContext;
 import com.gemstone.gemfire.cache.query.CqException;
-import com.gemstone.gemfire.cache.query.Query;
-import com.gemstone.gemfire.cache.query.internal.DefaultQuery;
 import com.gemstone.gemfire.cache.query.internal.DefaultQueryService;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
-import com.gemstone.gemfire.cache.query.internal.cq.InternalCqQuery;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
 import com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl;
-import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
 import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ChunkedMessage;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
-import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 
 public class GetDurableCQs extends BaseCQCommand {
@@ -74,15 +66,13 @@ public class GetDurableCQs extends BaseCQCommand {
 
     DefaultQueryService qService = null;
     CqService cqServiceForExec = null;
-    Query query = null;
-    Set cqRegionNames = null;
-    GetDurableCQsOperationContext getDurableCqsOperationContext = null;
-    InternalCqQuery cqQuery = null;
 
     try {
       qService = (DefaultQueryService) ((GemFireCacheImpl) crHelper.getCache())
           .getLocalQueryService();
 
+      GeodeSecurityUtil.authorizeClusterRead();
+
       // Authorization check
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();
       if (authzRequest != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/MonitorCQ.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/MonitorCQ.java b/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/MonitorCQ.java
index bf18dae..f9ca140 100644
--- a/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/MonitorCQ.java
+++ b/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/MonitorCQ.java
@@ -17,16 +17,17 @@
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
 
+import java.io.IOException;
+
+import com.gemstone.gemfire.cache.query.CqException;
+import com.gemstone.gemfire.cache.query.internal.cq.CqService;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
-import com.gemstone.gemfire.cache.query.CqException;
-import com.gemstone.gemfire.cache.query.internal.DefaultQueryService;
-import com.gemstone.gemfire.cache.query.internal.cq.CqService;
-
-import java.io.IOException;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 public class MonitorCQ extends BaseCQCommand {
 
@@ -73,6 +74,8 @@ public class MonitorCQ extends BaseCQCommand {
       logger.debug("{}: Received MonitorCq request from {} op: {}{}", servConn.getName(), servConn.getSocketString(), op, (regionName != null) ? " RegionName: " + regionName : "");
     }
 
+    GeodeSecurityUtil.authorizeClusterRead();
+
     try {
       CqService cqService = crHelper.getCache().getCqService();
       cqService.start();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/StopCQ.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/StopCQ.java b/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/StopCQ.java
index 1bdf352..9231bfc 100644
--- a/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/StopCQ.java
+++ b/geode-cq/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/StopCQ.java
@@ -16,25 +16,25 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
-import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
-import com.gemstone.gemfire.internal.cache.tier.Command;
-import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
 import com.gemstone.gemfire.cache.query.CqException;
-import com.gemstone.gemfire.cache.query.CqQuery;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-import com.gemstone.gemfire.cache.query.internal.DefaultQueryService;
 import com.gemstone.gemfire.cache.query.internal.cq.CqQueryImpl;
 import com.gemstone.gemfire.cache.query.internal.cq.CqService;
-import com.gemstone.gemfire.cache.query.internal.cq.CqServiceImpl;
 import com.gemstone.gemfire.cache.query.internal.cq.InternalCqQuery;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
+import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
+import com.gemstone.gemfire.internal.cache.tier.Command;
+import com.gemstone.gemfire.internal.cache.tier.MessageType;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ClientProxyMembershipID;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 
 public class StopCQ extends BaseCQCommand {
@@ -88,7 +88,9 @@ public class StopCQ extends BaseCQCommand {
         serverCqName = cqService.constructServerCqName(cqName, id);
       }
       InternalCqQuery cqQuery = cqService.getCq(serverCqName);
-      
+
+      GeodeSecurityUtil.authorizeDataManage();
+
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();
       if (authzRequest != null) {
         String queryStr = null;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fa66c65c/geode-cq/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-cq/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java b/geode-cq/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
new file mode 100644
index 0000000..a484160
--- /dev/null
+++ b/geode-cq/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
@@ -0,0 +1,100 @@
+/*
+ * 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.security;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.client.Pool;
+import com.gemstone.gemfire.cache.client.PoolManager;
+import com.gemstone.gemfire.cache.query.CqAttributes;
+import com.gemstone.gemfire.cache.query.CqAttributesFactory;
+import com.gemstone.gemfire.cache.query.CqQuery;
+import com.gemstone.gemfire.cache.query.QueryService;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+@Category(DistributedTest.class)
+public class IntegratedClientQueryAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
+
+  @Test
+  public void testQuery(){
+    client1.invoke(()-> {
+      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+
+      String query = "select * from /AuthRegion";
+      assertNotAuthorized(()->region.query(query), "DATA:READ:AuthRegion");
+
+      Pool pool = PoolManager.find(region);
+      assertNotAuthorized(()->pool.getQueryService().newQuery(query).execute(), "DATA:READ:AuthRegion");
+    });
+  }
+
+  @Test
+  public void testCQ(){
+    String query = "select * from /AuthRegion";
+    client1.invoke(()-> {
+      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      Pool pool = PoolManager.find(region);
+      QueryService qs = pool.getQueryService();
+
+      CqAttributes cqa = new CqAttributesFactory().create();
+
+      // Create the CqQuery
+      CqQuery cq = qs.newCq("CQ1", query, cqa);
+
+      assertNotAuthorized(()->cq.executeWithInitialResults(), "DATA:READ:AuthRegion");
+      assertNotAuthorized(()->cq.execute(), "DATA:READ:AuthRegion");
+
+      assertNotAuthorized(()->cq.close(), "DATA:MANAGE");
+    });
+
+    client2.invoke(()-> {
+      Cache cache = SecurityTestUtils.createCacheClient("authRegionReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      Pool pool = PoolManager.find(region);
+      QueryService qs = pool.getQueryService();
+
+      CqAttributes cqa = new CqAttributesFactory().create();
+      // Create the CqQuery
+      CqQuery cq = qs.newCq("CQ1", query, cqa);
+      cq.execute();
+
+      assertNotAuthorized(()->cq.stop(), "DATA:MANAGE");
+      assertNotAuthorized(()->qs.getAllDurableCqsFromServer(), "CLUSTER:READ");
+    });
+
+    client3.invoke(()-> {
+      Cache cache = SecurityTestUtils.createCacheClient("super-user", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      Pool pool = PoolManager.find(region);
+      QueryService qs = pool.getQueryService();
+
+      CqAttributes cqa = new CqAttributesFactory().create();
+
+      // Create the CqQuery
+      CqQuery cq = qs.newCq("CQ1", query, cqa);
+      cq.execute();
+
+      cq.stop();
+    });
+  }
+
+}


[41/50] [abbrv] incubator-geode git commit: GEODE-1751: putting security checks in all applicable client-server commands.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
index e6c946b..a54775d 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
@@ -19,25 +19,30 @@
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+
+import com.gemstone.gemfire.cache.DynamicRegionFactory;
+import com.gemstone.gemfire.cache.InterestResultPolicy;
+import com.gemstone.gemfire.cache.operations.RegisterInterestOperationContext;
+import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.InterestType;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientProxy;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ChunkedMessage;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.cache.vmotion.VMotionObserver;
 import com.gemstone.gemfire.internal.cache.vmotion.VMotionObserverHolder;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
-import com.gemstone.gemfire.cache.DynamicRegionFactory;
-import com.gemstone.gemfire.cache.InterestResultPolicy;
-import com.gemstone.gemfire.cache.operations.RegisterInterestOperationContext;
-import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
-
-import java.io.IOException;
 /**
  * @since GemFire 6.1
  */
@@ -153,19 +158,6 @@ public class RegisterInterest61 extends BaseCommand {
       VMotionObserver vmo = VMotionObserverHolder.getInstance();
       vmo.vMotionBeforeRegisterInterest();
     }
-
-    /*
-    AcceptorImpl acceptor = servConn.getAcceptor();
-    
-    //  Check if the Server is running in NotifyBySubscription=true mode.
-    if (!acceptor.getCacheClientNotifier().getNotifyBySubscription()) {
-      // This should have been taken care at the client.
-      String err = LocalizedStrings.RegisterInterest_INTEREST_REGISTRATION_IS_SUPPORTED_ONLY_FOR_SERVERS_WITH_NOTIFYBYSUBSCRIPTION_SET_TO_TRUE.toLocalizedString() ;
-      writeChunkedErrorResponse(msg, MessageType.REGISTER_INTEREST_DATA_ERROR,
-          err, servConn);
-      servConn.setAsTrue(RESPONDED);  return;
-    }
-    */
     
     // Process the register interest request
     if (key == null || regionName == null) {
@@ -183,10 +175,12 @@ public class RegisterInterest61 extends BaseCommand {
       return;
     }
 
-    if(interestType == InterestType.REGULAR_EXPRESSION)
+    if(interestType == InterestType.REGULAR_EXPRESSION) {
       GeodeSecurityUtil.authorizeRegionRead(regionName);
-    else
+    }
+    else {
       GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+    }
 
     // input key not null
     LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
index d04a585..b5c5221 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
@@ -127,6 +127,8 @@ public class RemoveAll extends BaseCommand {
         servConn.setAsTrue(RESPONDED);
         return;
       }
+
+      GeodeSecurityUtil.authorizeRegionWrite(regionName);
       
       // part 1: eventID
       eventPart = msg.getPart(1);
@@ -210,8 +212,6 @@ public class RemoveAll extends BaseCommand {
         servConn.setRequestSpecificTimeout(timeout);
       }
 
-      GeodeSecurityUtil.authorizeRegionWrite(regionName);
-
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();
       if (authzRequest != null) {
         // TODO SW: This is to handle DynamicRegionFactory create

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
index eeaf286..7d07cd5 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
@@ -15,22 +15,26 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+
+import com.gemstone.gemfire.cache.DynamicRegionFactory;
+import com.gemstone.gemfire.cache.operations.UnregisterInterestOperationContext;
+import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.cache.tier.Command;
+import com.gemstone.gemfire.internal.cache.tier.InterestType;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
 import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.security.NotAuthorizedException;
-import com.gemstone.gemfire.cache.DynamicRegionFactory;
-import com.gemstone.gemfire.cache.operations.UnregisterInterestOperationContext;
-import com.gemstone.gemfire.i18n.StringId;
-
-import java.io.IOException;
 
 
 public class UnregisterInterest extends BaseCommand {
@@ -46,7 +50,7 @@ public class UnregisterInterest extends BaseCommand {
 
   @Override
   public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws ClassNotFoundException, IOException {
+    throws ClassNotFoundException, IOException {
     Part regionNamePart = null, keyPart = null;
     String regionName = null;
     Object key = null;
@@ -58,68 +62,68 @@ public class UnregisterInterest extends BaseCommand {
     interestType = msg.getPart(1).getInt();
     keyPart = msg.getPart(2);
     Part isClosingPart = msg.getPart(3);
-    byte[] isClosingPartBytes = (byte[])isClosingPart.getObject();
+    byte[] isClosingPartBytes = (byte[]) isClosingPart.getObject();
     boolean isClosing = isClosingPartBytes[0] == 0x01;
     regionName = regionNamePart.getString();
     try {
       key = keyPart.getStringOrObject();
-    }
-    catch (Exception e) {
+    } catch (Exception e) {
       writeException(msg, e, false, servConn);
       servConn.setAsTrue(RESPONDED);
       return;
     }
-    boolean keepalive = false ;
+    boolean keepalive = false;
     try {
       Part keepalivePart = msg.getPart(4);
-      byte[] keepaliveBytes = (byte[])keepalivePart.getObject();
+      byte[] keepaliveBytes = (byte[]) keepalivePart.getObject();
       keepalive = keepaliveBytes[0] != 0x00;
-    }
-    catch (Exception e) {
+    } catch (Exception e) {
       writeException(msg, e, false, servConn);
       servConn.setAsTrue(RESPONDED);
       return;
     }
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received unregister interest request ({} bytes) from {} for region {} key {}", servConn.getName(), msg.getPayloadLength(), servConn.getSocketString(), regionName, key);
+      logger.debug("{}: Received unregister interest request ({} bytes) from {} for region {} key {}", servConn.getName(), msg
+        .getPayloadLength(), servConn.getSocketString(), regionName, key);
     }
 
     // Process the unregister interest request
     if ((key == null) && (regionName == null)) {
       errMessage = LocalizedStrings.UnRegisterInterest_THE_INPUT_REGION_NAME_AND_KEY_FOR_THE_UNREGISTER_INTEREST_REQUEST_ARE_NULL;
     } else if (key == null) {
-      errMessage = LocalizedStrings.UnRegisterInterest_THE_INPUT_KEY_FOR_THE_UNREGISTER_INTEREST_REQUEST_IS_NULL;   
+      errMessage = LocalizedStrings.UnRegisterInterest_THE_INPUT_KEY_FOR_THE_UNREGISTER_INTEREST_REQUEST_IS_NULL;
     } else if (regionName == null) {
       errMessage = LocalizedStrings.UnRegisterInterest_THE_INPUT_REGION_NAME_FOR_THE_UNREGISTER_INTEREST_REQUEST_IS_NULL;
       String s = errMessage.toLocalizedString();
       logger.warn("{}: {}", servConn.getName(), s);
-      writeErrorResponse(msg, MessageType.UNREGISTER_INTEREST_DATA_ERROR,
-          s, servConn);
+      writeErrorResponse(msg, MessageType.UNREGISTER_INTEREST_DATA_ERROR, s, servConn);
       servConn.setAsTrue(RESPONDED);
       return;
     }
 
-    GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+    if (interestType == InterestType.REGULAR_EXPRESSION) {
+      GeodeSecurityUtil.authorizeRegionRead(regionName);
+    } else {
+      GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+    }
 
-      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-      if (authzRequest != null) {
-        // TODO SW: This is a workaround for DynamicRegionFactory
-        // registerInterest calls. Remove this when the semantics of
-        // DynamicRegionFactory are cleaned up.
-        if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          try {
-            UnregisterInterestOperationContext unregisterContext = authzRequest
-                .unregisterInterestAuthorize(regionName, key, interestType);
-            key = unregisterContext.getKey();
-          }
-          catch (NotAuthorizedException ex) {
-            writeException(msg, ex, false, servConn);
-            servConn.setAsTrue(RESPONDED);
-            return;
-          }
+    AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+    if (authzRequest != null) {
+      // TODO SW: This is a workaround for DynamicRegionFactory
+      // registerInterest calls. Remove this when the semantics of
+      // DynamicRegionFactory are cleaned up.
+      if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
+        try {
+          UnregisterInterestOperationContext unregisterContext = authzRequest.unregisterInterestAuthorize(regionName, key, interestType);
+          key = unregisterContext.getKey();
+        } catch (NotAuthorizedException ex) {
+          writeException(msg, ex, false, servConn);
+          servConn.setAsTrue(RESPONDED);
+          return;
         }
       }
-      // Yogesh : bug fix for 36457 :
+    }
+    // Yogesh : bug fix for 36457 :
       /*
        * Region destroy message from server to client results in client calling
        * unregister to server (an unnecessary callback). The unregister
@@ -132,24 +136,25 @@ public class UnregisterInterest extends BaseCommand {
        * found during unregister interest request"); writeErrorResponse(msg,
        * MessageType.UNREGISTER_INTEREST_DATA_ERROR); responded = true; } else {
        */
-      // Unregister interest irrelevent of whether the region is present it or
-      // not
-      servConn.getAcceptor().getCacheClientNotifier().unregisterClientInterest(
-          regionName, key, interestType, isClosing, servConn.getProxyID(), keepalive);
+    // Unregister interest irrelevent of whether the region is present it or
+    // not
+    servConn.getAcceptor()
+            .getCacheClientNotifier()
+            .unregisterClientInterest(regionName, key, interestType, isClosing, servConn.getProxyID(), keepalive);
 
-      // Update the statistics and write the reply
-      // bserverStats.incLong(processDestroyTimeId,
-      // DistributionStats.getStatTime() - start);
-      // start = DistributionStats.getStatTime();
-      writeReply(msg, servConn);
-      servConn.setAsTrue(RESPONDED);
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: Sent unregister interest response for region {} key {}", servConn.getName(), regionName, key);
-      }
-      // bserverStats.incLong(writeDestroyResponseTimeId,
-      // DistributionStats.getStatTime() - start);
-      // bserverStats.incInt(destroyResponsesId, 1);
-      // }
+    // Update the statistics and write the reply
+    // bserverStats.incLong(processDestroyTimeId,
+    // DistributionStats.getStatTime() - start);
+    // start = DistributionStats.getStatTime();
+    writeReply(msg, servConn);
+    servConn.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Sent unregister interest response for region {} key {}", servConn.getName(), regionName, key);
+    }
+    // bserverStats.incLong(writeDestroyResponseTimeId,
+    // DistributionStats.getStatTime() - start);
+    // bserverStats.incInt(destroyResponsesId, 1);
+    // }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterestList.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterestList.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterestList.java
index 932a602..7cb29d4 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterestList.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterestList.java
@@ -15,23 +15,27 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.gemstone.gemfire.cache.DynamicRegionFactory;
+import com.gemstone.gemfire.cache.operations.UnregisterInterestOperationContext;
+import com.gemstone.gemfire.i18n.StringId;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.security.NotAuthorizedException;
-import com.gemstone.gemfire.cache.DynamicRegionFactory;
-import com.gemstone.gemfire.cache.operations.UnregisterInterestOperationContext;
-import com.gemstone.gemfire.i18n.StringId;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 
 
 public class UnregisterInterestList extends BaseCommand {
@@ -47,7 +51,7 @@ public class UnregisterInterestList extends BaseCommand {
 
   @Override
   public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException, ClassNotFoundException {
+    throws IOException, ClassNotFoundException {
     Part regionNamePart = null, keyPart = null, numberOfKeysPart = null;
     String regionName = null;
     Object key = null;
@@ -64,15 +68,14 @@ public class UnregisterInterestList extends BaseCommand {
     regionName = regionNamePart.getString();
 
     Part isClosingListPart = msg.getPart(1);
-    byte[] isClosingListPartBytes = (byte[])isClosingListPart.getObject();
+    byte[] isClosingListPartBytes = (byte[]) isClosingListPart.getObject();
     boolean isClosingList = isClosingListPartBytes[0] == 0x01;
-    boolean keepalive = false ;
+    boolean keepalive = false;
     try {
       Part keepalivePart = msg.getPart(2);
-      byte[] keepalivePartBytes = (byte[])keepalivePart.getObject();
+      byte[] keepalivePartBytes = (byte[]) keepalivePart.getObject();
       keepalive = keepalivePartBytes[0] == 0x01;
-    }
-    catch (Exception e) {
+    } catch (Exception e) {
       writeChunkedException(msg, e, false, servConn);
       servConn.setAsTrue(RESPONDED);
       return;
@@ -86,8 +89,7 @@ public class UnregisterInterestList extends BaseCommand {
       keyPart = msg.getPart(partNumber + i);
       try {
         key = keyPart.getStringOrObject();
-      }
-      catch (Exception e) {
+      } catch (Exception e) {
         writeException(msg, e, false, servConn);
         servConn.setAsTrue(RESPONDED);
         return;
@@ -95,45 +97,46 @@ public class UnregisterInterestList extends BaseCommand {
       keys.add(key);
     }
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received unregister interest request ({} bytes) from {} for the following {} keys in region {}: {}", servConn.getName(), msg.getPayloadLength(), servConn.getSocketString(), numberOfKeys, regionName, keys);
+      logger.debug("{}: Received unregister interest request ({} bytes) from {} for the following {} keys in region {}: {}", servConn
+        .getName(), msg.getPayloadLength(), servConn.getSocketString(), numberOfKeys, regionName, keys);
     }
 
     // Process the unregister interest request
     if (keys.isEmpty() || regionName == null) {
       StringId errMessage = null;
       if (keys.isEmpty() && regionName == null) {
-        errMessage = LocalizedStrings.UnRegisterInterestList_THE_INPUT_LIST_OF_KEYS_IS_EMPTY_AND_THE_INPUT_REGION_NAME_IS_NULL_FOR_THE_UNREGISTER_INTEREST_REQUEST;  
+        errMessage = LocalizedStrings.UnRegisterInterestList_THE_INPUT_LIST_OF_KEYS_IS_EMPTY_AND_THE_INPUT_REGION_NAME_IS_NULL_FOR_THE_UNREGISTER_INTEREST_REQUEST;
       } else if (keys.isEmpty()) {
-        errMessage = LocalizedStrings.UnRegisterInterestList_THE_INPUT_LIST_OF_KEYS_FOR_THE_UNREGISTER_INTEREST_REQUEST_IS_EMPTY;        
+        errMessage = LocalizedStrings.UnRegisterInterestList_THE_INPUT_LIST_OF_KEYS_FOR_THE_UNREGISTER_INTEREST_REQUEST_IS_EMPTY;
       } else if (regionName == null) {
         errMessage = LocalizedStrings.UnRegisterInterest_THE_INPUT_REGION_NAME_FOR_THE_UNREGISTER_INTEREST_REQUEST_IS_NULL;
       }
       String s = errMessage.toLocalizedString();
       logger.warn("{}: {}", servConn.getName(), s);
-      writeErrorResponse(msg, MessageType.UNREGISTER_INTEREST_DATA_ERROR,
-          s, servConn);
+      writeErrorResponse(msg, MessageType.UNREGISTER_INTEREST_DATA_ERROR, s, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-      if (authzRequest != null) {
-        // TODO SW: This is a workaround for DynamicRegionFactory
-        // registerInterest calls. Remove this when the semantics of
-        // DynamicRegionFactory are cleaned up.
-        if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-          try {
-            UnregisterInterestOperationContext unregisterContext = authzRequest
-                .unregisterInterestListAuthorize(regionName, keys);
-            keys = (List)unregisterContext.getKey();
-          }
-          catch (NotAuthorizedException ex) {
-            writeException(msg, ex, false, servConn);
-            servConn.setAsTrue(RESPONDED);
-            return;
-          }
+
+    GeodeSecurityUtil.authorizeRegionRead(regionName);
+
+    AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+    if (authzRequest != null) {
+      // TODO SW: This is a workaround for DynamicRegionFactory
+      // registerInterest calls. Remove this when the semantics of
+      // DynamicRegionFactory are cleaned up.
+      if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
+        try {
+          UnregisterInterestOperationContext unregisterContext = authzRequest.unregisterInterestListAuthorize(regionName, keys);
+          keys = (List) unregisterContext.getKey();
+        } catch (NotAuthorizedException ex) {
+          writeException(msg, ex, false, servConn);
+          servConn.setAsTrue(RESPONDED);
+          return;
         }
       }
-      // Yogesh : bug fix for 36457 :
+    }
+    // Yogesh : bug fix for 36457 :
       /*
        * Region destroy message from server to client results in client calling
        * unregister to server (an unnecessary callback). The unregister
@@ -146,25 +149,25 @@ public class UnregisterInterestList extends BaseCommand {
        * found during register interest list request"); writeErrorResponse(msg,
        * MessageType.UNREGISTER_INTEREST_DATA_ERROR); responded = true; } else {
        */
-      // Register interest
-      servConn.getAcceptor().getCacheClientNotifier().unregisterClientInterest(
-          regionName, keys, isClosingList, servConn.getProxyID(), keepalive);
-
-      // Update the statistics and write the reply
-      // bserverStats.incLong(processDestroyTimeId,
-      // DistributionStats.getStatTime() - start);
-      // start = DistributionStats.getStatTime(); WHY ARE GETTING START AND NOT
-      // USING IT?
-      writeReply(msg, servConn);
-      servConn.setAsTrue(RESPONDED);
-      if (logger.isDebugEnabled()) {
-        logger.debug("{}: Sent unregister interest response for the following {} keys in region {}: {}", servConn.getName(), numberOfKeys, regionName, keys);
-      }
-      // bserverStats.incLong(writeDestroyResponseTimeId,
-      // DistributionStats.getStatTime() - start);
-      // bserverStats.incInt(destroyResponsesId, 1);
-      // }
+    // Register interest
+    servConn.getAcceptor()
+            .getCacheClientNotifier()
+            .unregisterClientInterest(regionName, keys, isClosingList, servConn.getProxyID(), keepalive);
+
+    // Update the statistics and write the reply
+    // bserverStats.incLong(processDestroyTimeId,
+    // DistributionStats.getStatTime() - start);
+    // start = DistributionStats.getStatTime(); WHY ARE GETTING START AND NOT
+    // USING IT?
+    writeReply(msg, servConn);
+    servConn.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Sent unregister interest response for the following {} keys in region {}: {}", servConn.getName(), numberOfKeys, regionName, keys);
     }
+    // bserverStats.incLong(writeDestroyResponseTimeId,
+    // DistributionStats.getStatTime() - start);
+    // bserverStats.incInt(destroyResponsesId, 1);
+    // }
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
index 0ae3f7d..7cc7dbf 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/security/GeodeSecurityUtil.java
@@ -237,6 +237,13 @@ public class GeodeSecurityUtil {
     authorize("DATA", "READ", regionName, key);
   }
 
+  public static void authorizeFunctionExec(String function){
+    authorize("FUNCTION", "EXEC", function);
+  }
+  public static void authorizeFunctionRead(String function){
+    authorize("FUNCTION", "READ", function);
+  }
+
   public static void authorize(String resource, String operation) {
     authorize(resource, operation, null);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/security/GeodePermission.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/security/GeodePermission.java b/geode-core/src/main/java/com/gemstone/gemfire/security/GeodePermission.java
index fab8b74..b1aad84 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/security/GeodePermission.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/security/GeodePermission.java
@@ -27,14 +27,16 @@ public class GeodePermission extends WildcardPermission {
   public enum Resource {
     NULL,
     CLUSTER,
-    DATA
+    DATA,
+    FUNCTION
   }
 
   public enum Operation {
     NULL,
     MANAGE,
     WRITE,
-    READ;
+    READ,
+    EXEC
   }
 
   public Resource getResource() {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
index 854e2f6..4bd7191 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
@@ -16,7 +16,7 @@
  */
 package com.gemstone.gemfire.security;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -24,14 +24,14 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 @Category(DistributedTest.class)
 public class IntegratedClientGetPutAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
 
@@ -60,8 +60,7 @@ public class IntegratedClientGetPutAuthDistributedTest extends AbstractIntegrate
       Map keyValues =  region.getAll(keys);
       assertEquals(0, keyValues.size());
 
-      Set keySet = region.keySet();
-      assertEquals(0, keySet.size());
+      assertNotAuthorized(()->region.keySetOnServer(), "DATA:READ:AuthRegion");
     });
 
 
@@ -81,8 +80,8 @@ public class IntegratedClientGetPutAuthDistributedTest extends AbstractIntegrate
       assertEquals(2, keyValues.size());
 
       // keyset
-      Set keySet = region.keySet();
-      assertEquals(3, keySet.size());
+      Set keySet = region.keySetOnServer();
+      assertEquals(5, keySet.size());
     });
 
     // client3 connects to user as a user authorized to use key1 in AuthRegion region
@@ -100,8 +99,7 @@ public class IntegratedClientGetPutAuthDistributedTest extends AbstractIntegrate
       assertEquals(1, keyValues.size());
 
       // keyset
-      Set keySet = region.keySet();
-      assertEquals(1, keySet.size());
+      assertNotAuthorized(()->region.keySetOnServer(), "DATA:READ:AuthRegion");
     });
 
     ai1.join();


[47/50] [abbrv] incubator-geode git commit: Created GetAll test

Posted by ji...@apache.org.
Created GetAll test


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

Branch: refs/heads/feature/GEODE-1571
Commit: 088e406900335b83e925889d299f93da611d8bf7
Parents: 98244ec
Author: gmeilen <gr...@gmail.com>
Authored: Mon Jun 27 11:37:37 2016 -0700
Committer: gmeilen <gr...@gmail.com>
Committed: Mon Jun 27 11:37:37 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/command/GetAll.java      |  1 +
 .../cache/tier/sockets/command/GetAll651.java   |  1 +
 .../cache/tier/sockets/command/GetAll70.java    |  1 +
 ...tegratedClientGetAllAuthDistributedTest.java | 62 ++++++++++++++++++++
 4 files changed, 65 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/088e4069/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
index 5a49241..c88ccb3 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
@@ -116,6 +116,7 @@ public class GetAll extends BaseCommand {
       servConn.setAsTrue(RESPONDED);
       return;
     }
+    GeodeSecurityUtil.authorizeRegionRead(region.getName());
     // Send header
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     chunkedResponseMsg.setMessageType(MessageType.RESPONSE);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/088e4069/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
index 56100f2..600ee46 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
@@ -114,6 +114,7 @@ public class GetAll651 extends BaseCommand {
       servConn.setAsTrue(RESPONDED);
       return;
     }
+    GeodeSecurityUtil.authorizeRegionRead(region.getName());
     // Send header
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     chunkedResponseMsg.setMessageType(MessageType.RESPONSE);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/088e4069/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
index d162b19..1953051 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
@@ -127,6 +127,7 @@ public class GetAll70 extends BaseCommand {
       servConn.setAsTrue(RESPONDED);
       return;
     }
+    GeodeSecurityUtil.authorizeRegionRead(region.getName());
     // Send header
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     chunkedResponseMsg.setMessageType(MessageType.RESPONSE);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/088e4069/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetAllAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetAllAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetAllAuthDistributedTest.java
new file mode 100644
index 0000000..a690163
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetAllAuthDistributedTest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.security;
+
+import static com.googlecode.catchexception.CatchException.*;
+import static org.assertj.core.api.Assertions.*;
+
+import java.util.Arrays;
+
+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.client.ClientRegionFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
+import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(DistributedTest.class)
+public class IntegratedClientGetAllAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
+
+  @Test
+  public void testGetAll() {
+    client1.invoke("logging in super-user with correct password", () -> {
+      ClientCache cache = new ClientCacheFactory(createClientProperties("stranger", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      assertNotAuthorized(() -> region.getAll(Arrays.asList("key1", "key2", "key3", "key4")), "DATA:READ:AuthRegion");
+    });
+
+    client2.invoke("logging in super-user with correct password", () -> {
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionReader", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      region.getAll(Arrays.asList("key1", "key2", "key3", "key4"));
+    });
+  }
+}
+
+


[49/50] [abbrv] incubator-geode git commit: Merge branch 'feature/GEODE-1571' of ssh://github.com/jinmeiliao/incubator-geode into feature/GEODE-1571

Posted by ji...@apache.org.
Merge branch 'feature/GEODE-1571' of ssh://github.com/jinmeiliao/incubator-geode into feature/GEODE-1571

# Conflicts:
#	geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java


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

Branch: refs/heads/feature/GEODE-1571
Commit: 99dfa4535ef1f005d981121328be911c7a683ca9
Parents: fa66c65 088e406
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon Jun 27 13:29:04 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Jun 27 13:29:04 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/command/GetAll.java      |  1 +
 .../cache/tier/sockets/command/GetAll651.java   |  1 +
 .../cache/tier/sockets/command/GetAll70.java    |  1 +
 .../cache/tier/sockets/command/GetEntry70.java  |  1 +
 ...ractIntegratedClientAuthDistributedTest.java | 71 +++++++++++++----
 .../security/IntegratedClientAuthDUnitTest.java | 36 ++++++++-
 ...tedClientContainsKeyAuthDistributedTest.java | 27 ++++++-
 ...entDestroyInvalidateAuthDistributedTest.java | 31 +++++---
 ...dClientDestroyRegionAuthDistributedTest.java | 31 ++++++--
 ...tegratedClientGetAllAuthDistributedTest.java | 62 +++++++++++++++
 ...gratedClientGetEntryAuthDistributedTest.java | 80 ++++++++++++++++++++
 ...tegratedClientGetPutAuthDistributedTest.java | 34 ++++++---
 ...tedClientRegionClearAuthDistributedTest.java | 20 +++--
 ...ientRegisterInterestAuthDistributedTest.java | 76 ++++++++++++++-----
 ...ratedClientRemoveAllAuthDistributedTest.java | 28 ++++---
 ...ntUnregisterInterestAuthDistributedTest.java | 12 ++-
 16 files changed, 425 insertions(+), 87 deletions(-)
----------------------------------------------------------------------



[10/50] [abbrv] incubator-geode git commit: GEODE-11: Adding a dunit test of querying with a custom query object

Posted by ji...@apache.org.
GEODE-11: Adding a dunit test of querying with a custom query object


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

Branch: refs/heads/feature/GEODE-1571
Commit: d25dd3fe61a67027d10ebcb6b11f3b2ecf6cf444
Parents: 087da4e
Author: Dan Smith <up...@apache.org>
Authored: Wed Jun 15 15:33:13 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Tue Jun 21 18:03:38 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneQueriesBase.java | 26 ++++++++++++++++++++
 1 file changed, 26 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d25dd3fe/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
index 2fdf531..f077bad 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
@@ -33,6 +33,8 @@ import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
 import com.gemstone.gemfire.test.dunit.VM;
 
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.TermQuery;
 import org.junit.Test;
 
 /**
@@ -85,6 +87,30 @@ public abstract class LuceneQueriesBase extends LuceneDUnitTest {
   }
 
   @Test
+  public void canQueryWithCustomLuceneQueryObject() {
+    SerializableRunnableIF createIndex = () -> {
+      LuceneService luceneService = LuceneServiceProvider.get(getCache());
+      luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
+    };
+    dataStore1.invoke(() -> initDataStore(createIndex));
+    dataStore2.invoke(() -> initDataStore(createIndex));
+    accessor.invoke(() -> initAccessor(createIndex));
+    putDataInRegion(accessor);
+    assertTrue(waitForFlushBeforeExecuteTextSearch(dataStore1, 60000));
+
+    //Execute a query with a custom lucene query object
+    accessor.invoke(() -> {
+      Cache cache = getCache();
+      LuceneService service = LuceneServiceProvider.get(cache);
+      LuceneQuery query = service.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, index ->  {
+        return new TermQuery(new Term("text", "world"));
+      });
+      final LuceneQueryResults results = query.search();
+      assertEquals(3, results.size());
+    });
+  }
+
+  @Test
   public void entriesFlushedToIndexAfterWaitForFlushCalled() {
     SerializableRunnableIF createIndex = () -> {
       LuceneService luceneService = LuceneServiceProvider.get(getCache());


[50/50] [abbrv] incubator-geode git commit: Fixed GetAll and tests

Posted by ji...@apache.org.
Fixed GetAll and 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/b43d0266
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/b43d0266
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/b43d0266

Branch: refs/heads/feature/GEODE-1571
Commit: b43d0266dfac2df0c3569278bc0d4b7eafe9f1c9
Parents: 99dfa45
Author: Kevin J. Duling <kd...@pivotal.io>
Authored: Mon Jun 27 13:57:37 2016 -0700
Committer: Kevin J. Duling <kd...@pivotal.io>
Committed: Mon Jun 27 13:57:37 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/command/GetAll.java      |  2 +-
 .../cache/tier/sockets/command/GetAll651.java   |  2 +-
 .../cache/tier/sockets/command/GetAll70.java    |  2 +-
 ...tegratedClientGetAllAuthDistributedTest.java | 20 +++++++++++---------
 4 files changed, 14 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b43d0266/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
index c88ccb3..5ae5d12 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
@@ -116,7 +116,7 @@ public class GetAll extends BaseCommand {
       servConn.setAsTrue(RESPONDED);
       return;
     }
-    GeodeSecurityUtil.authorizeRegionRead(region.getName());
+
     // Send header
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     chunkedResponseMsg.setMessageType(MessageType.RESPONSE);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b43d0266/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
index 600ee46..5b278e3 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll651.java
@@ -114,7 +114,7 @@ public class GetAll651 extends BaseCommand {
       servConn.setAsTrue(RESPONDED);
       return;
     }
-    GeodeSecurityUtil.authorizeRegionRead(region.getName());
+
     // Send header
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     chunkedResponseMsg.setMessageType(MessageType.RESPONSE);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b43d0266/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
index 1953051..c1ab7a9 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll70.java
@@ -127,7 +127,7 @@ public class GetAll70 extends BaseCommand {
       servConn.setAsTrue(RESPONDED);
       return;
     }
-    GeodeSecurityUtil.authorizeRegionRead(region.getName());
+
     // Send header
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     chunkedResponseMsg.setMessageType(MessageType.RESPONSE);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/b43d0266/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetAllAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetAllAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetAllAuthDistributedTest.java
index a690163..f4b346d 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetAllAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetAllAuthDistributedTest.java
@@ -16,22 +16,21 @@
  */
 package com.gemstone.gemfire.security;
 
-import static com.googlecode.catchexception.CatchException.*;
-import static org.assertj.core.api.Assertions.*;
+import static com.gemstone.gemfire.internal.Assert.assertTrue;
+import static org.jgroups.util.Util.assertEquals;
 
 import java.util.Arrays;
+import java.util.Map;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
 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.client.ClientRegionFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
-import com.gemstone.gemfire.test.dunit.IgnoredException;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 @Category(DistributedTest.class)
 public class IntegratedClientGetAllAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
 
@@ -44,7 +43,8 @@ public class IntegratedClientGetAllAuthDistributedTest extends AbstractIntegrate
         .create();
 
       Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
-      assertNotAuthorized(() -> region.getAll(Arrays.asList("key1", "key2", "key3", "key4")), "DATA:READ:AuthRegion");
+      Map emptyMap = region.getAll(Arrays.asList("key1", "key2", "key3", "key4"));
+      assertTrue(emptyMap.isEmpty());
     });
 
     client2.invoke("logging in super-user with correct password", () -> {
@@ -54,7 +54,9 @@ public class IntegratedClientGetAllAuthDistributedTest extends AbstractIntegrate
         .create();
 
       Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
-      region.getAll(Arrays.asList("key1", "key2", "key3", "key4"));
+      Map filledMap = region.getAll(Arrays.asList("key1", "key2", "key3", "key4"));
+      assertEquals("Map should contain 4 entries", 4, filledMap.size());
+      assertTrue(filledMap.containsKey("key1"));
     });
   }
 }


[45/50] [abbrv] incubator-geode git commit: Merge branch 'develop' into feature/GEODE-1571

Posted by ji...@apache.org.
Merge branch 'develop' into feature/GEODE-1571


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

Branch: refs/heads/feature/GEODE-1571
Commit: a534931e7daee782d393e90def9c9558584cbfc3
Parents: 536c13b 7ad9cc9
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Mon Jun 27 11:08:26 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Mon Jun 27 11:08:26 2016 -0700

----------------------------------------------------------------------
 .../internal/cache/AbstractCacheServer.java     |  19 +-
 .../gemfire/internal/cache/CacheServerImpl.java |   2 -
 .../internal/cache/RemoteOperationMessage.java  |   3 +-
 .../gemfire/internal/cache/TXManagerImpl.java   | 138 ++++++-
 .../gemfire/internal/cache/TXState.java         |  11 +
 .../gemfire/internal/cache/TXStateProxy.java    |   3 +
 .../internal/cache/TXStateProxyImpl.java        |   9 +
 .../cache/partitioned/PartitionMessage.java     |   3 +-
 .../cache/tier/sockets/AcceptorImpl.java        |   3 +-
 .../cache/tier/sockets/BaseCommand.java         |   1 +
 .../cache/tier/sockets/CacheClientNotifier.java |  23 +-
 .../cache/tier/sockets/ClientHealthMonitor.java |  11 +-
 .../cache/xmlcache/CacheServerCreation.java     |   1 -
 .../management/internal/cli/CommandRequest.java |  22 +-
 .../management/internal/cli/GfshParser.java     |   8 +-
 .../disttx/DistributedTransactionDUnitTest.java |   3 +-
 .../cache/ClientServerTransactionDUnitTest.java | 276 +++++++++++--
 .../tier/sockets/AcceptorImplJUnitTest.java     |  10 +-
 .../internal/cli/CommandRequestTest.java        | 161 ++++++++
 .../cli/commands/CliCommandTestBase.java        |   2 +-
 .../commands/GemfireDataCommandsDUnitTest.java  |   1 -
 .../gemfire/cache/lucene/LuceneQuery.java       |  20 +-
 .../cache/lucene/LuceneQueryResults.java        |  57 ---
 .../gemfire/cache/lucene/LuceneService.java     |   2 +-
 .../lucene/PageableLuceneQueryResults.java      |  58 +++
 .../cache/lucene/internal/LuceneQueryImpl.java  |  63 ++-
 .../lucene/internal/LuceneQueryResultsImpl.java | 120 ------
 .../PageableLuceneQueryResultsImpl.java         | 121 ++++++
 .../lucene/internal/distributed/EntryScore.java |  10 +-
 .../lucene/internal/distributed/TopEntries.java |  16 +-
 ...IndexCreationPersistenceIntegrationTest.java |  11 +-
 .../LuceneIndexMaintenanceIntegrationTest.java  |  14 +-
 .../gemfire/cache/lucene/LuceneQueriesBase.java |   8 +-
 .../lucene/LuceneQueriesIntegrationTest.java    |  54 ++-
 ...LuceneQueriesPersistenceIntegrationTest.java |   2 +-
 .../LuceneQueryImplIntegrationTest.java         | 120 ------
 .../internal/LuceneQueryImplJUnitTest.java      | 150 +++++++
 .../LuceneQueryResultsImplJUnitTest.java        | 122 ------
 ...PageableLuceneQueryResultsImplJUnitTest.java | 142 +++++++
 .../DistributedScoringJUnitTest.java            |   8 +-
 .../distributed/EntryScoreJUnitTest.java        |   4 +-
 .../distributed/LuceneFunctionJUnitTest.java    |  11 +-
 .../TopEntriesCollectorJUnitTest.java           |  16 +-
 .../TopEntriesFunctionCollectorJUnitTest.java   |  13 +-
 .../distributed/TopEntriesJUnitTest.java        |  34 +-
 .../IndexRepositoryImplPerformanceTest.java     |   2 +-
 .../cache/lucene/test/LuceneTestUtilities.java  |  19 +-
 geode-site/website/README.md                    |   2 +-
 .../website/content/css/font-awesome.min.css    | 409 +------------------
 geode-site/website/content/css/geode-site.css   |  41 +-
 geode-site/website/content/font/FontAwesome.otf | Bin 61896 -> 124988 bytes
 .../content/font/fontawesome-webfont-eot.eot    | Bin 37405 -> 76518 bytes
 .../content/font/fontawesome-webfont-svg.svg    | 366 +++++++++++++++--
 .../content/font/fontawesome-webfont-ttf.ttf    | Bin 79076 -> 152796 bytes
 .../content/font/fontawesome-webfont-woff.woff  | Bin 43572 -> 90412 bytes
 geode-site/website/content/index.html           |  49 ++-
 geode-site/website/layouts/header.html          |  18 +-
 geode-site/website/lib/default.rb               |   4 +-
 geode-site/website/nanoc.yaml                   |   2 +
 .../DataCommandsOverHttpDistributedTest.java    | 150 +++++++
 60 files changed, 1844 insertions(+), 1104 deletions(-)
----------------------------------------------------------------------


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

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

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


[19/50] [abbrv] incubator-geode git commit: secure unregisterInterest

Posted by ji...@apache.org.
secure unregisterInterest


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

Branch: refs/heads/feature/GEODE-1571
Commit: 2c090322384bf1cef7e9bb539dae8cffb9d62229
Parents: 6a45ca0
Author: gmeilen <gr...@gmail.com>
Authored: Thu Jun 23 15:46:07 2016 -0700
Committer: gmeilen <gr...@gmail.com>
Committed: Thu Jun 23 15:46:07 2016 -0700

----------------------------------------------------------------------
 .../sockets/command/UnregisterInterest.java     |  6 ++-
 ...ntUnregisterInterestAuthDistributedTest.java | 41 ++++++++++++++++++++
 2 files changed, 45 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2c090322/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
index 39a462d..9161d6d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
@@ -24,6 +24,7 @@ import com.gemstone.gemfire.internal.cache.tier.MessageType;
 import com.gemstone.gemfire.internal.cache.tier.sockets.*;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.security.NotAuthorizedException;
 import com.gemstone.gemfire.cache.DynamicRegionFactory;
 import com.gemstone.gemfire.cache.operations.UnregisterInterestOperationContext;
@@ -96,7 +97,9 @@ public class UnregisterInterest extends BaseCommand {
           s, servConn);
       servConn.setAsTrue(RESPONDED);
     }
-    else {
+
+    GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();
       if (authzRequest != null) {
         // TODO SW: This is a workaround for DynamicRegionFactory
@@ -146,7 +149,6 @@ public class UnregisterInterest extends BaseCommand {
       // DistributionStats.getStatTime() - start);
       // bserverStats.incInt(destroyResponsesId, 1);
       // }
-    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2c090322/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
new file mode 100644
index 0000000..2853a56
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.security;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(DistributedTest.class)
+public class IntegratedClientUnregisterInterestAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest{
+  @Test
+  public void testRegisterInterest() throws InterruptedException {
+    // client2 connects to user as a user authorized to use AuthRegion region
+    AsyncInvocation ai1 =  client2.invokeAsync(()->{
+      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      region.registerInterest("key3");
+      region.unregisterInterest("key3");  //  DATA:READ:AuthRegion:key3;
+    });
+    ai1.join();
+    ai1.checkException();
+  }
+}


[40/50] [abbrv] incubator-geode git commit: GEODE-1571: security for removeAll

Posted by ji...@apache.org.
GEODE-1571: security for removeAll


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

Branch: refs/heads/feature/GEODE-1571
Commit: e504d978d77bd545b3e73c82462b0dba73c82759
Parents: 8493cfe
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Jun 24 21:20:51 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri Jun 24 21:20:51 2016 -0700

----------------------------------------------------------------------
 .../internal/cache/tier/sockets/command/RemoveAll.java       | 2 +-
 .../IntegratedClientRemoveAllAuthDistributedTest.java        | 8 ++++----
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e504d978/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
index 6ed5d2f..d04a585 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
@@ -210,7 +210,7 @@ public class RemoveAll extends BaseCommand {
         servConn.setRequestSpecificTimeout(timeout);
       }
 
-      GeodeSecurityUtil.authorizeDataWrite();
+      GeodeSecurityUtil.authorizeRegionWrite(regionName);
 
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();
       if (authzRequest != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e504d978/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
index 527972c..edfbdd6 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
@@ -35,17 +35,17 @@ public class IntegratedClientRemoveAllAuthDistributedTest extends AbstractIntegr
   public void testRemoveAll() throws InterruptedException {
 
     AsyncInvocation ai1 = client1.invokeAsync(() -> {
-      Cache cache = SecurityTestUtils.createCacheClient("dataReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      Cache cache = SecurityTestUtils.createCacheClient("authRegionReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
-      assertNotAuthorized(() -> region.removeAll(Arrays.asList("key1", "key2", "key3", "key4")), "DATA:WRITE");
+      assertNotAuthorized(() -> region.removeAll(Arrays.asList("key1", "key2", "key3", "key4")), "DATA:WRITE:AuthRegion");
     });
 
     AsyncInvocation ai2 = client2.invokeAsync(() -> {
-      Cache cache = SecurityTestUtils.createCacheClient("dataUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      Cache cache = SecurityTestUtils.createCacheClient("authRegionWriter", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
       region.removeAll(Arrays.asList("key1", "key2", "key3", "key4"));
       assertFalse(region.containsKey("key1"));
-      assertFalse(region.containsKeyOnServer("key1"));
+      assertNotAuthorized(()->region.containsKeyOnServer("key1"), "DATA:READ:AuthRegion:key1");
     });
     ai1.join();
     ai2.join();


[21/50] [abbrv] incubator-geode git commit: Fix test method name

Posted by ji...@apache.org.
Fix test method name


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

Branch: refs/heads/feature/GEODE-1571
Commit: cb3702145662b5f1b8bb7604115360239dd63d1b
Parents: 6dc3e50
Author: gmeilen <gr...@gmail.com>
Authored: Thu Jun 23 17:00:45 2016 -0700
Committer: gmeilen <gr...@gmail.com>
Committed: Thu Jun 23 17:00:45 2016 -0700

----------------------------------------------------------------------
 .../IntegratedClientUnregisterInterestAuthDistributedTest.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/cb370214/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
index 2853a56..aa46882 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
@@ -27,7 +27,7 @@ import org.junit.experimental.categories.Category;
 @Category(DistributedTest.class)
 public class IntegratedClientUnregisterInterestAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest{
   @Test
-  public void testRegisterInterest() throws InterruptedException {
+  public void testUnregisterInterest() throws InterruptedException {
     // client2 connects to user as a user authorized to use AuthRegion region
     AsyncInvocation ai1 =  client2.invokeAsync(()->{
       Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);


[28/50] [abbrv] incubator-geode git commit: GEODE-11: PageableLuceneQueryResults extends Iterator

Posted by ji...@apache.org.
GEODE-11: PageableLuceneQueryResults extends Iterator

Changing PageableLuceneQueryResults to extend
Iterator<List<LuceneResultStruct>> so that it can be used with utilities that
expect an iterator.

This closes #169


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

Branch: refs/heads/feature/GEODE-1571
Commit: 8bbea5b5809ffefb43f669b3ab59830797f50897
Parents: 2c4e17f
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Mon Jun 20 15:31:22 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri Jun 24 14:41:42 2016 -0700

----------------------------------------------------------------------
 .../lucene/PageableLuceneQueryResults.java      |  7 ++--
 .../cache/lucene/internal/LuceneQueryImpl.java  |  8 ++--
 .../PageableLuceneQueryResultsImpl.java         | 11 ++---
 .../LuceneIndexMaintenanceIntegrationTest.java  |  2 +-
 .../gemfire/cache/lucene/LuceneQueriesBase.java |  2 +-
 .../lucene/LuceneQueriesIntegrationTest.java    |  6 +--
 .../internal/LuceneQueryImplJUnitTest.java      | 13 +-----
 ...PageableLuceneQueryResultsImplJUnitTest.java | 44 ++++++++++++++------
 .../cache/lucene/test/LuceneTestUtilities.java  |  5 +--
 9 files changed, 53 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8bbea5b5/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/PageableLuceneQueryResults.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/PageableLuceneQueryResults.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/PageableLuceneQueryResults.java
index ef229da..c2def44 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/PageableLuceneQueryResults.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/PageableLuceneQueryResults.java
@@ -19,6 +19,7 @@
 
 package com.gemstone.gemfire.cache.lucene;
 
+import java.util.Iterator;
 import java.util.List;
 
 import com.gemstone.gemfire.annotations.Experimental;
@@ -32,7 +33,7 @@ import com.gemstone.gemfire.annotations.Experimental;
  * @param <V> The type of the value
  */
 @Experimental
-public interface PageableLuceneQueryResults<K, V> {
+public interface PageableLuceneQueryResults<K, V> extends Iterator<List<LuceneResultStruct<K,V>>> {
   /**
    * @return total number of hits for this query
    */
@@ -48,10 +49,10 @@ public interface PageableLuceneQueryResults<K, V> {
    * 
    * @return a page of results, or null if there are no more pages
    */
-  public List<LuceneResultStruct<K, V>> getNextPage();
+  public List<LuceneResultStruct<K, V>> next();
 
   /**
    *  True if there another page of results. 
    */
-  public boolean hasNextPage();
+  public boolean hasNext();
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8bbea5b5/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
index 3125193..ef2f4ee 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
@@ -19,7 +19,6 @@
 
 package com.gemstone.gemfire.cache.lucene.internal;
 
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -87,12 +86,11 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
   @Override
   public List<LuceneResultStruct<K, V>> findResults() throws LuceneQueryException {
     PageableLuceneQueryResults<K, V> pages = findPages(0);
-    final List<LuceneResultStruct<K, V>> page = pages.getNextPage();
-
-    if(page == null) {
+    if(!pages.hasNext()) {
       return Collections.emptyList();
     }
-    return page;
+
+    return pages.next();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8bbea5b5/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
index 487b740..17668a7 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
@@ -22,6 +22,7 @@ package com.gemstone.gemfire.cache.lucene.internal;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.NoSuchElementException;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
@@ -69,9 +70,9 @@ public class PageableLuceneQueryResultsImpl<K,V> implements PageableLuceneQueryR
   }
 
   @Override
-  public List<LuceneResultStruct<K,V>> getNextPage() {
-    if(!hasNextPage()) {
-      return null;
+  public List<LuceneResultStruct<K,V>> next() {
+    if(!hasNext()) {
+      throw new NoSuchElementException();
     }
     
     int end = currentHit + pageSize;
@@ -84,7 +85,7 @@ public class PageableLuceneQueryResultsImpl<K,V> implements PageableLuceneQueryR
     }
     
     Map<K,V> values = userRegion.getAll(keys);
-    
+
     ArrayList<LuceneResultStruct<K,V>> results = new ArrayList<LuceneResultStruct<K,V>>(hits.size());
     for(EntryScore<K> score : scores) {
       V value = values.get(score.getKey());
@@ -98,7 +99,7 @@ public class PageableLuceneQueryResultsImpl<K,V> implements PageableLuceneQueryR
   }
 
   @Override
-  public boolean hasNextPage() {
+  public boolean hasNext() {
     return hits.size() > currentHit;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8bbea5b5/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
index d0c91e5..7d7fa3d 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
@@ -155,7 +155,7 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
     index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
     // Execute query to fetch all the values for "description" field.
     LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "description:\"hello world\"", DEFAULT_FIELD);
-    LuceneQueryResults<Integer, TestObject> results = query.search();
+    PageableLuceneQueryResults<Integer, TestObject> results = query.findPages();
     // The query should return 0 results.
     assertEquals(0, results.size());
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8bbea5b5/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
index 77ce0bb..301d5af 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
@@ -152,7 +152,7 @@ public abstract class LuceneQueriesBase extends LuceneDUnitTest {
       query = service.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "text:world", DEFAULT_FIELD);
       PageableLuceneQueryResults<Integer, TestObject> results = query.findPages();
       assertEquals(3, results.size());
-      List<LuceneResultStruct<Integer, TestObject>> page = results.getNextPage();
+      List<LuceneResultStruct<Integer, TestObject>> page = results.next();
 
       Map<Integer, TestObject> data = new HashMap<Integer, TestObject>();
       for (LuceneResultStruct<Integer, TestObject> row : page) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8bbea5b5/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index d563f4e..988e4f5 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -122,10 +122,10 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     final LuceneQuery<Object, Object> query = addValuesAndCreateQuery();
 
     final PageableLuceneQueryResults<Object, Object> pages = query.findPages();
-    assertTrue(pages.hasNextPage());
+    assertTrue(pages.hasNext());
     assertEquals(3, pages.size());
-    final List<LuceneResultStruct<Object, Object>> page1 = pages.getNextPage();
-    final List<LuceneResultStruct<Object, Object>> page2 = pages.getNextPage();
+    final List<LuceneResultStruct<Object, Object>> page1 = pages.next();
+    final List<LuceneResultStruct<Object, Object>> page2 = pages.next();
     List<LuceneResultStruct<Object, Object>> allEntries=new ArrayList<>();
     allEntries.addAll(page1);
     allEntries.addAll(page2);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8bbea5b5/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
index 95371e4..c325f55 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
@@ -19,7 +19,6 @@
 
 package com.gemstone.gemfire.cache.lucene.internal;
 
-import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.DEFAULT_FIELD;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
@@ -32,20 +31,14 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.mockito.ArgumentCaptor;
 
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.execute.Execution;
-import com.gemstone.gemfire.cache.execute.FunctionAdapter;
-import com.gemstone.gemfire.cache.execute.FunctionContext;
-import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
-import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
@@ -55,10 +48,6 @@ import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntries;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollectorManager;
-import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector;
-import com.gemstone.gemfire.internal.cache.ValidatingDiskRegion;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -151,7 +140,7 @@ public class LuceneQueryImplJUnitTest {
     assertEquals("index", context.getIndexName());
 
     assertEquals(5, results.getMaxScore(), 0.01);
-    final List<LuceneResultStruct<Object, Object>> page = results.getNextPage();
+    final List<LuceneResultStruct<Object, Object>> page = results.next();
     assertEquals(1, page.size());
     LuceneResultStruct element = page.iterator().next();
     assertEquals("hi", element.getKey());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8bbea5b5/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
index 7703a74..902052b 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
@@ -22,13 +22,17 @@ import static org.junit.Assert.*;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.NoSuchElementException;
 
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
+import org.junit.rules.ExpectedException;
 import org.mockito.Mockito;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -36,11 +40,15 @@ import org.mockito.stubbing.Answer;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntries;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
 public class PageableLuceneQueryResultsImplJUnitTest {
 
+  @Rule
+  public ExpectedException thrown = ExpectedException.none();
+
   private List<EntryScore<String>> hits;
   private List<LuceneResultStruct> expected = new ArrayList<LuceneResultStruct>();
   private Region<String, String> userRegion;
@@ -86,22 +94,23 @@ public class PageableLuceneQueryResultsImplJUnitTest {
     
     assertEquals(23, results.size());
     
-    assertTrue(results.hasNextPage());
+    assertTrue(results.hasNext());
     
-    List<LuceneResultStruct<String, String>> next  = results.getNextPage();
+    List<LuceneResultStruct<String, String>> next  = results.next();
     assertEquals(expected.subList(0, 10), next);
     
-    assertTrue(results.hasNextPage());
-    next  = results.getNextPage();
+    assertTrue(results.hasNext());
+    next  = results.next();
     assertEquals(expected.subList(10, 20), next);
     
-    assertTrue(results.hasNextPage());
-    next  = results.getNextPage();
+    assertTrue(results.hasNext());
+    next  = results.next();
     assertEquals(expected.subList(20, 23), next);
     
     
-    assertFalse(results.hasNextPage());
-    assertNull(results.getNextPage());
+    assertFalse(results.hasNext());
+    thrown.expect(NoSuchElementException.class);
+    results.next();
   }
   
   @Test
@@ -110,13 +119,24 @@ public class PageableLuceneQueryResultsImplJUnitTest {
     
     assertEquals(23, results.size());
     
-    assertTrue(results.hasNextPage());
+    assertTrue(results.hasNext());
     
-    List<LuceneResultStruct<String, String>> next  = results.getNextPage();
+    List<LuceneResultStruct<String, String>> next  = results.next();
     assertEquals(expected, next);
     
-    assertFalse(results.hasNextPage());
-    assertNull(results.getNextPage());
+    assertFalse(results.hasNext());
+    thrown.expect(NoSuchElementException.class);
+    results.next();
+  }
+
+  @Test
+  public void shouldThrowNoSuchElementExceptionFromNextWithNoMorePages() {
+    PageableLuceneQueryResultsImpl<String, String> results = new PageableLuceneQueryResultsImpl<>(
+      Collections.emptyList(), userRegion, 0);
+
+    assertFalse(results.hasNext());
+    thrown.expect(NoSuchElementException.class);
+    results.next();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8bbea5b5/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
index 06c3c7d..8aca11c 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -21,7 +21,6 @@ package com.gemstone.gemfire.cache.lucene.test;
 import static org.junit.Assert.*;
 
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
@@ -100,8 +99,8 @@ public class LuceneTestUtilities {
   public static <K> void verifyQueryKeyAndValues(LuceneQuery<K,Object> query, HashMap expectedResults) throws LuceneQueryException {
     HashMap actualResults = new HashMap<>();
     final PageableLuceneQueryResults<K, Object> results = query.findPages();
-    while(results.hasNextPage()) {
-      results.getNextPage().stream()
+    while(results.hasNext()) {
+      results.next().stream()
         .forEach(struct -> {
           Object value = struct.getValue();
           if (value instanceof PdxInstance) {


[11/50] [abbrv] incubator-geode git commit: GEODE-1494: Fixing javadocs and stats descriptions for stat suppliers

Posted by ji...@apache.org.
GEODE-1494: Fixing javadocs and stats descriptions for stat suppliers

Based on review feedback, correcting some javadocs and the description
of the sample callbacks stat.


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

Branch: refs/heads/feature/GEODE-1571
Commit: 5882693954d6f71e5479e08bfb0a2abb3ac956c2
Parents: d25dd3f
Author: Dan Smith <up...@apache.org>
Authored: Tue Jun 21 11:04:54 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Tue Jun 21 18:04:00 2016 -0700

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/Statistics.java   | 32 +++++++-------------
 .../gemfire/internal/StatSamplerStats.java      |  2 +-
 2 files changed, 12 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/58826939/geode-core/src/main/java/com/gemstone/gemfire/Statistics.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/Statistics.java b/geode-core/src/main/java/com/gemstone/gemfire/Statistics.java
index 0b9c18e..80426e2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/Statistics.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/Statistics.java
@@ -466,9 +466,8 @@ public interface Statistics {
    * replaces any previously registered supplier. If the passed in suppplier is null, it
    * will remove any existing supplier
    * @return the previously registered supplier, or null if there was no previously registered supplier
-   * @throws ArrayIndexOutOfBoundsException If the id is invalid.
-   * @throws IllegalArgumentException if the type of this
-   * stat is not int
+   * @throws IllegalArgumentException If the id is invalid.
+   * @since Geode 1.0
    */
   public IntSupplier setIntSupplier(int id, IntSupplier supplier);
 
@@ -495,8 +494,7 @@ public interface Statistics {
    *         If no statistic exists with name <code>name</code> or
    *         if the statistic named <code>name</code> is not of
    *         type <code>int</code>.
-   * @throws IllegalArgumentException if the type of this
-   * stat is not int
+   * @since Geode 1.0
    */
   public IntSupplier setIntSupplier(String name, IntSupplier supplier);
 
@@ -524,8 +522,7 @@ public interface Statistics {
    *         If no statistic exists with the given <code>descriptor</code> or
    *         if the described statistic is not of
    *         type <code>int</code>.
-   * @throws IllegalArgumentException if the type of this
-   * stat is not int
+   * @since Geode 1.0
    */
   public IntSupplier setIntSupplier(StatisticDescriptor descriptor, IntSupplier supplier);
 
@@ -549,9 +546,8 @@ public interface Statistics {
    * replaces any previously registered supplier. If the passed in suppplier is null, it
    * will remove any existing supplier
    * @return the previously registered supplier, or null if there was no previously registered supplier
-   * @throws ArrayIndexOutOfBoundsException If the id is invalid.
-   * @throws IllegalArgumentException if the type of this
-   * stat is not long
+   * @throws IllegalArgumentException If the id is invalid.
+   * @since Geode 1.0
    */
   public LongSupplier setLongSupplier(int id, LongSupplier supplier);
 
@@ -578,8 +574,6 @@ public interface Statistics {
    *         If no statistic exists with name <code>name</code> or
    *         if the statistic named <code>name</code> is not of
    *         type <code>long</code>.
-   * @throws IllegalArgumentException if the type of this
-   * stat is not long
    */
   public LongSupplier setLongSupplier(String name, LongSupplier supplier);
 
@@ -607,8 +601,7 @@ public interface Statistics {
    *         If no statistic exists with the given <code>descriptor</code> or
    *         if the described statistic is not of
    *         type <code>long</code>.
-   * @throws IllegalArgumentException if the type of this
-   * stat is not long
+   * @since Geode 1.0
    */
   public LongSupplier setLongSupplier(StatisticDescriptor descriptor, LongSupplier supplier);
 
@@ -632,9 +625,8 @@ public interface Statistics {
    * replaces any previously registered supplier. If the passed in suppplier is null, it
    * will remove any existing supplier
    * @return the previously registered supplier, or null if there was no previously registered supplier
-   * @throws ArrayIndexOutOfBoundsException If the id is invalid.
-   * @throws IllegalArgumentException if the type of this
-   * stat is not double
+   * @throws IllegalArgumentException If the id is invalid.
+   * @since Geode 1.0
    */
   public DoubleSupplier setDoubleSupplier(int id, DoubleSupplier supplier);
 
@@ -661,8 +653,7 @@ public interface Statistics {
    *         If no statistic exists with name <code>name</code> or
    *         if the statistic named <code>name</code> is not of
    *         type <code>double</code>.
-   * @throws IllegalArgumentException if the type of this
-   * stat is not double
+   * @since Geode 1.0
    */
   public DoubleSupplier setDoubleSupplier(String name, DoubleSupplier supplier);
 
@@ -690,8 +681,7 @@ public interface Statistics {
    *         If no statistic exists with the given <code>descriptor</code> or
    *         if the described statistic is not of
    *         type <code>double</code>.
-   * @throws IllegalArgumentException if the type of this
-   * stat is not double
+   * @since Geode 1.0
    */
   public DoubleSupplier setDoubleSupplier(StatisticDescriptor descriptor, DoubleSupplier supplier);
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/58826939/geode-core/src/main/java/com/gemstone/gemfire/internal/StatSamplerStats.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/StatSamplerStats.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/StatSamplerStats.java
index bbe333c..c3d556f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/StatSamplerStats.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/StatSamplerStats.java
@@ -61,7 +61,7 @@ public class StatSamplerStats {
                                                     "Total number of JVM pauses (which may or may not be full GC pauses) detected by this sampler. A JVM pause is defined as a system event which kept the statistics sampler thread from sampling for 3000 or more milliseconds. This threshold can be customized by setting the system property gemfire.statSamplerDelayThreshold (units are milliseconds).",
                                                     "jvmPauses", false),
                                  f.createIntGauge(SAMPLE_CALLBACKS,
-                                   "Total number of statistics that are sampled using callbacks.",
+                                   "Current number of statistics that are sampled using callbacks.",
                                    "resources", false),
                                  f.createIntCounter(SAMPLE_CALLBACK_ERRORS,
                                    "Total number of exceptions thrown by callbacks when performing sampling",


[25/50] [abbrv] incubator-geode git commit: GEODE-1546: fix issue when a proxy server was shut down before its scheduled task of removing departed client is invoked.

Posted by ji...@apache.org.
GEODE-1546: fix issue when a proxy server was shut down before its scheduled task of removing departed client is invoked.

Move transactionTimeToLive setting into TXManagerImpl.
Update proxy server to track the latest proxy after failover.
Expire client transactions if they do not fail over to new proxy servers and after transaction timeout period.
Add test cases for the above scenario.
Refactored the test code and made transactionTimeToLive setting adjustable to reduce test running time.


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

Branch: refs/heads/feature/GEODE-1571
Commit: 87876a720c26ea67f284d64fcbb73ab3f33e5040
Parents: f38d696
Author: eshu <es...@pivotal.io>
Authored: Fri Jun 24 11:51:27 2016 -0700
Committer: eshu <es...@pivotal.io>
Committed: Fri Jun 24 11:59:59 2016 -0700

----------------------------------------------------------------------
 .../internal/cache/AbstractCacheServer.java     |  19 +-
 .../gemfire/internal/cache/CacheServerImpl.java |   2 -
 .../internal/cache/RemoteOperationMessage.java  |   3 +-
 .../gemfire/internal/cache/TXManagerImpl.java   | 138 +++++++++-
 .../gemfire/internal/cache/TXState.java         |  11 +
 .../gemfire/internal/cache/TXStateProxy.java    |   3 +
 .../internal/cache/TXStateProxyImpl.java        |   9 +
 .../cache/partitioned/PartitionMessage.java     |   3 +-
 .../cache/tier/sockets/AcceptorImpl.java        |   3 +-
 .../cache/tier/sockets/BaseCommand.java         |   1 +
 .../cache/tier/sockets/CacheClientNotifier.java |  23 +-
 .../cache/tier/sockets/ClientHealthMonitor.java |  11 +-
 .../cache/xmlcache/CacheServerCreation.java     |   1 -
 .../disttx/DistributedTransactionDUnitTest.java |   3 +-
 .../cache/ClientServerTransactionDUnitTest.java | 276 ++++++++++++++++---
 .../tier/sockets/AcceptorImplJUnitTest.java     |  10 +-
 16 files changed, 421 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractCacheServer.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractCacheServer.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractCacheServer.java
index c9648a3..60e256d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractCacheServer.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/AbstractCacheServer.java
@@ -117,13 +117,6 @@ public abstract class AbstractCacheServer implements CacheServer {
    * members as clients of this server leave/crash. 
    */
   protected final ClientMembershipListener listener;
-
-  /**
-   * The number of seconds to keep transaction states for disconnected clients.
-   * This allows the client to fail over to another server and still find
-   * the transaction state to complete the transaction.
-   */
-  private int transactionTimeToLive;
   
   //////////////////////  Constructors  //////////////////////
 
@@ -147,9 +140,7 @@ public abstract class AbstractCacheServer implements CacheServer {
     this.tcpNoDelay = CacheServer.DEFAULT_TCP_NO_DELAY;
     this.maximumTimeBetweenPings = CacheServer.DEFAULT_MAXIMUM_TIME_BETWEEN_PINGS;
     this.maximumMessageCount = CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT;
-    this.messageTimeToLive = CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE;
-    // TODO this should be configurable in CacheServer
-    this.transactionTimeToLive = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "cacheServer.transactionTimeToLive", 180);
+    this.messageTimeToLive = CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE;    
     this.groups = CacheServer.DEFAULT_GROUPS;
     this.bindAddress = CacheServer.DEFAULT_BIND_ADDRESS;
     this.hostnameForClients = CacheServer.DEFAULT_HOSTNAME_FOR_CLIENTS;
@@ -307,14 +298,6 @@ public abstract class AbstractCacheServer implements CacheServer {
     this.maximumMessageCount = maximumMessageCount;
   }
   
-  public void setTransactionTimeToLive(int seconds) {
-    this.transactionTimeToLive = seconds;
-  }
-  
-  public int getTransactionTimeToLive() {
-    return this.transactionTimeToLive;
-  }
-  
   public int getMessageTimeToLive() {
     return this.messageTimeToLive;
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
index 2c72410..37c05eb 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/CacheServerImpl.java
@@ -264,7 +264,6 @@ public class CacheServerImpl
     setMaximumTimeBetweenPings(other.getMaximumTimeBetweenPings());
     setMaximumMessageCount(other.getMaximumMessageCount());
     setMessageTimeToLive(other.getMessageTimeToLive());
-//    setTransactionTimeToLive(other.getTransactionTimeToLive());  not implemented in CacheServer for v6.6
     setGroups(other.getGroups());
     setLoadProbe(other.getLoadProbe());
     setLoadPollInterval(other.getLoadPollInterval());
@@ -322,7 +321,6 @@ public class CacheServerImpl
                                      getMaxThreads(), 
                                      getMaximumMessageCount(),
                                      getMessageTimeToLive(),
-                                     getTransactionTimeToLive(),
                                      this.loadMonitor,
                                      overflowAttributesList, 
                                      this.isGatewayReceiver,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteOperationMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteOperationMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteOperationMessage.java
index db5bcca..c6ec0ab 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteOperationMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/RemoteOperationMessage.java
@@ -245,7 +245,8 @@ public abstract class RemoteOperationMessage extends DistributionMessage impleme
             // NO DISTRIBUTED MESSAGING CAN BE DONE HERE!
             sendReply = false;
           } else if (tx.isInProgress()) {
-            sendReply = operateOnRegion(dm, r, startTime);       
+            sendReply = operateOnRegion(dm, r, startTime);
+            tx.updateProxyServer(this.getSender());
           }  
         } finally {
           txMgr.unmasquerade(tx);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
index 1ea7f71..1512234 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXManagerImpl.java
@@ -27,6 +27,7 @@ import com.gemstone.gemfire.distributed.internal.*;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.SystemTimer.SystemTimerTask;
 import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.concurrent.ConcurrentHashSet;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
@@ -137,7 +138,13 @@ public class TXManagerImpl implements CacheTransactionManager,
    */
   private final ThreadLocal<Boolean> isTXDistributed;
   
-
+  /**
+   * The number of seconds to keep transaction states for disconnected clients.
+   * This allows the client to fail over to another server and still find
+   * the transaction state to complete the transaction.
+   */
+  private int transactionTimeToLive;
+  
   /** Constructor that implements the {@link CacheTransactionManager}
    * interface. Only only one instance per {@link com.gemstone.gemfire.cache.Cache}
    *
@@ -155,6 +162,7 @@ public class TXManagerImpl implements CacheTransactionManager,
     this.hostedTXStates = new HashMap<TXId, TXStateProxy>();
     this.txContext = new ThreadLocal<TXStateProxy>();
     this.isTXDistributed = new ThreadLocal<Boolean>();
+    this.transactionTimeToLive = Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "cacheServer.transactionTimeToLive", 180);
     currentInstance = this;
   }
 
@@ -937,6 +945,7 @@ public class TXManagerImpl implements CacheTransactionManager,
         }
       }
     }
+    expireClientTransactionsSentFromDepartedProxy(id);
   }
 
   public void memberJoined(InternalDistributedMember id) {
@@ -951,9 +960,9 @@ public class TXManagerImpl implements CacheTransactionManager,
   
 
   /**
-   * retrieve the transaction states for the given client
+   * retrieve the transaction TXIds for the given client
    * @param id the client's membership ID
-   * @return a set of the currently open transaction states
+   * @return a set of the currently open TXIds
    */
   public Set<TXId> getTransactionsForClient(InternalDistributedMember id) {
     Set<TXId> result = new HashSet<TXId>();
@@ -966,6 +975,23 @@ public class TXManagerImpl implements CacheTransactionManager,
     }
     return result;
   }
+  
+  /**
+   * retrieve the transaction states for the given client
+   * @param id the client's membership ID
+   * @return a set of the currently open transaction states
+   */
+  public Set<TXStateProxy> getTransactionStatesForClient(InternalDistributedMember id) {
+    Set<TXStateProxy> result = new HashSet<TXStateProxy>();
+    synchronized (this.hostedTXStates) {
+      for (Map.Entry<TXId, TXStateProxy> entry: this.hostedTXStates.entrySet()) {
+        if (entry.getKey().getMemberId().equals(id)) {
+          result.add(entry.getValue());
+        }
+      }
+    }
+    return result;
+  }
 
   /** remove the given TXStates */
   public void removeTransactions(Set<TXId> txIds, boolean distribute) {
@@ -1500,6 +1526,112 @@ public class TXManagerImpl implements CacheTransactionManager,
     }
   }
   
+  public void setTransactionTimeToLiveForTest(int seconds) {
+    this.transactionTimeToLive = seconds;
+  }
+  
+  /**
+   * @return the time-to-live for abandoned transactions, in seconds
+   */
+  public int getTransactionTimeToLive() {
+    return this.transactionTimeToLive;
+  }
+  
+  public InternalDistributedMember getMemberId() {
+    return this.distributionMgrId;
+  }
+  
+  //expire the transaction states for the lost proxy server based on timeout setting.  
+  private void expireClientTransactionsSentFromDepartedProxy(InternalDistributedMember proxyServer) {
+    if (this.cache.isClosed()) {
+      return; 
+    }
+    long timeout = getTransactionTimeToLive() * 1000;
+    if (timeout <= 0) {
+      removeTransactionsSentFromDepartedProxy(proxyServer);
+    } else {
+      if (departedProxyServers != null) departedProxyServers.add(proxyServer);  
+      SystemTimerTask task = new SystemTimerTask() {
+        @Override
+        public void run2() {
+          removeTransactionsSentFromDepartedProxy(proxyServer);
+          if (departedProxyServers != null) departedProxyServers.remove(proxyServer);
+        }
+      };
+      try {
+        ((GemFireCacheImpl)this.cache).getCCPTimer().schedule(task, timeout);
+      } catch (IllegalStateException ise) {
+        if (!((GemFireCacheImpl)this.cache).isClosed()) {
+          throw ise;
+        }
+        //task not able to be scheduled due to cache is closing,
+        //do not set it in the test hook.
+        if (departedProxyServers != null) departedProxyServers.remove(proxyServer);
+      }
+    }
+  }
+  
+  private final Set<InternalDistributedMember> departedProxyServers = Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "trackScheduledToBeRemovedTx") ?
+      new ConcurrentHashSet<InternalDistributedMember>() : null;
+
+  /**
+   * provide a test hook to track departed peers
+   */
+  public Set<InternalDistributedMember> getDepartedProxyServers() {
+    return departedProxyServers;
+  }
+  
+  /**
+   * Find all client originated transactions sent from the departed proxy server.
+   * Remove them from the hostedTXStates map after the set TransactionTimeToLive period.
+   * @param proxyServer the departed proxy server
+   */
+  public void removeTransactionsSentFromDepartedProxy(InternalDistributedMember proxyServer) {
+    final Set<TXId> txIds = getTransactionsSentFromDepartedProxy(proxyServer);
+    if (txIds.isEmpty()) {
+      return;
+    }
+    if (logger.isDebugEnabled()) {
+      logger.debug("expiring the following transactions: {}", txIds);
+    }
+    synchronized (this.hostedTXStates) {
+      Iterator<Map.Entry<TXId, TXStateProxy>> iterator = this.hostedTXStates.entrySet().iterator();
+      while (iterator.hasNext()) {
+        Map.Entry<TXId,TXStateProxy> entry = iterator.next();
+        if (txIds.contains(entry.getKey())) {
+          //The TXState was not updated by any other proxy server, 
+          //The client would fail over to another proxy server.
+          //Remove it after waiting for transactionTimeToLive period.
+          entry.getValue().close();
+          iterator.remove();
+        }
+      }
+    }
+  }
+  
+  /*
+   * retrieve the transaction states for the given client from a certain proxy server.
+   * if transactions failed over, the new proxy server information should be stored
+   * in the TXState
+   * @param id the proxy server
+   * @return a set of the currently open transaction states
+   */
+  private Set<TXId> getTransactionsSentFromDepartedProxy(InternalDistributedMember proxyServer) {
+    Set<TXId> result = new HashSet<TXId>();
+    synchronized (this.hostedTXStates) {
+      for (Map.Entry<TXId, TXStateProxy> entry: this.hostedTXStates.entrySet()) {
+        TXStateProxy tx = entry.getValue();
+        if (tx.isRealDealLocal() && tx.isOnBehalfOfClient()) {
+          TXState txstate = (TXState) ((TXStateProxyImpl)tx).realDeal;          
+          if (proxyServer.equals(txstate.getProxyServer())) {
+            result.add(entry.getKey());
+          }
+        }
+      }
+    }
+    return result;
+  }
+  
   public void setDistributed(boolean flag) {
     checkClosed();
     TXStateProxy tx = getTXState();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXState.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXState.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXState.java
index c42f63c..d64426b 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXState.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXState.java
@@ -51,6 +51,7 @@ import com.gemstone.gemfire.cache.TransactionWriter;
 import com.gemstone.gemfire.cache.TransactionWriterException;
 import com.gemstone.gemfire.cache.UnsupportedOperationInTransactionException;
 import com.gemstone.gemfire.cache.client.internal.ServerRegionDataAccess;
+import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.distributed.TXManagerCancelledException;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.Assert;
@@ -138,6 +139,8 @@ public class TXState implements TXStateInterface {
   private Map<EventID, Boolean> seenResults = new HashMap<EventID, Boolean>();
 
   static final TXEntryState ENTRY_EXISTS = new TXEntryState();
+  
+  private volatile DistributedMember proxyServer;
 
   public TXState(TXStateProxy proxy,boolean onBehalfOfRemoteStub) 
   {
@@ -1849,4 +1852,12 @@ public class TXState implements TXStateInterface {
   public boolean isCreatedOnDistTxCoordinator() {
     return false;
   }
+  
+  public void setProxyServer(DistributedMember proxyServer) {
+    this.proxyServer = proxyServer;
+  }
+  
+  public DistributedMember getProxyServer() {
+    return this.proxyServer;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxy.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxy.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxy.java
index ba9866e..8f8d1cd 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxy.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxy.java
@@ -21,6 +21,7 @@ package com.gemstone.gemfire.internal.cache;
 
 import com.gemstone.gemfire.cache.client.internal.ServerRegionDataAccess;
 import com.gemstone.gemfire.distributed.DistributedMember;
+import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.cache.tx.TransactionalOperation.ServerRegionOperation;
 
 /**
@@ -92,4 +93,6 @@ public interface TXStateProxy extends TXStateInterface {
    * @param progress
    */
   public void setInProgress(boolean progress);
+  
+  public void updateProxyServer(InternalDistributedMember proxy);
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxyImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxyImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxyImpl.java
index 0939ab0..c4ebadd 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxyImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/TXStateProxyImpl.java
@@ -1044,4 +1044,13 @@ public class TXStateProxyImpl implements TXStateProxy {
   public boolean isCreatedOnDistTxCoordinator() {
     return false;
   }
+
+  @Override
+  public void updateProxyServer(InternalDistributedMember proxy) {
+    //only update in TXState if it has one
+    if (this.realDeal != null && this.realDeal.isRealDealLocal() 
+        && isOnBehalfOfClient()) {
+      ((TXState)this.realDeal).setProxyServer(proxy);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionMessage.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionMessage.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionMessage.java
index 14fce08..351638a 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionMessage.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/partitioned/PartitionMessage.java
@@ -343,7 +343,8 @@ public abstract class PartitionMessage extends DistributionMessage implements
             // NO DISTRIBUTED MESSAGING CAN BE DONE HERE!
             sendReply = false;
           } else if (tx.isInProgress()) {
-            sendReply = operateOnPartitionedRegion(dm, pr, startTime);        
+            sendReply = operateOnPartitionedRegion(dm, pr, startTime); 
+            tx.updateProxyServer(this.getSender());
           }  
         } finally {
           txMgr.unmasquerade(tx);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
index ffcb8c5..cfa6333 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImpl.java
@@ -270,7 +270,6 @@ public class AcceptorImpl extends Acceptor implements Runnable
                       int socketBufferSize, int maximumTimeBetweenPings,
                       InternalCache c, int maxConnections, int maxThreads,
                       int maximumMessageCount, int messageTimeToLive,
-                      int transactionTimeToLive,
                       ConnectionListener listener,List overflowAttributesList, 
                       boolean isGatewayReceiver, List<GatewayTransportFilter> transportFilter,
                       boolean tcpNoDelay)
@@ -487,7 +486,7 @@ public class AcceptorImpl extends Acceptor implements Runnable
     this.crHelper = new CachedRegionHelper(this.cache);
 
     this.clientNotifier = CacheClientNotifier.getInstance(cache, this.stats,
-        maximumMessageCount,messageTimeToLive, transactionTimeToLive,
+        maximumMessageCount,messageTimeToLive, 
         connectionListener,overflowAttributesList, isGatewayReceiver);
     this.socketBufferSize = socketBufferSize;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
index bdebb6b..aa61e76 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
@@ -135,6 +135,7 @@ public abstract class BaseCommand implements Command {
         try {
           tx = txMgr.masqueradeAs(msg, member, false);
           cmdExecute(msg, servConn, start);
+          tx.updateProxyServer(txMgr.getMemberId());
         } finally {
           txMgr.unmasquerade(tx);
         }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
index 30ab4a4..09ffd8f 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/CacheClientNotifier.java
@@ -79,7 +79,6 @@ public class CacheClientNotifier {
    * @param acceptorStats        
    * @param maximumMessageCount
    * @param messageTimeToLive 
-   * @param transactionTimeToLive - ttl for txstates for disconnected clients
    * @param listener 
    * @param overflowAttributesList 
    * @return A <code>CacheClientNotifier</code> instance
@@ -87,13 +86,11 @@ public class CacheClientNotifier {
   public static synchronized CacheClientNotifier getInstance(Cache cache,
       CacheServerStats acceptorStats,
       int maximumMessageCount, int messageTimeToLive,
-      int transactionTimeToLive,
       ConnectionListener listener, List overflowAttributesList, boolean isGatewayReceiver)
   {
     if (ccnSingleton == null) {
       ccnSingleton = new CacheClientNotifier(cache, acceptorStats, maximumMessageCount, 
-          messageTimeToLive, transactionTimeToLive,
-          listener, overflowAttributesList, isGatewayReceiver);
+          messageTimeToLive, listener, overflowAttributesList, isGatewayReceiver);
     }
     
     if (!isGatewayReceiver && ccnSingleton.getHaContainer() == null) {
@@ -114,9 +111,6 @@ public class CacheClientNotifier {
   public static CacheClientNotifier getInstance(){
     return ccnSingleton;
   }
-
-  /** the amount of time in seconds to keep a disconnected client's txstates around */
-  private final int transactionTimeToLive;
   
   /**
    * Writes a given message to the output stream
@@ -2080,15 +2074,13 @@ public class CacheClientNotifier {
    * @param acceptorStats
    * @param maximumMessageCount
    * @param messageTimeToLive
-   * @param transactionTimeToLive - ttl for txstates for disconnected clients
    * @param listener a listener which should receive notifications
    *          abouts queues being added or removed.
    * @param overflowAttributesList
    */
   private CacheClientNotifier(Cache cache, CacheServerStats acceptorStats, 
-      int maximumMessageCount, int messageTimeToLive, int transactionTimeToLive,
-      ConnectionListener listener,
-      List overflowAttributesList, boolean isGatewayReceiver) {
+      int maximumMessageCount, int messageTimeToLive, 
+      ConnectionListener listener, List overflowAttributesList, boolean isGatewayReceiver) {
     // Set the Cache
     this.setCache((GemFireCacheImpl)cache);
     this.acceptorStats = acceptorStats;
@@ -2104,7 +2096,6 @@ public class CacheClientNotifier {
 
     this.maximumMessageCount = maximumMessageCount;
     this.messageTimeToLive = messageTimeToLive;
-    this.transactionTimeToLive = transactionTimeToLive;
 
     // Initialize the statistics
     StatisticsFactory factory ;
@@ -2689,13 +2680,5 @@ public class CacheClientNotifier {
       }
     }
   }
-
-
-  /**
-   * @return the time-to-live for abandoned transactions, in seconds
-   */
-  public int getTransactionTimeToLive() {
-    return this.transactionTimeToLive;
-  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitor.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitor.java
index eb701fb..cdb0133 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/ClientHealthMonitor.java
@@ -281,7 +281,7 @@ public class ClientHealthMonitor {
 
   /**
    * expire the transaction states for the given client.  This uses the
-   * transactionTimeToLive setting that is inherited from the CacheServer.
+   * transactionTimeToLive setting that is inherited from the TXManagerImpl.
    * If that setting is non-positive we expire the states immediately
    * @param proxyID
    */
@@ -289,12 +289,11 @@ public class ClientHealthMonitor {
     final TXManagerImpl txMgr = (TXManagerImpl)this._cache.getCacheTransactionManager(); 
     final Set<TXId> txids = txMgr.getTransactionsForClient(
           (InternalDistributedMember)proxyID.getDistributedMember());
-    CacheClientNotifier notifier = CacheClientNotifier.getInstance();
-    if (notifier == null || this._cache.isClosed()) {
-      return; // notifier is null when shutting down
+    if (this._cache.isClosed()) {
+      return; 
     }
-    long timeout = notifier.getTransactionTimeToLive() * 1000;
-    if (txids.size() > 0) {
+    long timeout = txMgr.getTransactionTimeToLive() * 1000;
+    if (!txids.isEmpty()) {
       if (logger.isDebugEnabled()) {
         logger.debug("expiring {} transaction contexts for {} timeout={}", txids.size(), proxyID, timeout/1000);
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheServerCreation.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheServerCreation.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheServerCreation.java
index 22d684f..10f5366 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheServerCreation.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/xmlcache/CacheServerCreation.java
@@ -71,7 +71,6 @@ public class CacheServerCreation extends AbstractCacheServer {
     setMaximumTimeBetweenPings(other.getMaximumTimeBetweenPings());
     setMaximumMessageCount(other.getMaximumMessageCount());
     setMessageTimeToLive(other.getMessageTimeToLive());
-    //      setTransactionTimeToLive(other.getTransactionTimeToLive());  not implemented in CacheServer for v6.6
     setGroups(other.getGroups());
     setLoadProbe(other.getLoadProbe());
     setLoadPollInterval(other.getLoadPollInterval());

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
index 8f962f7..6d9a172 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/disttx/DistributedTransactionDUnitTest.java
@@ -138,8 +138,9 @@ public class DistributedTransactionDUnitTest extends JUnit4CacheTestCase {
         int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
         CacheServer s = getCache().addCacheServer();
         s.setPort(port);
-        ((CacheServerImpl) s).setTransactionTimeToLive(10);
         s.start();
+        TXManagerImpl txMgr = (TXManagerImpl) getCache().getCacheTransactionManager();
+        txMgr.setTransactionTimeToLiveForTest(10);
         return port;
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
index 860833e..69d3fe4 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
@@ -36,6 +36,9 @@ import java.util.concurrent.TimeUnit;
 
 import javax.naming.Context;
 import javax.transaction.UserTransaction;
+
+import com.jayway.awaitility.Awaitility;
+
 import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
 
 import com.gemstone.gemfire.cache.*;
@@ -87,6 +90,14 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
   protected void postSetUpClientServerTransactionDUnitTest() throws Exception {
   }
 
+
+  private Integer createRegionsAndStartServerWithTimeout(VM vm, boolean accessor, int txTimeoutSecs) {
+    return createRegionOnServerWithTimeout(vm, true, accessor, txTimeoutSecs);
+  }
+  private Integer createRegionOnServerWithTimeout(VM vm, final boolean startServer,
+      final boolean accessor, final int txTimeoutSecs) {
+    return createRegionOnServerWithTimeout(vm, startServer, accessor, 0, txTimeoutSecs);
+  }
   private Integer createRegionsAndStartServer(VM vm, boolean accessor) {
     return createRegionOnServer(vm, true, accessor);
   }
@@ -98,14 +109,20 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
     return createRegionOnServer(vm, startServer, accessor, 0);
   }
   private Integer createRegionOnServer(VM vm, final boolean startServer, final boolean accessor, final int redundantCopies) {
+    return createRegionOnServerWithTimeout(vm, startServer, accessor, redundantCopies, 10);
+  }
+  
+  private Integer createRegionOnServerWithTimeout(VM vm, final boolean startServer, final boolean accessor, 
+      final int redundantCopies, final int txTimeoutSecs) {
     return (Integer)vm.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         createRegion(accessor, redundantCopies, null);
+        TXManagerImpl txMgr = (TXManagerImpl) getCache().getCacheTransactionManager();
+        txMgr.setTransactionTimeToLiveForTest(txTimeoutSecs);
         if (startServer) {
           int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
           CacheServer s = getCache().addCacheServer();
           s.setPort(port);
-          ((CacheServerImpl)s).setTransactionTimeToLive(10);
           s.start();
           return port;
         }
@@ -126,11 +143,12 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
         InternalDistributedSystem system = getSystem(props);
         Cache cache = CacheFactory.create(system);
         cache.createRegion(OTHER_REGION,af.create());
+        TXManagerImpl txMgr = (TXManagerImpl) cache.getCacheTransactionManager();
+        txMgr.setTransactionTimeToLiveForTest(10);
         if (startServer) {
           int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
           CacheServer s = cache.addCacheServer();
           s.setPort(port);
-          ((CacheServerImpl)s).setTransactionTimeToLive(10);
           s.start();
           return port;
         }
@@ -162,9 +180,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
     vm.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         ClientCacheFactory ccf = new ClientCacheFactory();
-        ccf.addPoolServer("localhost"/*getServerHostName(Host.getHost(0))*/, port);
-        ccf.setPoolSubscriptionEnabled(false);
-        ccf.set(LOG_LEVEL, getDUnitLogLevel());
+        setCCF(port, ccf);
         // these settings were used to manually check that tx operation stats were being updated
         //ccf.set(STATISTIC_SAMPLING_ENABLED, "true");
         //ccf.set(STATISTIC_ARCHIVE_FILE, "clientStats.gfs");
@@ -195,10 +211,11 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
     // a client VM
     final int serverPort = (Integer)accessor.invoke(new SerializableCallable("create cache server") {
       public Object call() throws Exception {
+        TXManagerImpl txMgr = (TXManagerImpl) getCache().getCacheTransactionManager();
+        txMgr.setTransactionTimeToLiveForTest(10);
         int port = AvailablePort.getRandomAvailablePort(AvailablePort.SOCKET);
         CacheServer s = getCache().addCacheServer();
         s.setPort(port);
-        ((CacheServerImpl)s).setTransactionTimeToLive(10);
         s.start();
         return port;
       }
@@ -222,10 +239,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
     final int port1 = createRegionsAndStartServer(datastore1, false);
     System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "bridge.disableShufflingOfEndpoints", "true");
     ClientCacheFactory ccf = new ClientCacheFactory();
-    ccf.addPoolServer("localhost"/*getServerHostName(Host.getHost(0))*/, port1);
-    ccf.setPoolSubscriptionEnabled(false);
-
-    ccf.set(LOG_LEVEL, getDUnitLogLevel());
+    setCCF(port1, ccf);
 
     ClientCache cCache = getClientCache(ccf);
     
@@ -250,13 +264,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
 
     TXManagerImpl mgr = getGemfireCache().getTxManager();
     mgr.begin();
-    for (int i=0; i<5; i++) {
-      CustId custId = new CustId(i);
-      Customer cust = new Customer("name"+i, "address"+i);
-      getGemfireCache().getLogger().info("putting:"+custId);
-      pr.put(custId, cust);
-      r.put(i, "value"+i);
-    }
+    doTxOps(r, pr);
     boolean exceptionThrown = false;
     try {
       otherRegion.put("tx", "not allowed");
@@ -290,14 +298,12 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
     
     disconnectAllFromDS(); // some other VMs seem to be hanging around and have the region this tests uses
 
-    final int port1 = createRegionsAndStartServer(accessor, true);
-    createRegionOnServer(datastore, false, false);
+    final int port1 = createRegionsAndStartServerWithTimeout(accessor, true, 5);
+    createRegionOnServerWithTimeout(datastore, false, false, 5);
 
     System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "bridge.disableShufflingOfEndpoints", "true");
     ClientCacheFactory ccf = new ClientCacheFactory();
-    ccf.addPoolServer("localhost"/*getServerHostName(Host.getHost(0))*/, port1);
-    ccf.setPoolSubscriptionEnabled(false);
-    ccf.set(LOG_LEVEL, getDUnitLogLevel());
+    setCCF(port1, ccf);
     ClientCache cCache = getClientCache(ccf);
     ClientRegionFactory<CustId, Customer> custrf = cCache
       .createClientRegionFactory(cachingProxy ? ClientRegionShortcut.CACHING_PROXY : ClientRegionShortcut.PROXY);
@@ -308,13 +314,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
 
     TXManagerImpl mgr = getGemfireCache().getTxManager();
     mgr.begin();
-    for (int i=0; i<5; i++) {
-      CustId custId = new CustId(i);
-      Customer cust = new Customer("name"+i, "address"+i);
-      getGemfireCache().getLogger().info("putting:"+custId);
-      pr.put(custId, cust);
-      r.put(i, "value"+i);
-    }
+    doTxOps(r, pr);
 
     final DistributedMember myId = cCache.getDistributedSystem().getDistributedMember(); 
     
@@ -359,6 +359,188 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
       cCache.close();
     }
   }
+  
+  @Test
+  public void testCleanupAfterClientAndProxyFailure() {
+    Host host = Host.getHost(0);
+    VM accessor = host.getVM(0);
+    VM datastore = host.getVM(1);
+    final boolean cachingProxy = false;
+    
+    disconnectAllFromDS(); // some other VMs seem to be hanging around and have the region this tests uses
+
+    final int port1 = createRegionsAndStartServerWithTimeout(accessor, true, 5);
+    createRegionOnServerWithTimeout(datastore, false, false, 5);
+
+    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "bridge.disableShufflingOfEndpoints", "true");
+    ClientCacheFactory ccf = new ClientCacheFactory();
+    setCCF(port1, ccf);
+    ClientCache cCache = getClientCache(ccf);
+    ClientRegionFactory<CustId, Customer> custrf = cCache
+      .createClientRegionFactory(cachingProxy ? ClientRegionShortcut.CACHING_PROXY : ClientRegionShortcut.PROXY);
+    ClientRegionFactory<Integer, String> refrf = cCache
+      .createClientRegionFactory(cachingProxy ? ClientRegionShortcut.CACHING_PROXY : ClientRegionShortcut.PROXY);
+    Region<Integer, String> r = refrf.create(D_REFERENCE);
+    Region<CustId, Customer> pr = custrf.create(CUSTOMER);
+
+    TXManagerImpl mgr = getGemfireCache().getTxManager();
+    mgr.begin();
+    doTxOps(r, pr);
+
+    final DistributedMember myId = cCache.getDistributedSystem().getDistributedMember(); 
+    
+    SerializableCallable verifyExists = new SerializableCallable("verify txstate for client exists") {
+      public Object call() throws Exception {
+        TXManagerImpl txmgr = getGemfireCache().getTxManager();
+        Set states = txmgr.getTransactionsForClient((InternalDistributedMember)myId);
+        assertEquals(1, states.size()); // only one in-progress transaction
+        return null;
+      }
+    };
+
+    accessor.invoke(verifyExists);
+    datastore.invoke(verifyExists);
+    
+    accessor.invoke(()->closeCache());
+    accessor.invoke(()->disconnectFromDS());
+    
+    SerializableCallable verifyExpired = new SerializableCallable("verify txstate is expired") {
+      public Object call() throws Exception {
+        final TXManagerImpl txmgr = getGemfireCache().getTxManager();
+        return verifyTXStateExpired(myId, txmgr);
+      }
+    };
+    try {
+      datastore.invoke(verifyExpired);
+    } finally {
+      cCache.close();
+    }
+  }
+
+  void doTxOps(Region<Integer, String> r, Region<CustId, Customer> pr) {
+    for (int i=0; i<5; i++) {
+      CustId custId = new CustId(i);
+      Customer cust = new Customer("name"+i, "address"+i);
+      getGemfireCache().getLogger().info("putting:"+custId);
+      pr.put(custId, cust);
+      r.put(i, "value"+i);
+    }
+  }
+  
+  public static DistributedMember getVMDistributedMember() {
+    return InternalDistributedSystem.getAnyInstance().getDistributedMember();
+  }
+  
+  @SuppressWarnings("unchecked")
+  @Test
+  public void testFailoverAfterProxyFailure() throws InterruptedException {
+    Host host = Host.getHost(0);
+    VM accessor = host.getVM(0);
+    VM datastore = host.getVM(1);
+    VM accessor2 = host.getVM(2);
+    final boolean cachingProxy = false;
+    
+    disconnectAllFromDS(); // some other VMs seem to be hanging around and have the region this tests uses
+
+    int[] ports = new int[2];
+    ports[0] = createRegionsAndStartServerWithTimeout(accessor, true, 5);
+    ports[1] = createRegionsAndStartServerWithTimeout(accessor2, true, 5);
+    createRegionOnServerWithTimeout(datastore, false, false, 5);
+    
+    System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "bridge.disableShufflingOfEndpoints", "true");
+    ClientCacheFactory ccf = new ClientCacheFactory();
+    setCCF(ports, ccf);
+    ClientCache cCache = getClientCache(ccf);
+    ClientRegionFactory<CustId, Customer> custrf = cCache
+      .createClientRegionFactory(cachingProxy ? ClientRegionShortcut.CACHING_PROXY : ClientRegionShortcut.PROXY);
+    ClientRegionFactory<Integer, String> refrf = cCache
+      .createClientRegionFactory(cachingProxy ? ClientRegionShortcut.CACHING_PROXY : ClientRegionShortcut.PROXY);
+    Region<Integer, String> r = refrf.create(D_REFERENCE);
+    Region<CustId, Customer> pr = custrf.create(CUSTOMER);
+
+    TXManagerImpl mgr = getGemfireCache().getTxManager();
+    mgr.begin();
+    doTxOps(r, pr);
+
+    final DistributedMember myId = cCache.getDistributedSystem().getDistributedMember(); 
+    final DistributedMember accessorId = (DistributedMember)accessor.invoke(() -> ClientServerTransactionDUnitTest.getVMDistributedMember());
+    final DistributedMember accessor2Id = (DistributedMember)accessor2.invoke(() -> ClientServerTransactionDUnitTest.getVMDistributedMember());
+    
+    SerializableCallable verifyExists = new SerializableCallable("verify txstate for client exists") {
+      public Object call() throws Exception {
+        TXManagerImpl txmgr = getGemfireCache().getTxManager();
+        Set<TXId> states = txmgr.getTransactionsForClient((InternalDistributedMember)myId);
+        assertEquals(1, states.size()); // only one in-progress transaction
+        return null;
+      }
+    };
+
+    datastore.invoke(verifyExists);
+    
+    SerializableCallable getProxyServer = new SerializableCallable("get proxy server") {
+      public Object call() throws Exception {
+        final TXManagerImpl txmgr = getGemfireCache().getTxManager();
+        DistributedMember proxyServer = null;
+        TXStateProxyImpl tx = null;
+        Set<TXStateProxy> states = txmgr.getTransactionStatesForClient((InternalDistributedMember)myId);
+        assertEquals(1, states.size()); 
+        Iterator<TXStateProxy> iterator = states.iterator();
+        if (iterator.hasNext()) {
+          tx = (TXStateProxyImpl)iterator.next();
+          assertTrue(tx.isRealDealLocal());
+          proxyServer = ((TXState)tx.realDeal).getProxyServer();
+        }
+        return proxyServer;
+      }
+    };
+    
+    final DistributedMember proxy = (DistributedMember) datastore.invoke(getProxyServer);
+    
+    if (proxy.equals(accessorId)) {
+      accessor.invoke(()->closeCache());
+      accessor.invoke(()->disconnectFromDS());
+    } else {
+      assertTrue(proxy.equals(accessor2Id));
+      accessor2.invoke(()->closeCache());
+      accessor2.invoke(()->disconnectFromDS());
+    }
+    
+    doTxOps(r, pr);
+    
+    SerializableCallable verifyProxyServerChanged = new SerializableCallable("verify proxy server is updated") {
+      public Object call() throws Exception {
+        final TXManagerImpl txmgr = getGemfireCache().getTxManager();
+        TXStateProxyImpl tx = null;
+        Set<TXStateProxy> states = txmgr.getTransactionStatesForClient((InternalDistributedMember)myId);
+        assertEquals(1, states.size()); 
+        Iterator<TXStateProxy> iterator = states.iterator();
+        if (iterator.hasNext()) {
+          tx = (TXStateProxyImpl)iterator.next();
+          assertTrue(tx.isRealDealLocal());
+        }
+        return verifyProxyServerChanged(tx, proxy);
+      }
+    };
+    try {
+      datastore.invoke(verifyProxyServerChanged);
+    } finally {
+      cCache.close();
+    }
+  }
+
+  void setCCF(final int port1, ClientCacheFactory ccf) {
+    ccf.addPoolServer("localhost"/*getServerHostName(Host.getHost(0))*/, port1);
+    ccf.setPoolSubscriptionEnabled(false);
+    ccf.set(LOG_LEVEL, getDUnitLogLevel());
+  }
+  
+  void setCCF(final int[] ports, ClientCacheFactory ccf) {
+    for (int port: ports) {
+      ccf.addPoolServer("localhost", port);
+    }
+    ccf.setPoolSubscriptionEnabled(false);
+    ccf.set(LOG_LEVEL, getDUnitLogLevel());
+  }
 
   @Test
   public void testBasicCommitOnEmpty() {
@@ -1371,9 +1553,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
     client.invoke(new SerializableCallable() {
       public Object call() throws Exception {
         ClientCacheFactory ccf = new ClientCacheFactory();
-        ccf.addPoolServer("localhost"/*getServerHostName(Host.getHost(0))*/, port1);
-        ccf.setPoolSubscriptionEnabled(false);
-        ccf.set(LOG_LEVEL, getDUnitLogLevel());
+        setCCF(port1, ccf);
         ClientCache cCache = getClientCache(ccf);
         ClientRegionFactory<CustId, Customer> custrf = cCache
             .createClientRegionFactory(ClientRegionShortcut.PROXY);
@@ -2785,9 +2965,7 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
           System.setProperty(DistributionConfig.GEMFIRE_PREFIX + "bridge.disableShufflingOfEndpoints", "true");
           ClientCacheFactory ccf = new ClientCacheFactory();
           ccf.addPoolServer("localhost"/*getServerHostName(Host.getHost(0))*/, port);
-          ccf.addPoolServer("localhost", port2);
-          ccf.setPoolSubscriptionEnabled(false);
-          ccf.set(LOG_LEVEL, getDUnitLogLevel());
+          setCCF(port2, ccf);
           // these settings were used to manually check that tx operation stats were being updated
           //ccf.set(STATISTIC_SAMPLING_ENABLED, "true");
           //ccf.set(STATISTIC_ARCHIVE_FILE, "clientStats.gfs");
@@ -3473,4 +3651,32 @@ public class ClientServerTransactionDUnitTest extends RemoteTransactionDUnitTest
       }
     });
   }
+
+  Object verifyTXStateExpired(final DistributedMember myId, final TXManagerImpl txmgr) {
+    try {
+      Wait.waitForCriterion(new WaitCriterion() {
+        public boolean done() {
+          Set states = txmgr.getTransactionsForClient((InternalDistributedMember)myId);
+          com.gemstone.gemfire.test.dunit.LogWriterUtils.getLogWriter().info("found " + states.size() + " tx states for " + myId);
+          return states.isEmpty();
+        }
+        public String description() {
+          return "Waiting for transaction state to expire";
+        }
+      }, 15000, 500, true);
+      return null;
+    } finally {
+      getGemfireCache().getDistributedSystem().disconnect();
+    }
+  }
+  
+  Object verifyProxyServerChanged(final TXStateProxyImpl tx, final DistributedMember newProxy) {
+    try {
+      Awaitility.await().pollInterval(10, TimeUnit.MILLISECONDS).pollDelay(10, TimeUnit.MILLISECONDS)
+      .atMost(30, TimeUnit.SECONDS).until(() -> !((TXState)tx.realDeal).getProxyServer().equals(newProxy));
+      return null;
+    } finally {
+      getGemfireCache().getDistributedSystem().disconnect();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/87876a72/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImplJUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImplJUnitTest.java
index 7c52a89..b703f00 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImplJUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/internal/cache/tier/sockets/AcceptorImplJUnitTest.java
@@ -84,7 +84,7 @@ public class AcceptorImplJUnitTest
           AcceptorImpl.MINIMUM_MAX_CONNECTIONS - 1,
           CacheServer.DEFAULT_MAX_THREADS,
           CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null, false, Collections.EMPTY_LIST,
+          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,null,null, false, Collections.EMPTY_LIST,
           CacheServer.DEFAULT_TCP_NO_DELAY);
         fail("Expected an IllegalArgumentExcption due to max conns < min pool size");
       } catch (IllegalArgumentException expected) {
@@ -101,7 +101,7 @@ public class AcceptorImplJUnitTest
           0,
           CacheServer.DEFAULT_MAX_THREADS,
           CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null,false, Collections.EMPTY_LIST,
+          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,null,null,false, Collections.EMPTY_LIST,
           CacheServer.DEFAULT_TCP_NO_DELAY);
         fail("Expected an IllegalArgumentExcption due to max conns of zero");
       } catch (IllegalArgumentException expected) {
@@ -118,7 +118,7 @@ public class AcceptorImplJUnitTest
           AcceptorImpl.MINIMUM_MAX_CONNECTIONS,
           CacheServer.DEFAULT_MAX_THREADS,
           CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null,false, Collections.EMPTY_LIST,
+          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,null,null,false, Collections.EMPTY_LIST,
           CacheServer.DEFAULT_TCP_NO_DELAY);
         a2 = new AcceptorImpl(
           port1,
@@ -130,7 +130,7 @@ public class AcceptorImplJUnitTest
           AcceptorImpl.MINIMUM_MAX_CONNECTIONS,
           CacheServer.DEFAULT_MAX_THREADS,
           CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null,false, Collections.EMPTY_LIST,
+          CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,null,null,false, Collections.EMPTY_LIST,
           CacheServer.DEFAULT_TCP_NO_DELAY);
         fail("Expecetd a BindException while attaching to the same port");
       } catch (BindException expected) {
@@ -146,7 +146,7 @@ public class AcceptorImplJUnitTest
         AcceptorImpl.MINIMUM_MAX_CONNECTIONS,
         CacheServer.DEFAULT_MAX_THREADS,
         CacheServer.DEFAULT_MAXIMUM_MESSAGE_COUNT,
-        CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,0,null,null, false, Collections.EMPTY_LIST,
+        CacheServer.DEFAULT_MESSAGE_TIME_TO_LIVE,null,null, false, Collections.EMPTY_LIST,
         CacheServer.DEFAULT_TCP_NO_DELAY);
       assertEquals(port2, a3.getPort());
       InternalDistributedSystem isystem = (InternalDistributedSystem) this.cache.getDistributedSystem();


[44/50] [abbrv] incubator-geode git commit: GEODE-1751: putting security checks in all applicable client-server commands.

Posted by ji...@apache.org.
GEODE-1751: putting security checks in all applicable client-server commands.


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

Branch: refs/heads/feature/GEODE-1571
Commit: 536c13bdef16db663194b33bb7ebd64dd78216b5
Parents: e504d97
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Jun 24 22:59:32 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri Jun 24 22:59:32 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/command/Destroy65.java   | 247 ++++++------
 .../tier/sockets/command/ExecuteFunction.java   |   7 +-
 .../tier/sockets/command/ExecuteFunction65.java | 252 ++++++-------
 .../tier/sockets/command/ExecuteFunction66.java | 345 ++++++++---------
 .../sockets/command/ExecuteRegionFunction.java  | 235 ++++++------
 .../command/ExecuteRegionFunction65.java        | 357 ++++++++----------
 .../command/ExecuteRegionFunction66.java        | 374 ++++++++-----------
 .../command/ExecuteRegionFunctionSingleHop.java | 356 ++++++++----------
 .../cache/tier/sockets/command/GetAll.java      | 145 +++----
 .../cache/tier/sockets/command/GetAll651.java   | 157 ++++----
 .../cache/tier/sockets/command/GetAll70.java    | 292 ++++++++-------
 .../sockets/command/GetAllWithCallback.java     |  68 ++--
 .../sockets/command/GetFunctionAttribute.java   |  37 +-
 .../cache/tier/sockets/command/Invalidate.java  | 188 +++++-----
 .../cache/tier/sockets/command/KeySet.java      | 149 ++++----
 .../cache/tier/sockets/command/Put61.java       | 306 +++++++--------
 .../cache/tier/sockets/command/Put65.java       |   3 +-
 .../cache/tier/sockets/command/PutAll.java      |  49 +--
 .../cache/tier/sockets/command/PutAll70.java    |   5 +-
 .../cache/tier/sockets/command/PutAll80.java    |   1 +
 .../tier/sockets/command/RegisterInterest.java  |  42 +--
 .../sockets/command/RegisterInterest61.java     |  38 +-
 .../cache/tier/sockets/command/RemoveAll.java   |   4 +-
 .../sockets/command/UnregisterInterest.java     | 113 +++---
 .../sockets/command/UnregisterInterestList.java | 119 +++---
 .../internal/security/GeodeSecurityUtil.java    |   7 +
 .../gemfire/security/GeodePermission.java       |   6 +-
 ...tegratedClientGetPutAuthDistributedTest.java |  18 +-
 28 files changed, 1876 insertions(+), 2044 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Destroy65.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Destroy65.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Destroy65.java
index c88ea24..41cc0be 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Destroy65.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/Destroy65.java
@@ -188,145 +188,146 @@ public class Destroy65 extends BaseCommand {
       }
       writeErrorResponse(msg, MessageType.DESTROY_DATA_ERROR, errMessage.toString(), servConn);
       servConn.setAsTrue(RESPONDED);
-    } else {
-      LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = LocalizedStrings.Destroy__0_WAS_NOT_FOUND_DURING_DESTROY_REQUEST.toLocalizedString(regionName);
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      } else {
-        // Destroy the entry
-        ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
-        long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-        long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
-        EventID eventId = new EventID(servConn.getEventMemberIDByteArray(), threadId, sequenceId);
-        EventIDHolder clientEvent = new EventIDHolder(eventId);
-
-        Breadcrumbs.setEventId(eventId);
-
-        // msg.isRetry might be set by v7.0 and later clients
-        if (msg.isRetry()) {
-          //          if (logger.isDebugEnabled()) {
-          //            logger.debug("DEBUG: encountered isRetry in Destroy65");
-          //          }
-          clientEvent.setPossibleDuplicate(true);
-          if (region.getAttributes().getConcurrencyChecksEnabled()) {
-            // recover the version tag from other servers
-            clientEvent.setRegion(region);
-            if (!recoverVersionTagForRetriedOperation(clientEvent)) {
-              clientEvent.setPossibleDuplicate(false); // no-one has seen this event
-            }
-          }
-        }
+      return;
+    }
+
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = LocalizedStrings.Destroy__0_WAS_NOT_FOUND_DURING_DESTROY_REQUEST.toLocalizedString(regionName);
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    // for integrated security
+    GeodeSecurityUtil.authorizeRegionWrite(regionName, key.toString());
+
+    // Destroy the entry
+    ByteBuffer eventIdPartsBuffer = ByteBuffer.wrap(eventPart.getSerializedForm());
+    long threadId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    long sequenceId = EventID.readEventIdPartsFromOptmizedByteArray(eventIdPartsBuffer);
+    EventID eventId = new EventID(servConn.getEventMemberIDByteArray(), threadId, sequenceId);
+    EventIDHolder clientEvent = new EventIDHolder(eventId);
+
+    Breadcrumbs.setEventId(eventId);
 
-        // for integrated security
-        GeodeSecurityUtil.authorizeRegionWrite(regionName, key.toString());
+    // msg.isRetry might be set by v7.0 and later clients
+    if (msg.isRetry()) {
+      //          if (logger.isDebugEnabled()) {
+      //            logger.debug("DEBUG: encountered isRetry in Destroy65");
+      //          }
+      clientEvent.setPossibleDuplicate(true);
+      if (region.getAttributes().getConcurrencyChecksEnabled()) {
+        // recover the version tag from other servers
+        clientEvent.setRegion(region);
+        if (!recoverVersionTagForRetriedOperation(clientEvent)) {
+          clientEvent.setPossibleDuplicate(false); // no-one has seen this event
+        }
+      }
+    }
 
+    try {
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      if (authzRequest != null) {
+        // TODO SW: This is to handle DynamicRegionFactory destroy
+        // calls. Rework this when the semantics of DynamicRegionFactory are
+        // cleaned up.
+        if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
+          RegionDestroyOperationContext destroyContext = authzRequest.destroyRegionAuthorize((String) key, callbackArg);
+          callbackArg = destroyContext.getCallbackArg();
+        } else {
+          DestroyOperationContext destroyContext = authzRequest.destroyAuthorize(regionName, key, callbackArg);
+          callbackArg = destroyContext.getCallbackArg();
+        }
+      }
+      if (operation == null || operation == Operation.DESTROY) {
+        region.basicBridgeDestroy(key, callbackArg, servConn.getProxyID(), true, clientEvent);
+      } else {
+        // this throws exceptions if expectedOldValue checks fail
         try {
-          AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-          if (authzRequest != null) {
-            // TODO SW: This is to handle DynamicRegionFactory destroy
-            // calls. Rework this when the semantics of DynamicRegionFactory are
-            // cleaned up.
-            if (DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-              RegionDestroyOperationContext destroyContext = authzRequest.destroyRegionAuthorize((String) key, callbackArg);
-              callbackArg = destroyContext.getCallbackArg();
-            } else {
-              DestroyOperationContext destroyContext = authzRequest.destroyAuthorize(regionName, key, callbackArg);
-              callbackArg = destroyContext.getCallbackArg();
-            }
+          if (expectedOldValue == null) {
+            expectedOldValue = Token.INVALID;
           }
-          if (operation == null || operation == Operation.DESTROY) {
-            region.basicBridgeDestroy(key, callbackArg, servConn.getProxyID(), true, clientEvent);
-          } else {
-            // this throws exceptions if expectedOldValue checks fail
+          if (operation == Operation.REMOVE && msg.isRetry() && clientEvent.getVersionTag() != null) {
+            // the operation was successful last time it was tried, so there's
+            // no need to perform it again.  Just return the version tag and
+            // success status
+            if (logger.isDebugEnabled()) {
+              logger.debug("remove(k,v) operation was successful last time with version {}", clientEvent.getVersionTag());
+            }
+            // try the operation anyway to ensure that it's been distributed to all servers
             try {
-              if (expectedOldValue == null) {
-                expectedOldValue = Token.INVALID;
-              }
-              if (operation == Operation.REMOVE && msg.isRetry() && clientEvent.getVersionTag() != null) {
-                // the operation was successful last time it was tried, so there's
-                // no need to perform it again.  Just return the version tag and
-                // success status
-                if (logger.isDebugEnabled()) {
-                  logger.debug("remove(k,v) operation was successful last time with version {}", clientEvent.getVersionTag());
-                }
-                // try the operation anyway to ensure that it's been distributed to all servers
-                try {
-                  region.basicBridgeRemove(key, expectedOldValue, callbackArg, servConn.getProxyID(), true, clientEvent);
-                } catch (EntryNotFoundException e) {
-                  // ignore, and don't set entryNotFoundForRemove because this was a successful
-                  // operation - bug #51664
-                }
-              } else {
-                region.basicBridgeRemove(key, expectedOldValue, callbackArg, servConn.getProxyID(), true, clientEvent);
-                if (logger.isDebugEnabled()) {
-                  logger.debug("region.remove succeeded");
-                }
-              }
+              region.basicBridgeRemove(key, expectedOldValue, callbackArg, servConn.getProxyID(), true, clientEvent);
             } catch (EntryNotFoundException e) {
-              servConn.setModificationInfo(true, regionName, key);
-              if (logger.isDebugEnabled()) {
-                logger.debug("writing entryNotFound response");
-              }
-              entryNotFoundForRemove = true;
+              // ignore, and don't set entryNotFoundForRemove because this was a successful
+              // operation - bug #51664
             }
-          }
-          servConn.setModificationInfo(true, regionName, key);
-        } catch (EntryNotFoundException e) {
-          // Don't send an exception back to the client if this
-          // exception happens. Just log it and continue.
-          logger.info(LocalizedMessage.create(LocalizedStrings.Destroy_0_DURING_ENTRY_DESTROY_NO_ENTRY_WAS_FOUND_FOR_KEY_1, new Object[] {
-            servConn.getName(),
-            key
-          }));
-          entryNotFoundForRemove = true;
-        } catch (RegionDestroyedException rde) {
-          writeException(msg, rde, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        } catch (Exception e) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, e);
-
-          // If an exception occurs during the destroy, preserve the connection
-          writeException(msg, e, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          if (e instanceof GemFireSecurityException) {
-            // Fine logging for security exceptions since these are already
-            // logged by the security logger
+          } else {
+            region.basicBridgeRemove(key, expectedOldValue, callbackArg, servConn.getProxyID(), true, clientEvent);
             if (logger.isDebugEnabled()) {
-              logger.debug("{}: Unexpected Security exception", servConn.getName(), e);
+              logger.debug("region.remove succeeded");
             }
-          } else {
-            logger.warn(LocalizedMessage.create(LocalizedStrings.Destroy_0_UNEXPECTED_EXCEPTION, servConn.getName()), e);
           }
-          return;
-        }
-
-        // Update the statistics and write the reply
-        now = DistributionStats.getStatTime();
-        stats.incProcessDestroyTime(now - start);
-
-        if (region instanceof PartitionedRegion) {
-          PartitionedRegion pr = (PartitionedRegion) region;
-          if (pr.isNetworkHop() != (byte) 0) {
-            writeReplyWithRefreshMetadata(msg, servConn, pr, entryNotFoundForRemove, pr.isNetworkHop(), clientEvent.getVersionTag());
-            pr.setIsNetworkHop((byte) 0);
-            pr.setMetadataVersion((byte) 0);
-          } else {
-            writeReply(msg, servConn, entryNotFoundForRemove | clientEvent.getIsRedestroyedEntry(), clientEvent.getVersionTag());
+        } catch (EntryNotFoundException e) {
+          servConn.setModificationInfo(true, regionName, key);
+          if (logger.isDebugEnabled()) {
+            logger.debug("writing entryNotFound response");
           }
-        } else {
-          writeReply(msg, servConn, entryNotFoundForRemove | clientEvent.getIsRedestroyedEntry(), clientEvent.getVersionTag());
+          entryNotFoundForRemove = true;
         }
-        servConn.setAsTrue(RESPONDED);
+      }
+      servConn.setModificationInfo(true, regionName, key);
+    } catch (EntryNotFoundException e) {
+      // Don't send an exception back to the client if this
+      // exception happens. Just log it and continue.
+      logger.info(LocalizedMessage.create(LocalizedStrings.Destroy_0_DURING_ENTRY_DESTROY_NO_ENTRY_WAS_FOUND_FOR_KEY_1, new Object[] {
+        servConn.getName(), key
+      }));
+      entryNotFoundForRemove = true;
+    } catch (RegionDestroyedException rde) {
+      writeException(msg, rde, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
+
+      // If an exception occurs during the destroy, preserve the connection
+      writeException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      if (e instanceof GemFireSecurityException) {
+        // Fine logging for security exceptions since these are already
+        // logged by the security logger
         if (logger.isDebugEnabled()) {
-          logger.debug("{}: Sent destroy response for region {} key {}", servConn.getName(), regionName, key);
+          logger.debug("{}: Unexpected Security exception", servConn.getName(), e);
         }
-        stats.incWriteDestroyResponseTime(DistributionStats.getStatTime() - start);
+      } else {
+        logger.warn(LocalizedMessage.create(LocalizedStrings.Destroy_0_UNEXPECTED_EXCEPTION, servConn.getName()), e);
+      }
+      return;
+    }
+
+    // Update the statistics and write the reply
+    now = DistributionStats.getStatTime();
+    stats.incProcessDestroyTime(now - start);
+
+    if (region instanceof PartitionedRegion) {
+      PartitionedRegion pr = (PartitionedRegion) region;
+      if (pr.isNetworkHop() != (byte) 0) {
+        writeReplyWithRefreshMetadata(msg, servConn, pr, entryNotFoundForRemove, pr.isNetworkHop(), clientEvent.getVersionTag());
+        pr.setIsNetworkHop((byte) 0);
+        pr.setMetadataVersion((byte) 0);
+      } else {
+        writeReply(msg, servConn, entryNotFoundForRemove | clientEvent.getIsRedestroyedEntry(), clientEvent.getVersionTag());
       }
+    } else {
+      writeReply(msg, servConn, entryNotFoundForRemove | clientEvent.getIsRedestroyedEntry(), clientEvent.getVersionTag());
+    }
+    servConn.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Sent destroy response for region {} key {}", servConn.getName(), regionName, key);
     }
+    stats.incWriteDestroyResponseTime(DistributionStats.getStatTime() - start);
 
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction.java
index 8ceb001..0f3bdec 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction.java
@@ -53,6 +53,7 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 /**
  * This is the base command which read the parts for the
@@ -110,7 +111,7 @@ public class ExecuteFunction extends BaseCommand {
       sendError(hasResult, msg, message, servConn);
       return;
     }
-    else {
+
       // Execute function on the cache
       try {
         Function functionObject = null;
@@ -128,6 +129,9 @@ public class ExecuteFunction extends BaseCommand {
         else {
           functionObject = (Function)function;
         }
+
+        GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+
         FunctionStats stats = FunctionStats.getFunctionStats(functionObject.getId(), null);
         
         // check if the caller is authorized to do this operation on server
@@ -217,7 +221,6 @@ public class ExecuteFunction extends BaseCommand {
         final String message = e.getMessage();
         sendException(hasResult, msg, message, servConn,e);
       }
-    }
   }
 
   private void sendException(byte hasResult, Message msg, String message,

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction65.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction65.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction65.java
index d2869d1..ff6cdd6 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction65.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction65.java
@@ -52,6 +52,7 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 /**
  * @since GemFire 6.5
@@ -68,8 +69,7 @@ public class ExecuteFunction65 extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException {
     Object function = null;
     Object args = null;
     MemberMappedArgument memberMappedArg = null;
@@ -86,11 +86,10 @@ public class ExecuteFunction65 extends BaseCommand {
         functionState = AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE;
         isReexecute = true;
       }
-      
-      if(functionState != 1) {
+
+      if (functionState != 1) {
         hasResult = (byte) ((functionState & 2) - 1);
-      }
-      else  {
+      } else {
         hasResult = functionState;
       }
       if (hasResult == 1) {
@@ -99,13 +98,12 @@ public class ExecuteFunction65 extends BaseCommand {
       }
       function = msg.getPart(1).getStringOrObject();
       args = msg.getPart(2).getObject();
-      
+
       Part part = msg.getPart(3);
       if (part != null) {
-        memberMappedArg = (MemberMappedArgument)part.getObject();
+        memberMappedArg = (MemberMappedArgument) part.getObject();
       }
-    }
-    catch (ClassNotFoundException exception) {
+    } catch (ClassNotFoundException exception) {
       logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), exception);
       if (hasResult == 1) {
         writeChunkedException(msg, exception, false, servConn);
@@ -114,157 +112,137 @@ public class ExecuteFunction65 extends BaseCommand {
       }
     }
     if (function == null) {
-      final String message = 
-        LocalizedStrings.ExecuteFunction_THE_INPUT_FUNCTION_FOR_THE_EXECUTE_FUNCTION_REQUEST_IS_NULL
-          .toLocalizedString();
+      final String message = LocalizedStrings.ExecuteFunction_THE_INPUT_FUNCTION_FOR_THE_EXECUTE_FUNCTION_REQUEST_IS_NULL
+        .toLocalizedString();
       logger.warn("{}: {}", servConn.getName(), message);
       sendError(hasResult, msg, message, servConn);
       return;
     }
-    else {
-      // Execute function on the cache
-      try {
-        Function functionObject = null;
-        if (function instanceof String) {
-          functionObject = FunctionService.getFunction((String)function);
-          if (functionObject == null) {
-            final String message = 
-              LocalizedStrings.ExecuteFunction_FUNCTION_NAMED_0_IS_NOT_REGISTERED
-                .toLocalizedString(function);
-            logger.warn("{}: {}", servConn.getName(), message);
-            sendError(hasResult, msg, message, servConn);
-            return;
-          }
-          else {
-            byte functionStateOnServerSide = AbstractExecution.getFunctionState(
-                functionObject.isHA(), functionObject.hasResult(),
-                functionObject.optimizeForWrite());
-            if (logger.isDebugEnabled()) {
-              logger.debug("Function State on server side: {} on client: {}", functionStateOnServerSide, functionState);
-            }
-            if (functionStateOnServerSide != functionState) {
-              String message = LocalizedStrings.FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER
-                  .toLocalizedString(function);
-              logger.warn("{}: {}", servConn.getName(), message);
-              sendError(hasResult, msg, message, servConn);
-              return;
-            }
-          }
-        }
-        else {
-          functionObject = (Function)function;
-        }
-        FunctionStats stats = FunctionStats.getFunctionStats(functionObject.getId(), null);
-        // check if the caller is authorized to do this operation on server
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        ExecuteFunctionOperationContext executeContext = null;
-        if (authzRequest != null) {
-          executeContext = authzRequest.executeFunctionAuthorize(functionObject
-              .getId(), null, null, args, functionObject.optimizeForWrite());
-        }
-        ChunkedMessage m = servConn.getFunctionResponseMessage();
-        m.setTransactionId(msg.getTransactionId());
-        ResultSender resultSender = new ServerToClientFunctionResultSender65(m,
-            MessageType.EXECUTE_FUNCTION_RESULT, servConn, functionObject, executeContext);
-        
-        InternalDistributedMember localVM = InternalDistributedSystem
-            .getAnyInstance().getDistributedMember();
-        FunctionContext context = null;
 
-        if (memberMappedArg != null) {
-          context = new FunctionContextImpl(functionObject.getId(),
-              memberMappedArg.getArgumentsForMember(localVM.getId()),
-              resultSender, isReexecute);
-        }
-        else {
-          context = new FunctionContextImpl(functionObject.getId(), args,
-              resultSender, isReexecute);
-        }
-        HandShake handShake = (HandShake)servConn.getHandshake();
-        int earlierClientReadTimeout = handShake.getClientReadTimeout();
-        handShake.setClientReadTimeout(0);
-        try {
-          long startExecution = stats.startTime();
-          stats.startFunctionExecution(functionObject.hasResult());
+    // Execute function on the cache
+    try {
+      Function functionObject = null;
+      if (function instanceof String) {
+        functionObject = FunctionService.getFunction((String) function);
+        if (functionObject == null) {
+          final String message = LocalizedStrings.ExecuteFunction_FUNCTION_NAMED_0_IS_NOT_REGISTERED.toLocalizedString(function);
+          logger.warn("{}: {}", servConn.getName(), message);
+          sendError(hasResult, msg, message, servConn);
+          return;
+        } else {
+          byte functionStateOnServerSide = AbstractExecution.getFunctionState(functionObject.isHA(), functionObject.hasResult(), functionObject
+            .optimizeForWrite());
           if (logger.isDebugEnabled()) {
-            logger.debug("Executing Function on Server: {} with context: {}", servConn, context);
+            logger.debug("Function State on server side: {} on client: {}", functionStateOnServerSide, functionState);
           }
-          GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
-          HeapMemoryMonitor hmm = ((InternalResourceManager) cache.getResourceManager()).getHeapMonitor();
-          if (functionObject.optimizeForWrite() && cache != null &&
-              hmm.getState().isCritical() &&
-              !MemoryThresholds.isLowMemoryExceptionDisabled()) {
-            Set<DistributedMember> sm = Collections.singleton((DistributedMember)cache.getMyId());
-            Exception e = new LowMemoryException(LocalizedStrings.ResourceManager_LOW_MEMORY_FOR_0_FUNCEXEC_MEMBERS_1
-                .toLocalizedString(new Object[] {functionObject.getId(), sm}), sm);
-            
-            sendException(hasResult, msg, e.getMessage(), servConn, e);
+          if (functionStateOnServerSide != functionState) {
+            String message = LocalizedStrings.FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER.toLocalizedString(function);
+            logger.warn("{}: {}", servConn.getName(), message);
+            sendError(hasResult, msg, message, servConn);
             return;
           }
-          functionObject.execute(context);
-          if (!((ServerToClientFunctionResultSender65)resultSender)
-              .isLastResultReceived() && functionObject.hasResult()) { 
-            throw new FunctionException( 
-                LocalizedStrings.ExecuteFunction_THE_FUNCTION_0_DID_NOT_SENT_LAST_RESULT
-                .toString(functionObject.getId())); 
-          } 
-          stats.endFunctionExecution(startExecution, 
-              functionObject.hasResult()); 
-        }
-        catch (FunctionException functionException) {
-          stats.endFunctionExecutionWithException(functionObject.hasResult());
-          throw functionException;
-        }
-        catch (Exception exception) {
-          stats.endFunctionExecutionWithException(functionObject.hasResult());
-          throw new FunctionException(exception);
-        }
-        finally{
-          handShake.setClientReadTimeout(earlierClientReadTimeout);
         }
+      } else {
+        functionObject = (Function) function;
       }
-      catch (IOException ioException) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioException);
-        String message = LocalizedStrings.ExecuteFunction_SERVER_COULD_NOT_SEND_THE_REPLY.toLocalizedString();
-        sendException(hasResult, msg, message, servConn, ioException);
+
+      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+
+      FunctionStats stats = FunctionStats.getFunctionStats(functionObject.getId(), null);
+      // check if the caller is authorized to do this operation on server
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      ExecuteFunctionOperationContext executeContext = null;
+      if (authzRequest != null) {
+        executeContext = authzRequest.executeFunctionAuthorize(functionObject.getId(), null, null, args, functionObject.optimizeForWrite());
       }
-      catch (InternalFunctionInvocationTargetException internalfunctionException) {
-        // Fix for #44709: User should not be aware of
-        // InternalFunctionInvocationTargetException. No instance of
-        // InternalFunctionInvocationTargetException is giving useful
-        // information to user to take any corrective action hence logging
-        // this at fine level logging
-        // 1> In case of HA FucntionInvocationTargetException thrown. Since
-        // it is HA, function will be reexecuted on right node
-        // 2> in case of HA member departed
+      ChunkedMessage m = servConn.getFunctionResponseMessage();
+      m.setTransactionId(msg.getTransactionId());
+      ResultSender resultSender = new ServerToClientFunctionResultSender65(m, MessageType.EXECUTE_FUNCTION_RESULT, servConn, functionObject, executeContext);
+
+      InternalDistributedMember localVM = InternalDistributedSystem.getAnyInstance().getDistributedMember();
+      FunctionContext context = null;
+
+      if (memberMappedArg != null) {
+        context = new FunctionContextImpl(functionObject.getId(), memberMappedArg.getArgumentsForMember(localVM.getId()), resultSender, isReexecute);
+      } else {
+        context = new FunctionContextImpl(functionObject.getId(), args, resultSender, isReexecute);
+      }
+      HandShake handShake = (HandShake) servConn.getHandshake();
+      int earlierClientReadTimeout = handShake.getClientReadTimeout();
+      handShake.setClientReadTimeout(0);
+      try {
+        long startExecution = stats.startTime();
+        stats.startFunctionExecution(functionObject.hasResult());
         if (logger.isDebugEnabled()) {
-          logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] { function }), internalfunctionException);
+          logger.debug("Executing Function on Server: {} with context: {}", servConn, context);
         }
-        final String message = internalfunctionException.getMessage();
-        sendException(hasResult, msg, message, servConn, internalfunctionException);
-      } 
-      catch (Exception e) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
-        final String message = e.getMessage();
-        sendException(hasResult, msg, message, servConn,e);
+        GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+        HeapMemoryMonitor hmm = ((InternalResourceManager) cache.getResourceManager()).getHeapMonitor();
+        if (functionObject.optimizeForWrite() && cache != null &&
+            hmm.getState().isCritical() &&
+            !MemoryThresholds.isLowMemoryExceptionDisabled()) {
+          Set<DistributedMember> sm = Collections.singleton((DistributedMember) cache.getMyId());
+          Exception e = new LowMemoryException(LocalizedStrings.ResourceManager_LOW_MEMORY_FOR_0_FUNCEXEC_MEMBERS_1.toLocalizedString(new Object[] {
+            functionObject.getId(),
+            sm
+          }), sm);
+
+          sendException(hasResult, msg, e.getMessage(), servConn, e);
+          return;
+        }
+        functionObject.execute(context);
+        if (!((ServerToClientFunctionResultSender65) resultSender).isLastResultReceived() && functionObject.hasResult()) {
+          throw new FunctionException(LocalizedStrings.ExecuteFunction_THE_FUNCTION_0_DID_NOT_SENT_LAST_RESULT.toString(functionObject
+            .getId()));
+        }
+        stats.endFunctionExecution(startExecution, functionObject.hasResult());
+      } catch (FunctionException functionException) {
+        stats.endFunctionExecutionWithException(functionObject.hasResult());
+        throw functionException;
+      } catch (Exception exception) {
+        stats.endFunctionExecutionWithException(functionObject.hasResult());
+        throw new FunctionException(exception);
+      } finally {
+        handShake.setClientReadTimeout(earlierClientReadTimeout);
+      }
+    } catch (IOException ioException) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioException);
+      String message = LocalizedStrings.ExecuteFunction_SERVER_COULD_NOT_SEND_THE_REPLY.toLocalizedString();
+      sendException(hasResult, msg, message, servConn, ioException);
+    } catch (InternalFunctionInvocationTargetException internalfunctionException) {
+      // Fix for #44709: User should not be aware of
+      // InternalFunctionInvocationTargetException. No instance of
+      // InternalFunctionInvocationTargetException is giving useful
+      // information to user to take any corrective action hence logging
+      // this at fine level logging
+      // 1> In case of HA FucntionInvocationTargetException thrown. Since
+      // it is HA, function will be reexecuted on right node
+      // 2> in case of HA member departed
+      if (logger.isDebugEnabled()) {
+        logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] {
+          function
+        }), internalfunctionException);
       }
+      final String message = internalfunctionException.getMessage();
+      sendException(hasResult, msg, message, servConn, internalfunctionException);
+    } catch (Exception e) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
+      final String message = e.getMessage();
+      sendException(hasResult, msg, message, servConn, e);
     }
   }
 
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection servConn, Throwable e) throws IOException {
+  private void sendException(byte hasResult, Message msg, String message, ServerConnection servConn, Throwable e)
+    throws IOException {
     if (hasResult == 1) {
-      writeFunctionResponseException(msg, MessageType.EXCEPTION, message,
-          servConn, e);
+      writeFunctionResponseException(msg, MessageType.EXCEPTION, message, servConn, e);
       servConn.setAsTrue(RESPONDED);
     }
   }
 
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection servConn) throws IOException {
+  private void sendError(byte hasResult, Message msg, String message, ServerConnection servConn) throws IOException {
     if (hasResult == 1) {
-      writeFunctionResponseError(msg, MessageType.EXECUTE_FUNCTION_ERROR,
-          message, servConn);
+      writeFunctionResponseError(msg, MessageType.EXECUTE_FUNCTION_ERROR, message, servConn);
       servConn.setAsTrue(RESPONDED);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction66.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction66.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction66.java
index 3a20bc0..d5f3660 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction66.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteFunction66.java
@@ -62,6 +62,7 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 /**
  * @since GemFire 6.6
@@ -72,18 +73,16 @@ public class ExecuteFunction66 extends BaseCommand {
 
   protected static volatile boolean ASYNC_TX_WARNING_ISSUED = false;
 
-  static final ExecutorService execService = Executors
-      .newCachedThreadPool(new ThreadFactory() {
-        AtomicInteger threadNum = new AtomicInteger();
+  static final ExecutorService execService = Executors.newCachedThreadPool(new ThreadFactory() {
+    AtomicInteger threadNum = new AtomicInteger();
+
+    public Thread newThread(final Runnable r) {
+      Thread result = new Thread(r, "Function Execution Thread-" + threadNum.incrementAndGet());
+      result.setDaemon(true);
+      return result;
+    }
+  });
 
-        public Thread newThread(final Runnable r) {
-          Thread result = new Thread(r, "Function Execution Thread-"
-              + threadNum.incrementAndGet());
-          result.setDaemon(true);
-          return result;
-        }
-      });
-  
   public static Command getCommand() {
     return singleton;
   }
@@ -92,8 +91,7 @@ public class ExecuteFunction66 extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException {
     Object function = null;
     Object args = null;
     MemberMappedArgument memberMappedArg = null;
@@ -114,16 +112,14 @@ public class ExecuteFunction66 extends BaseCommand {
       if (functionState == AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE_REEXECUTE) {
         functionState = AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE;
         isReexecute = true;
-      }
-      else if (functionState == AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE_REEXECUTE) {
+      } else if (functionState == AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE_REEXECUTE) {
         functionState = AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE;
         isReexecute = true;
       }
 
       if (functionState != 1) {
-        hasResult = (byte)((functionState & 2) - 1);
-      }
-      else {
+        hasResult = (byte) ((functionState & 2) - 1);
+      } else {
         hasResult = functionState;
       }
       if (hasResult == 1) {
@@ -135,173 +131,153 @@ public class ExecuteFunction66 extends BaseCommand {
 
       Part part = msg.getPart(3);
       if (part != null) {
-        memberMappedArg = (MemberMappedArgument)part.getObject();
+        memberMappedArg = (MemberMappedArgument) part.getObject();
       }
 
       groups = getGroups(msg);
       allMembers = getAllMembers(msg);
       ignoreFailedMembers = getIgnoreFailedMembers(msg);
-    }
-    catch (ClassNotFoundException exception) {
+    } catch (ClassNotFoundException exception) {
       logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), exception);
       if (hasResult == 1) {
         writeChunkedException(msg, exception, false, servConn);
-      }
-      else {
+      } else {
         writeException(msg, exception, false, servConn);
       }
       servConn.setAsTrue(RESPONDED);
       return;
     }
+
     if (function == null) {
       final String message = LocalizedStrings.ExecuteFunction_THE_INPUT_FUNCTION_FOR_THE_EXECUTE_FUNCTION_REQUEST_IS_NULL
-          .toLocalizedString();
-      logger.warn(LocalizedMessage.create(LocalizedStrings.TWO_ARG_COLON, new Object[] { servConn.getName(), message }));
+        .toLocalizedString();
+      logger.warn(LocalizedMessage.create(LocalizedStrings.TWO_ARG_COLON, new Object[] {
+        servConn.getName(),
+        message
+      }));
       sendError(hasResult, msg, message, servConn);
       return;
     }
-    else {
-      // Execute function on the cache
-      try {
-        Function functionObject = null;
-        if (function instanceof String) {
-          functionObject = FunctionService.getFunction((String)function);
-          if (functionObject == null) {
-            final String message = LocalizedStrings.ExecuteFunction_FUNCTION_NAMED_0_IS_NOT_REGISTERED
-                .toLocalizedString(function);
+
+    // Execute function on the cache
+    try {
+      Function functionObject = null;
+      if (function instanceof String) {
+        functionObject = FunctionService.getFunction((String) function);
+        if (functionObject == null) {
+          final String message = LocalizedStrings.ExecuteFunction_FUNCTION_NAMED_0_IS_NOT_REGISTERED.toLocalizedString(function);
+          logger.warn("{}: {}", servConn.getName(), message);
+          sendError(hasResult, msg, message, servConn);
+          return;
+        } else {
+          byte functionStateOnServerSide = AbstractExecution.getFunctionState(functionObject.isHA(), functionObject.hasResult(), functionObject
+            .optimizeForWrite());
+          if (logger.isDebugEnabled()) {
+            logger.debug("Function State on server side: {} on client: {}", functionStateOnServerSide, functionState);
+          }
+          if (functionStateOnServerSide != functionState) {
+            String message = LocalizedStrings.FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER.toLocalizedString(function);
             logger.warn("{}: {}", servConn.getName(), message);
             sendError(hasResult, msg, message, servConn);
             return;
           }
-          else {
-            byte functionStateOnServerSide = AbstractExecution
-                .getFunctionState(functionObject.isHA(), functionObject
-                    .hasResult(), functionObject.optimizeForWrite());
-            if (logger.isDebugEnabled()) {
-              logger.debug("Function State on server side: {} on client: {}", functionStateOnServerSide, functionState);
-            }
-            if (functionStateOnServerSide != functionState) {
-              String message = LocalizedStrings.FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER
-                  .toLocalizedString(function);
-              logger.warn("{}: {}", servConn.getName(), message);
-              sendError(hasResult, msg, message, servConn);
-              return;
-            }
-          }
         }
-        else {
-          functionObject = (Function)function;
-        }
-        FunctionStats stats = FunctionStats.getFunctionStats(functionObject
-            .getId(), null);
-        // check if the caller is authorized to do this operation on server
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        ExecuteFunctionOperationContext executeContext = null;
-        if (authzRequest != null) {
-          executeContext = authzRequest.executeFunctionAuthorize(functionObject
-              .getId(), null, null, args, functionObject.optimizeForWrite());
-        }
-        ChunkedMessage m = servConn.getFunctionResponseMessage();
-        m.setTransactionId(msg.getTransactionId());
-        ServerToClientFunctionResultSender resultSender = new ServerToClientFunctionResultSender65(m,
-            MessageType.EXECUTE_FUNCTION_RESULT, servConn, functionObject,
-            executeContext);
-
-        InternalDistributedMember localVM = InternalDistributedSystem
-            .getAnyInstance().getDistributedMember();
-        FunctionContext context = null;
-
-        if (memberMappedArg != null) {
-          context = new FunctionContextImpl(functionObject.getId(),
-              memberMappedArg.getArgumentsForMember(localVM.getId()),
-              resultSender, isReexecute);
-        }
-        else {
-          context = new FunctionContextImpl(functionObject.getId(), args,
-              resultSender, isReexecute);
+      } else {
+        functionObject = (Function) function;
+      }
+
+      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+
+      FunctionStats stats = FunctionStats.getFunctionStats(functionObject.getId(), null);
+      // check if the caller is authorized to do this operation on server
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      ExecuteFunctionOperationContext executeContext = null;
+      if (authzRequest != null) {
+        executeContext = authzRequest.executeFunctionAuthorize(functionObject.getId(), null, null, args, functionObject.optimizeForWrite());
+      }
+      ChunkedMessage m = servConn.getFunctionResponseMessage();
+      m.setTransactionId(msg.getTransactionId());
+      ServerToClientFunctionResultSender resultSender = new ServerToClientFunctionResultSender65(m, MessageType.EXECUTE_FUNCTION_RESULT, servConn, functionObject, executeContext);
+
+      InternalDistributedMember localVM = InternalDistributedSystem.getAnyInstance().getDistributedMember();
+      FunctionContext context = null;
+
+      if (memberMappedArg != null) {
+        context = new FunctionContextImpl(functionObject.getId(), memberMappedArg.getArgumentsForMember(localVM.getId()), resultSender, isReexecute);
+      } else {
+        context = new FunctionContextImpl(functionObject.getId(), args, resultSender, isReexecute);
+      }
+      HandShake handShake = (HandShake) servConn.getHandshake();
+      int earlierClientReadTimeout = handShake.getClientReadTimeout();
+      handShake.setClientReadTimeout(functionTimeout);
+      try {
+        if (logger.isDebugEnabled()) {
+          logger.debug("Executing Function on Server: {} with context: {}", servConn, context);
         }
-        HandShake handShake = (HandShake)servConn.getHandshake();
-        int earlierClientReadTimeout = handShake.getClientReadTimeout();
-        handShake.setClientReadTimeout(functionTimeout);
-        try {
-          if (logger.isDebugEnabled()) {
-            logger.debug("Executing Function on Server: {} with context: {}", servConn, context);
-          }
-          GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
-          HeapMemoryMonitor hmm = ((InternalResourceManager) cache.getResourceManager()).getHeapMonitor();
-          if (functionObject.optimizeForWrite() && cache != null
-              && hmm.getState().isCritical()
-              && !MemoryThresholds.isLowMemoryExceptionDisabled()) {
-            Set<DistributedMember> sm = Collections
-                .singleton((DistributedMember)cache.getMyId());
-            Exception e = new LowMemoryException(
-                LocalizedStrings.ResourceManager_LOW_MEMORY_FOR_0_FUNCEXEC_MEMBERS_1
-                    .toLocalizedString(new Object[] { functionObject.getId(),
-                        sm }), sm);
-
-            sendException(hasResult, msg, e.getMessage(), servConn, e);
-            return;
-          }
-          /**
-           * if cache is null, then either cache has not yet been created on
-           * this node or it is a shutdown scenario.
-           */
-          DM dm = null;
-          if (cache != null) {
-            dm = cache.getDistributionManager();
-          }
-          if (groups != null && groups.length > 0) {
-            executeFunctionOnGroups(function, args, groups, allMembers,
-                functionObject, resultSender, ignoreFailedMembers);
-          } else {
-            executeFunctionaLocally(functionObject, context,
-                (ServerToClientFunctionResultSender65)resultSender, dm, stats);
-          }
+        GemFireCacheImpl cache = GemFireCacheImpl.getInstance();
+        HeapMemoryMonitor hmm = ((InternalResourceManager) cache.getResourceManager()).getHeapMonitor();
+        if (functionObject.optimizeForWrite() && cache != null && hmm.getState()
+                                                                     .isCritical() && !MemoryThresholds.isLowMemoryExceptionDisabled()) {
+          Set<DistributedMember> sm = Collections.singleton((DistributedMember) cache.getMyId());
+          Exception e = new LowMemoryException(LocalizedStrings.ResourceManager_LOW_MEMORY_FOR_0_FUNCEXEC_MEMBERS_1.toLocalizedString(new Object[] {
+            functionObject.getId(), sm
+          }), sm);
 
-          if (!functionObject.hasResult()) {
-            writeReply(msg, servConn);
-          }
+          sendException(hasResult, msg, e.getMessage(), servConn, e);
+          return;
         }
-        catch (FunctionException functionException) {
-          stats.endFunctionExecutionWithException(functionObject.hasResult());
-          throw functionException;
+        /**
+         * if cache is null, then either cache has not yet been created on
+         * this node or it is a shutdown scenario.
+         */
+        DM dm = null;
+        if (cache != null) {
+          dm = cache.getDistributionManager();
         }
-        catch (Exception exception) {
-          stats.endFunctionExecutionWithException(functionObject.hasResult());
-          throw new FunctionException(exception);
+        if (groups != null && groups.length > 0) {
+          executeFunctionOnGroups(function, args, groups, allMembers, functionObject, resultSender, ignoreFailedMembers);
+        } else {
+          executeFunctionaLocally(functionObject, context, (ServerToClientFunctionResultSender65) resultSender, dm, stats);
         }
-        finally {
-          handShake.setClientReadTimeout(earlierClientReadTimeout);
+
+        if (!functionObject.hasResult()) {
+          writeReply(msg, servConn);
         }
+      } catch (FunctionException functionException) {
+        stats.endFunctionExecutionWithException(functionObject.hasResult());
+        throw functionException;
+      } catch (Exception exception) {
+        stats.endFunctionExecutionWithException(functionObject.hasResult());
+        throw new FunctionException(exception);
+      } finally {
+        handShake.setClientReadTimeout(earlierClientReadTimeout);
       }
-      catch (IOException ioException) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioException);
-        String message = LocalizedStrings.ExecuteFunction_SERVER_COULD_NOT_SEND_THE_REPLY
-            .toLocalizedString();
-        sendException(hasResult, msg, message, servConn, ioException);
-      } 
-      catch (InternalFunctionInvocationTargetException internalfunctionException) {
-        // Fix for #44709: User should not be aware of
-        // InternalFunctionInvocationTargetException. No instance of
-        // InternalFunctionInvocationTargetException is giving useful
-        // information to user to take any corrective action hence logging
-        // this at fine level logging
-        // 1> When bucket is moved
-        // 2> Incase of HA FucntionInvocationTargetException thrown. Since
-        // it is HA, fucntion will be reexecuted on right node
-        // 3> Multiple target nodes found for single hop operation
-        // 4> in case of HA member departed
-        if (logger.isDebugEnabled()) {
-          logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] { function }), internalfunctionException);
-        }
-        final String message = internalfunctionException.getMessage();
-        sendException(hasResult, msg, message, servConn, internalfunctionException);
-      } 
-      catch (Exception e) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
-        final String message = e.getMessage();
-        sendException(hasResult, msg, message, servConn, e);
+    } catch (IOException ioException) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioException);
+      String message = LocalizedStrings.ExecuteFunction_SERVER_COULD_NOT_SEND_THE_REPLY.toLocalizedString();
+      sendException(hasResult, msg, message, servConn, ioException);
+    } catch (InternalFunctionInvocationTargetException internalfunctionException) {
+      // Fix for #44709: User should not be aware of
+      // InternalFunctionInvocationTargetException. No instance of
+      // InternalFunctionInvocationTargetException is giving useful
+      // information to user to take any corrective action hence logging
+      // this at fine level logging
+      // 1> When bucket is moved
+      // 2> Incase of HA FucntionInvocationTargetException thrown. Since
+      // it is HA, fucntion will be reexecuted on right node
+      // 3> Multiple target nodes found for single hop operation
+      // 4> in case of HA member departed
+      if (logger.isDebugEnabled()) {
+        logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] {
+          function
+        }), internalfunctionException);
       }
+      final String message = internalfunctionException.getMessage();
+      sendException(hasResult, msg, message, servConn, internalfunctionException);
+    } catch (Exception e) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
+      final String message = e.getMessage();
+      sendException(hasResult, msg, message, servConn, e);
     }
   }
 
@@ -313,9 +289,13 @@ public class ExecuteFunction66 extends BaseCommand {
     return false;
   }
 
-  protected void executeFunctionOnGroups(Object function, Object args,
-      String[] groups, boolean allMembers, Function functionObject,
-      ServerToClientFunctionResultSender resultSender, boolean ignoreFailedMembers) {
+  protected void executeFunctionOnGroups(Object function,
+                                         Object args,
+                                         String[] groups,
+                                         boolean allMembers,
+                                         Function functionObject,
+                                         ServerToClientFunctionResultSender resultSender,
+                                         boolean ignoreFailedMembers) {
     throw new InternalGemFireError();
   }
 
@@ -324,19 +304,18 @@ public class ExecuteFunction66 extends BaseCommand {
   }
 
   private void executeFunctionaLocally(final Function fn,
-      final FunctionContext cx,
-      final ServerToClientFunctionResultSender65 sender, DM dm,
-      final FunctionStats stats) throws IOException {
+                                       final FunctionContext cx,
+                                       final ServerToClientFunctionResultSender65 sender,
+                                       DM dm,
+                                       final FunctionStats stats) throws IOException {
     long startExecution = stats.startTime();
     stats.startFunctionExecution(fn.hasResult());
 
     if (fn.hasResult()) {
       fn.execute(cx);
-      if (!((ServerToClientFunctionResultSender65)sender)
-          .isLastResultReceived() && fn.hasResult()) {
-        throw new FunctionException(
-            LocalizedStrings.ExecuteFunction_THE_FUNCTION_0_DID_NOT_SENT_LAST_RESULT
-                .toString(fn.getId()));
+      if (!((ServerToClientFunctionResultSender65) sender).isLastResultReceived() && fn.hasResult()) {
+        throw new FunctionException(LocalizedStrings.ExecuteFunction_THE_FUNCTION_0_DID_NOT_SENT_LAST_RESULT.toString(fn
+          .getId()));
       }
     } else {
       /**
@@ -351,13 +330,9 @@ public class ExecuteFunction66 extends BaseCommand {
             if (txState != null) {
               cache = GemFireCacheImpl.getExisting("executing function");
               cache.getTxManager().masqueradeAs(txState);
-              if (cache.getLoggerI18n().warningEnabled()
-                  && !ASYNC_TX_WARNING_ISSUED) {
+              if (cache.getLoggerI18n().warningEnabled() && !ASYNC_TX_WARNING_ISSUED) {
                 ASYNC_TX_WARNING_ISSUED = true;
-                cache
-                    .getLoggerI18n()
-                    .warning(
-                        LocalizedStrings.ExecuteFunction66_TRANSACTIONAL_FUNCTION_WITHOUT_RESULT);
+                cache.getLoggerI18n().warning(LocalizedStrings.ExecuteFunction66_TRANSACTIONAL_FUNCTION_WITHOUT_RESULT);
               }
             }
             fn.execute(cx);
@@ -372,7 +347,9 @@ public class ExecuteFunction66 extends BaseCommand {
             // 2> in case of HA member departed
             stats.endFunctionExecutionWithException(fn.hasResult());
             if (logger.isDebugEnabled()) {
-              logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] { fn }), internalfunctionException);
+              logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] {
+                fn
+              }), internalfunctionException);
             }
           } catch (FunctionException functionException) {
             stats.endFunctionExecutionWithException(fn.hasResult());
@@ -395,34 +372,28 @@ public class ExecuteFunction66 extends BaseCommand {
          */
         execService.execute(functionExecution);
       } else {
-        final DistributionManager newDM = (DistributionManager)dm;
+        final DistributionManager newDM = (DistributionManager) dm;
         newDM.getFunctionExcecutor().execute(functionExecution);
       }
     }
     stats.endFunctionExecution(startExecution, fn.hasResult());
   }
 
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection servConn, Throwable e) throws IOException {
+  private void sendException(byte hasResult, Message msg, String message, ServerConnection servConn, Throwable e)
+    throws IOException {
     if (hasResult == 1) {
-      writeFunctionResponseException(msg, MessageType.EXCEPTION, message,
-          servConn, e);
-    }
-    else {
+      writeFunctionResponseException(msg, MessageType.EXCEPTION, message, servConn, e);
+    } else {
       writeException(msg, e, false, servConn);
     }
     servConn.setAsTrue(RESPONDED);
   }
 
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection servConn) throws IOException {
+  private void sendError(byte hasResult, Message msg, String message, ServerConnection servConn) throws IOException {
     if (hasResult == 1) {
-      writeFunctionResponseError(msg, MessageType.EXECUTE_FUNCTION_ERROR,
-          message, servConn);
-    }
-    else {
-      writeErrorResponse(msg, MessageType.EXECUTE_FUNCTION_ERROR, message,
-          servConn);
+      writeFunctionResponseError(msg, MessageType.EXECUTE_FUNCTION_ERROR, message, servConn);
+    } else {
+      writeErrorResponse(msg, MessageType.EXECUTE_FUNCTION_ERROR, message, servConn);
     }
     servConn.setAsTrue(RESPONDED);
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction.java
index 40f0ee5..6889e32 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction.java
@@ -18,7 +18,6 @@
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -47,6 +46,7 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 /**
  * This is the base command which reads the parts for the
@@ -54,7 +54,7 @@ import com.gemstone.gemfire.internal.security.AuthorizeRequest;
  * region.<br>
  * If the hasResult byte is 1, then this command send back the result after the
  * execution to the client else do not send the reply back to the client
- * 
+ *
  * @since GemFire 5.8LA
  */
 public class ExecuteRegionFunction extends BaseCommand {
@@ -69,8 +69,7 @@ public class ExecuteRegionFunction extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException {
     String regionName = null;
     Object function = null;
     Object args = null;
@@ -92,7 +91,7 @@ public class ExecuteRegionFunction extends BaseCommand {
       if (part != null) {
         Object obj = part.getObject();
         if (obj instanceof MemberMappedArgument) {
-          memberMappedArg = (MemberMappedArgument)obj;
+          memberMappedArg = (MemberMappedArgument) obj;
         }
       }
       filterSize = msg.getPart(5).getInt();
@@ -103,9 +102,8 @@ public class ExecuteRegionFunction extends BaseCommand {
           filter.add(msg.getPart(partNumber + i).getStringOrObject());
         }
       }
-      
-    }
-    catch (ClassNotFoundException exception) {
+
+    } catch (ClassNotFoundException exception) {
       logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), exception);
       if (hasResult == 1) {
         writeChunkedException(msg, exception, false, servConn);
@@ -125,143 +123,126 @@ public class ExecuteRegionFunction extends BaseCommand {
       sendError(hasResult, msg, message, servConn);
       return;
     }
-    else {
-      Region region = crHelper.getRegion(regionName);
-      if (region == null) {
-        String message = 
-          LocalizedStrings.ExecuteRegionFunction_THE_REGION_NAMED_0_WAS_NOT_FOUND_DURING_EXECUTE_FUNCTION_REQUEST
-          .toLocalizedString(regionName);
-        logger.warn("{}: {}", servConn.getName(), message);
-        sendError(hasResult, msg, message, servConn);
-        return;
-      }
-      HandShake handShake = (HandShake)servConn.getHandshake();
-      int earlierClientReadTimeout = handShake.getClientReadTimeout();
-      handShake.setClientReadTimeout(0);
-      ServerToClientFunctionResultSender resultSender = null;
-      Function functionObject = null;
-      try { 
-        if (function instanceof String) {
-          functionObject = FunctionService.getFunction((String)function);
-          if (functionObject == null) {
-            String message = LocalizedStrings.
-              ExecuteRegionFunction_THE_FUNCTION_0_HAS_NOT_BEEN_REGISTERED
-                .toLocalizedString(function);
-            logger.warn("{}: {}", servConn.getName(), message);
-            sendError(hasResult, msg, message, servConn);
-            return;
-          }
-        }
-        else {
-          functionObject = (Function)function;
-        }
-        // check if the caller is authorized to do this operation on server
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        final String functionName = functionObject.getId();
-        final String regionPath = region.getFullPath();
-        ExecuteFunctionOperationContext executeContext = null;
-        if (authzRequest != null) {
-          executeContext = authzRequest.executeFunctionAuthorize(functionName,
-              regionPath, filter, args, functionObject.optimizeForWrite());
-        }
-        
-        //Construct execution 
-        AbstractExecution execution = (AbstractExecution)FunctionService.onRegion(region);
-        ChunkedMessage m = servConn.getFunctionResponseMessage();
-        m.setTransactionId(msg.getTransactionId());        
-        resultSender = new ServerToClientFunctionResultSender(m,
-            MessageType.EXECUTE_REGION_FUNCTION_RESULT, servConn,functionObject,executeContext);
-        
-        
-        if (execution instanceof PartitionedRegionFunctionExecutor) {
-          execution = new PartitionedRegionFunctionExecutor(
-              (PartitionedRegion)region, filter, args, memberMappedArg,
-              resultSender, null, false);
-        }
-        else {
-          execution = new DistributedRegionFunctionExecutor(
-              (DistributedRegion)region, filter, args, memberMappedArg,
-              resultSender);          
-        }
-        
-        if (logger.isDebugEnabled()) {
-          logger.debug("Executing Function: {} on Server: {} with Execution: {}", functionObject.getId(), servConn, execution);
+
+    Region region = crHelper.getRegion(regionName);
+    if (region == null) {
+      String message = LocalizedStrings.ExecuteRegionFunction_THE_REGION_NAMED_0_WAS_NOT_FOUND_DURING_EXECUTE_FUNCTION_REQUEST
+        .toLocalizedString(regionName);
+      logger.warn("{}: {}", servConn.getName(), message);
+      sendError(hasResult, msg, message, servConn);
+      return;
+    }
+
+    HandShake handShake = (HandShake) servConn.getHandshake();
+    int earlierClientReadTimeout = handShake.getClientReadTimeout();
+    handShake.setClientReadTimeout(0);
+    ServerToClientFunctionResultSender resultSender = null;
+    Function functionObject = null;
+    try {
+      if (function instanceof String) {
+        functionObject = FunctionService.getFunction((String) function);
+        if (functionObject == null) {
+          String message = LocalizedStrings.
+            ExecuteRegionFunction_THE_FUNCTION_0_HAS_NOT_BEEN_REGISTERED.toLocalizedString(function);
+          logger.warn("{}: {}", servConn.getName(), message);
+          sendError(hasResult, msg, message, servConn);
+          return;
         }
-        if (hasResult == 1) {
-          if (function instanceof String) {
-            execution.execute((String)function).getResult();
-          }
-          else {
-            execution.execute(functionObject).getResult();
-          }
-        }else {
-          if (function instanceof String) {
-            execution.execute((String)function);
-          }
-          else {
-            execution.execute(functionObject);
-          }
-        }        
-      }
-      catch (IOException ioe) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioe);
-        final String message = LocalizedStrings.
-          ExecuteRegionFunction_SERVER_COULD_NOT_SEND_THE_REPLY
-            .toLocalizedString();
-        sendException(hasResult, msg, message, servConn,ioe);
+      } else {
+        functionObject = (Function) function;
       }
-      catch (InternalFunctionInvocationTargetException internalfunctionException) {
-        // Fix for #44709: User should not be aware of
-        // InternalFunctionInvocationTargetException. No instance of
-        // InternalFunctionInvocationTargetException is giving useful
-        // information to user to take any corrective action hence logging
-        // this at fine level logging
-        // 1> When bucket is moved
-        // 2> Incase of HA FucntionInvocationTargetException thrown. Since
-        // it is HA, fucntion will be reexecuted on right node
-        // 3> Multiple target nodes found for single hop operation
-        // 4> in case of HA member departed
-        if (logger.isDebugEnabled()) {
-          logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] { function }), internalfunctionException);
-        }
-        final String message = internalfunctionException.getMessage();
-        sendException(hasResult, msg, message, servConn, internalfunctionException);
-      } 
-      catch (FunctionException fe) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), fe);
-        String message = fe.getMessage();
 
-        sendException(hasResult, msg, message, servConn,fe);
+      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+
+      // check if the caller is authorized to do this operation on server
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      final String functionName = functionObject.getId();
+      final String regionPath = region.getFullPath();
+      ExecuteFunctionOperationContext executeContext = null;
+      if (authzRequest != null) {
+        executeContext = authzRequest.executeFunctionAuthorize(functionName, regionPath, filter, args, functionObject.optimizeForWrite());
       }
-      catch (Exception e) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
-        String message = e.getMessage();
-        sendException(hasResult, msg, message, servConn,e);
+
+      //Construct execution
+      AbstractExecution execution = (AbstractExecution) FunctionService.onRegion(region);
+      ChunkedMessage m = servConn.getFunctionResponseMessage();
+      m.setTransactionId(msg.getTransactionId());
+      resultSender = new ServerToClientFunctionResultSender(m, MessageType.EXECUTE_REGION_FUNCTION_RESULT, servConn, functionObject, executeContext);
+
+
+      if (execution instanceof PartitionedRegionFunctionExecutor) {
+        execution = new PartitionedRegionFunctionExecutor((PartitionedRegion) region, filter, args, memberMappedArg, resultSender, null, false);
+      } else {
+        execution = new DistributedRegionFunctionExecutor((DistributedRegion) region, filter, args, memberMappedArg, resultSender);
       }
 
-      finally{
-        handShake.setClientReadTimeout(earlierClientReadTimeout);
+      if (logger.isDebugEnabled()) {
+        logger.debug("Executing Function: {} on Server: {} with Execution: {}", functionObject.getId(), servConn, execution);
+      }
+      if (hasResult == 1) {
+        if (function instanceof String) {
+          execution.execute((String) function).getResult();
+        } else {
+          execution.execute(functionObject).getResult();
+        }
+      } else {
+        if (function instanceof String) {
+          execution.execute((String) function);
+        } else {
+          execution.execute(functionObject);
+        }
+      }
+    } catch (IOException ioe) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioe);
+      final String message = LocalizedStrings.
+        ExecuteRegionFunction_SERVER_COULD_NOT_SEND_THE_REPLY.toLocalizedString();
+      sendException(hasResult, msg, message, servConn, ioe);
+    } catch (InternalFunctionInvocationTargetException internalfunctionException) {
+      // Fix for #44709: User should not be aware of
+      // InternalFunctionInvocationTargetException. No instance of
+      // InternalFunctionInvocationTargetException is giving useful
+      // information to user to take any corrective action hence logging
+      // this at fine level logging
+      // 1> When bucket is moved
+      // 2> Incase of HA FucntionInvocationTargetException thrown. Since
+      // it is HA, fucntion will be reexecuted on right node
+      // 3> Multiple target nodes found for single hop operation
+      // 4> in case of HA member departed
+      if (logger.isDebugEnabled()) {
+        logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] {
+          function
+        }), internalfunctionException);
       }
+      final String message = internalfunctionException.getMessage();
+      sendException(hasResult, msg, message, servConn, internalfunctionException);
+    } catch (FunctionException fe) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), fe);
+      String message = fe.getMessage();
+
+      sendException(hasResult, msg, message, servConn, fe);
+    } catch (Exception e) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
+      String message = e.getMessage();
+      sendException(hasResult, msg, message, servConn, e);
+    } finally {
+      handShake.setClientReadTimeout(earlierClientReadTimeout);
     }
   }
 
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection servConn, Throwable e) throws IOException {
+  private void sendException(byte hasResult, Message msg, String message, ServerConnection servConn, Throwable e)
+    throws IOException {
     synchronized (msg) {
       if (hasResult == 1) {
-        writeFunctionResponseException(msg, MessageType.EXCEPTION, message,
-            servConn, e);
+        writeFunctionResponseException(msg, MessageType.EXCEPTION, message, servConn, e);
         servConn.setAsTrue(RESPONDED);
       }
     }
   }
-  
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection servConn) throws IOException {
+
+  private void sendError(byte hasResult, Message msg, String message, ServerConnection servConn) throws IOException {
     synchronized (msg) {
       if (hasResult == 1) {
-        writeFunctionResponseError(msg,
-            MessageType.EXECUTE_REGION_FUNCTION_ERROR, message, servConn);
+        writeFunctionResponseError(msg, MessageType.EXECUTE_REGION_FUNCTION_ERROR, message, servConn);
         servConn.setAsTrue(RESPONDED);
       }
     }


[07/50] [abbrv] incubator-geode git commit: GEODE-1575 - Unhandled NoClassDefFound Exception when creating regions

Posted by ji...@apache.org.
GEODE-1575 - Unhandled NoClassDefFound Exception when creating regions

Added exception clause for NoClassDefFoundError

This closes #170


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

Branch: refs/heads/feature/GEODE-1571
Commit: 087da4e3d084f64516181e7f06317cae64c8af40
Parents: 170919b
Author: Kevin J. Duling <kd...@pivotal.io>
Authored: Tue Jun 21 10:17:53 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Tue Jun 21 10:27:47 2016 -0700

----------------------------------------------------------------------
 .../management/internal/cli/functions/RegionCreateFunction.java | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/087da4e3/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/RegionCreateFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/RegionCreateFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/RegionCreateFunction.java
index 2344074..7e060c2 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/RegionCreateFunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/RegionCreateFunction.java
@@ -385,9 +385,10 @@ public class RegionCreateFunction extends FunctionAdapter implements InternalEnt
       if (classToLoadName != null && !classToLoadName.isEmpty()) {
         loadedClass = (Class<K>) classPathLoader.forName(classToLoadName);
       }
-    } catch (ClassNotFoundException e) {
+    } catch (ClassNotFoundException | NoClassDefFoundError e) {
       throw new RuntimeException(CliStrings.format(CliStrings.CREATE_REGION__MSG__COULDNOT_FIND_CLASS_0_SPECIFIED_FOR_1, new Object[] {classToLoadName, neededFor}), e);
-    } catch (ClassCastException e) {
+    }
+    catch (ClassCastException e) {
       throw new RuntimeException(CliStrings.format(CliStrings.CREATE_REGION__MSG__CLASS_SPECIFIED_FOR_0_SPECIFIED_FOR_1_IS_NOT_OF_EXPECTED_TYPE, new Object[] {classToLoadName, neededFor}), e);
     }
 


[04/50] [abbrv] incubator-geode git commit: GEODE-1549: Correct "Help" hyperlink in pulse

Posted by ji...@apache.org.
GEODE-1549: Correct "Help" hyperlink in pulse

Corrected the links to point to geode docs instead of gemfire
This closes #161


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

Branch: refs/heads/feature/GEODE-1571
Commit: 9ccb4d5bb5897c9c01d7b6eb59f4c137865cd263
Parents: 91f92a4
Author: Srikanth Manvi <sr...@gmail.com>
Authored: Wed Jun 15 23:13:58 2016 -0400
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Tue Jun 21 09:50:09 2016 -0700

----------------------------------------------------------------------
 geode-pulse/src/main/resources/gemfire.properties            | 2 +-
 geode-pulse/src/main/webapp/DataBrowser.html                 | 2 +-
 geode-pulse/src/main/webapp/MemberDetails.html               | 2 +-
 geode-pulse/src/main/webapp/QueryStatistics.html             | 2 +-
 geode-pulse/src/main/webapp/clusterDetail.html               | 2 +-
 geode-pulse/src/main/webapp/properties/gemfire.properties    | 2 +-
 geode-pulse/src/main/webapp/properties/gemfire_en.properties | 2 +-
 geode-pulse/src/main/webapp/regionDetail.html                | 2 +-
 8 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ccb4d5b/geode-pulse/src/main/resources/gemfire.properties
----------------------------------------------------------------------
diff --git a/geode-pulse/src/main/resources/gemfire.properties b/geode-pulse/src/main/resources/gemfire.properties
index 84ae196..bfb1286 100644
--- a/geode-pulse/src/main/resources/gemfire.properties
+++ b/geode-pulse/src/main/resources/gemfire.properties
@@ -25,7 +25,7 @@ pulse-writes-custom=Writes
 pulse-reads-custom=Reads
 pulse-monitoring-custom=images/pulse-monitoring.png
 pulse-aboutimg-custom=images/about-geode.png
-pulse-help-custom=http://gemfire.docs.pivotal.io/latest/userguide/index.html#tools_modules/pulse/chapter_overview.html
+pulse-help-custom=http://geode.docs.pivotal.io/docs/tools_modules/pulse/chapter_overview.html
 pulse-about-custom=The Pulse tool monitors Apache Geode&#0169; system in real time. It provides health information, detailed operational and configuration data, system alerts, throughput performance and statistics for system members and connected clients.
 pulse-regionstableCaps-custom=Regions
 pulse-rtSummaryBySize-custom=Regions Summary - By Entry Count

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ccb4d5b/geode-pulse/src/main/webapp/DataBrowser.html
----------------------------------------------------------------------
diff --git a/geode-pulse/src/main/webapp/DataBrowser.html b/geode-pulse/src/main/webapp/DataBrowser.html
index a046cf4..165fa6f 100644
--- a/geode-pulse/src/main/webapp/DataBrowser.html
+++ b/geode-pulse/src/main/webapp/DataBrowser.html
@@ -137,7 +137,7 @@
 	        </div>
 				</div>
 				<div class="left headerTopSeperator"></div>
-	      <div class="left"><a target="_blank" href="http://gemfire.docs.pivotal.io/latest/userguide/index.html#tools_modules/pulse/chapter_overview.html" class="left headerTopLinks">Help</a></div>
+	      <div class="left"><a target="_blank" href="http://geode.docs.pivotal.io/docs/tools_modules/pulse/chapter_overview.html" class="left headerTopLinks">Help</a></div>
 	      <div class="left headerTopSeperator"></div>
 	      <div class="left headerTopLinks welcomeLabelPRZero">Welcome</div>
 	      <div class="left headerTopLinks textbold font-size12" id="userName"></div>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ccb4d5b/geode-pulse/src/main/webapp/MemberDetails.html
----------------------------------------------------------------------
diff --git a/geode-pulse/src/main/webapp/MemberDetails.html b/geode-pulse/src/main/webapp/MemberDetails.html
index f358c1c..b416926 100644
--- a/geode-pulse/src/main/webapp/MemberDetails.html
+++ b/geode-pulse/src/main/webapp/MemberDetails.html
@@ -128,7 +128,7 @@
       
       </div>
       <div class="left headerTopSeperator"></div>
-      <div class="left"><a data-prod-custom="pulse-help-custom" target="_blank" href="http://gemfire.docs.pivotal.io/latest/userguide/index.html#tools_modules/pulse/chapter_overview.html" class="left headerTopLinks" class="left headerTopLinks">Help</a></div>
+      <div class="left"><a data-prod-custom="pulse-help-custom" target="_blank" href="http://geode.docs.pivotal.io/docs/tools_modules/pulse/chapter_overview.html" class="left headerTopLinks" class="left headerTopLinks">Help</a></div>
       <div class="left headerTopSeperator"></div>
       <div class="left headerTopLinks welcomeLabelPRZero">Welcome</div>
       <div class="left headerTopLinks textbold font-size12" id="userName"></div>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ccb4d5b/geode-pulse/src/main/webapp/QueryStatistics.html
----------------------------------------------------------------------
diff --git a/geode-pulse/src/main/webapp/QueryStatistics.html b/geode-pulse/src/main/webapp/QueryStatistics.html
index c5fb1a0..07f61c6 100644
--- a/geode-pulse/src/main/webapp/QueryStatistics.html
+++ b/geode-pulse/src/main/webapp/QueryStatistics.html
@@ -150,7 +150,7 @@
 				<div class="left headerTopSeperator"></div>
 				<div class="left">
 					<a data-prod-custom="pulse-help-custom" target="_blank"
-						href="http://gemfire.docs.pivotal.io/latest/userguide/index.html#tools_modules/pulse/chapter_overview.html"
+						href="http://geode.docs.pivotal.io/docs/tools_modules/pulse/chapter_overview.html"
 						class="left headerTopLinks">Help</a>
 				</div>
 				<div class="left headerTopSeperator"></div>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ccb4d5b/geode-pulse/src/main/webapp/clusterDetail.html
----------------------------------------------------------------------
diff --git a/geode-pulse/src/main/webapp/clusterDetail.html b/geode-pulse/src/main/webapp/clusterDetail.html
index 718a5b9..211e366 100644
--- a/geode-pulse/src/main/webapp/clusterDetail.html
+++ b/geode-pulse/src/main/webapp/clusterDetail.html
@@ -118,7 +118,7 @@
         </div>
       </div>
       <div class="left headerTopSeperator"></div>
-      <div class="left"><a data-prod-custom="pulse-help-custom" target="_blank" href="http://gemfire.docs.pivotal.io/latest/userguide/index.html#tools_modules/pulse/chapter_overview.html" class="left headerTopLinks" class="left headerTopLinks">Help</a></div>
+      <div class="left"><a data-prod-custom="pulse-help-custom" target="_blank" href="http://geode.docs.pivotal.io/docs/tools_modules/pulse/chapter_overview.html" class="left headerTopLinks" class="left headerTopLinks">Help</a></div>
       <div class="left headerTopSeperator"></div>
       <div class="left headerTopLinks welcomeLabelPRZero">Welcome</div>
       <div class="left headerTopLinks textbold font-size12" id="userName"></div>

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ccb4d5b/geode-pulse/src/main/webapp/properties/gemfire.properties
----------------------------------------------------------------------
diff --git a/geode-pulse/src/main/webapp/properties/gemfire.properties b/geode-pulse/src/main/webapp/properties/gemfire.properties
index 9e45d25..17e2259 100644
--- a/geode-pulse/src/main/webapp/properties/gemfire.properties
+++ b/geode-pulse/src/main/webapp/properties/gemfire.properties
@@ -23,7 +23,7 @@ pulse-writes-custom=Writes
 pulse-reads-custom=Reads
 pulse-monitoring-custom=images/pulse-monitoring.png
 pulse-aboutimg-custom=images/about-geode.png
-pulse-help-custom=http://gemfire.docs.pivotal.io/latest/userguide/index.html#tools_modules/pulse/chapter_overview.html
+pulse-help-custom=http://geode.docs.pivotal.io/docs/tools_modules/pulse/chapter_overview.html
 pulse-about-custom=The Pulse tool monitors Apache Geode system in real time. It provides health information, detailed operational and configuration data, system alerts, throughput performance and statistics for system members and connected clients.
 pulse-regionstableCaps-custom=Regions
 pulse-rtSummaryBySize-custom=Regions Summary - By Entry Count

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ccb4d5b/geode-pulse/src/main/webapp/properties/gemfire_en.properties
----------------------------------------------------------------------
diff --git a/geode-pulse/src/main/webapp/properties/gemfire_en.properties b/geode-pulse/src/main/webapp/properties/gemfire_en.properties
index 9e45d25..17e2259 100644
--- a/geode-pulse/src/main/webapp/properties/gemfire_en.properties
+++ b/geode-pulse/src/main/webapp/properties/gemfire_en.properties
@@ -23,7 +23,7 @@ pulse-writes-custom=Writes
 pulse-reads-custom=Reads
 pulse-monitoring-custom=images/pulse-monitoring.png
 pulse-aboutimg-custom=images/about-geode.png
-pulse-help-custom=http://gemfire.docs.pivotal.io/latest/userguide/index.html#tools_modules/pulse/chapter_overview.html
+pulse-help-custom=http://geode.docs.pivotal.io/docs/tools_modules/pulse/chapter_overview.html
 pulse-about-custom=The Pulse tool monitors Apache Geode system in real time. It provides health information, detailed operational and configuration data, system alerts, throughput performance and statistics for system members and connected clients.
 pulse-regionstableCaps-custom=Regions
 pulse-rtSummaryBySize-custom=Regions Summary - By Entry Count

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/9ccb4d5b/geode-pulse/src/main/webapp/regionDetail.html
----------------------------------------------------------------------
diff --git a/geode-pulse/src/main/webapp/regionDetail.html b/geode-pulse/src/main/webapp/regionDetail.html
index 8742769..98404cb 100644
--- a/geode-pulse/src/main/webapp/regionDetail.html
+++ b/geode-pulse/src/main/webapp/regionDetail.html
@@ -210,7 +210,7 @@
         <!-- Version Details Popup -->
       </div>
       <div class="left headerTopSeperator"></div>
-      <div class="left"><a data-prod-custom="pulse-help-custom"  target="_blank" href="http://gemfire.docs.pivotal.io/latest/userguide/index.html#tools_modules/pulse/chapter_overview.html" class="left headerTopLinks">Help</a></div>
+      <div class="left"><a data-prod-custom="pulse-help-custom"  target="_blank" href="http://geode.docs.pivotal.io/docs/tools_modules/pulse/chapter_overview.html" class="left headerTopLinks">Help</a></div>
       <div class="left headerTopSeperator"></div>
       <div class="left headerTopLinks welcomeLabelPRZero">Welcome</div>
       <div class="left headerTopLinks textbold font-size12" id="userName"></div>


[16/50] [abbrv] incubator-geode git commit: Merge branch 'develop' into feature/GEODE-1571

Posted by ji...@apache.org.
Merge branch 'develop' into feature/GEODE-1571


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

Branch: refs/heads/feature/GEODE-1571
Commit: 2b32829e3901004702b75b221cb1d48b0fd2d84b
Parents: 7ae6110 fc42db8
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Thu Jun 23 11:35:38 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Thu Jun 23 11:35:38 2016 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[36/50] [abbrv] incubator-geode git commit: GEODE-1576: Website needs to be upgraded to include WAN and CQ

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/30928d92/geode-site/website/content/font/fontawesome-webfont-svg.svg
----------------------------------------------------------------------
diff --git a/geode-site/website/content/font/fontawesome-webfont-svg.svg b/geode-site/website/content/font/fontawesome-webfont-svg.svg
old mode 100755
new mode 100644
index 2edb4ec..8b66187
--- a/geode-site/website/content/font/fontawesome-webfont-svg.svg
+++ b/geode-site/website/content/font/fontawesome-webfont-svg.svg
@@ -14,10 +14,11 @@
 <glyph unicode="&#xae;" horiz-adv-x="1792" />
 <glyph unicode="&#xb4;" horiz-adv-x="1792" />
 <glyph unicode="&#xc6;" horiz-adv-x="1792" />
+<glyph unicode="&#xd8;" horiz-adv-x="1792" />
 <glyph unicode="&#x2000;" horiz-adv-x="768" />
-<glyph unicode="&#x2001;" />
+<glyph unicode="&#x2001;" horiz-adv-x="1537" />
 <glyph unicode="&#x2002;" horiz-adv-x="768" />
-<glyph unicode="&#x2003;" />
+<glyph unicode="&#x2003;" horiz-adv-x="1537" />
 <glyph unicode="&#x2004;" horiz-adv-x="512" />
 <glyph unicode="&#x2005;" horiz-adv-x="384" />
 <glyph unicode="&#x2006;" horiz-adv-x="256" />
@@ -30,7 +31,7 @@
 <glyph unicode="&#x2122;" horiz-adv-x="1792" />
 <glyph unicode="&#x221e;" horiz-adv-x="1792" />
 <glyph unicode="&#x2260;" horiz-adv-x="1792" />
-<glyph unicode="&#xe000;" horiz-adv-x="500" d="M0 0z" />
+<glyph unicode="&#x25fc;" horiz-adv-x="500" d="M0 0z" />
 <glyph unicode="&#xf000;" horiz-adv-x="1792" d="M1699 1350q0 -35 -43 -78l-632 -632v-768h320q26 0 45 -19t19 -45t-19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45t45 19h320v768l-632 632q-43 43 -43 78q0 23 18 36.5t38 17.5t43 4h1408q23 0 43 -4t38 -17.5t18 -36.5z" />
 <glyph unicode="&#xf001;" d="M1536 1312v-1120q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v537l-768 -237v-709q0 -50 -34 -89t-86 -60.5t-103.5 -32t-96.5 -10.5t-96.5 10.5t-103.5 32t-86 60.5t-34 89 t34 89t86 60.5t103.5 32t96.5 10.5q105 0 192 -39v967q0 31 19 56.5t49 35.5l832 256q12 4 28 4q40 0 68 -28t28 -68z" />
 <glyph unicode="&#xf002;" horiz-adv-x="1664" d="M1152 704q0 185 -131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5t316.5 131.5t131.5 316.5zM1664 -128q0 -52 -38 -90t-90 -38q-54 0 -90 38l-343 342q-179 -124 -399 -124q-143 0 -273.5 55.5t-225 150t-150 225t-55.5 273.5 t55.5 273.5t150 225t225 150t273.5 55.5t273.5 -55.5t225 -150t150 -225t55.5 -273.5q0 -220 -124 -399l343 -343q37 -37 37 -90z" />
@@ -52,7 +53,7 @@
 <glyph unicode="&#xf013;" d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1536 749v-222q0 -12 -8 -23t-20 -13l-185 -28q-19 -54 -39 -91q35 -50 107 -138q10 -12 10 -25t-9 -23q-27 -37 -99 -108t-94 -71q-12 0 -26 9l-138 108q-44 -23 -91 -38 q-16 -136 -29 -186q-7 -28 -36 -28h-222q-14 0 -24.5 8.5t-11.5 21.5l-28 184q-49 16 -90 37l-141 -107q-10 -9 -25 -9q-14 0 -25 11q-126 114 -165 168q-7 10 -7 23q0 12 8 23q15 21 51 66.5t54 70.5q-27 50 -41 99l-183 27q-13 2 -21 12.5t-8 23.5v222q0 12 8 23t19 13 l186 28q14 46 39 92q-40 57 -107 138q-10 12 -10 24q0 10 9 23q26 36 98.5 107.5t94.5 71.5q13 0 26 -10l138 -107q44 23 91 38q16 136 29 186q7 28 36 28h222q14 0 24.5 -8.5t11.5 -21.5l28 -184q49 -16 90 -37l142 107q9 9 24 9q13 0 25 -10q129 -119 165 -170q7 -8 7 -22 q0 -12 -8 -23q-15 -21 -51 -66.5t-54 -70.5q26 -50 41 -98l183 -28q13 -2 21 -12.5t8 -23.5z" />
 <glyph unicode="&#xf014;" horiz-adv-x="1408" d="M512 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM768 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1024 800v-576q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v576 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1152 76v948h-896v-948q0 -22 7 -40.5t14.5 -27t10.5 -8.5h832q3 0 10.5 8.5t14.5 27t7 40.5zM480 1152h448l-48 117q-7 9 -17 11h-317q-10 -2 -17 -11zM1408 1120v-64q0 -14 -9 -23t-23 -9h-96v-948q0 -83 -47 -143.5t-113 -60.5h-832 q-66 0 -113 58.5t-47 141.5v952h-96q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h309l70 167q15 37 54 63t79 26h320q40 0 79 -26t54 -63l70 -167h309q14 0 23 -9t9 -23z" />
 <glyph unicode="&#xf015;" horiz-adv-x="1664" d="M1408 544v-480q0 -26 -19 -45t-45 -19h-384v384h-256v-384h-384q-26 0 -45 19t-19 45v480q0 1 0.5 3t0.5 3l575 474l575 -474q1 -2 1 -6zM1631 613l-62 -74q-8 -9 -21 -11h-3q-13 0 -21 7l-692 577l-692 -577q-12 -8 -24 -7q-13 2 -21 11l-62 74q-8 10 -7 23.5t11 21.5 l719 599q32 26 76 26t76 -26l244 -204v195q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-408l219 -182q10 -8 11 -21.5t-7 -23.5z" />
-<glyph unicode="&#xf016;" horiz-adv-x="1280" d="M128 0h1024v768h-416q-40 0 -68 28t-28 68v416h-512v-1280zM768 896h376q-10 29 -22 41l-313 313q-12 12 -41 22v-376zM1280 864v-896q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h640q40 0 88 -20t76 -48l312 -312q28 -28 48 -76t20 -88z " />
+<glyph unicode="&#xf016;" d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z " />
 <glyph unicode="&#xf017;" d="M896 992v-448q0 -14 -9 -23t-23 -9h-320q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v352q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1312 640q0 148 -73 273t-198 198t-273 73t-273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
 <glyph unicode="&#xf018;" horiz-adv-x="1920" d="M1111 540v4l-24 320q-1 13 -11 22.5t-23 9.5h-186q-13 0 -23 -9.5t-11 -22.5l-24 -320v-4q-1 -12 8 -20t21 -8h244q12 0 21 8t8 20zM1870 73q0 -73 -46 -73h-704q13 0 22 9.5t8 22.5l-20 256q-1 13 -11 22.5t-23 9.5h-272q-13 0 -23 -9.5t-11 -22.5l-20 -256 q-1 -13 8 -22.5t22 -9.5h-704q-46 0 -46 73q0 54 26 116l417 1044q8 19 26 33t38 14h339q-13 0 -23 -9.5t-11 -22.5l-15 -192q-1 -14 8 -23t22 -9h166q13 0 22 9t8 23l-15 192q-1 13 -11 22.5t-23 9.5h339q20 0 38 -14t26 -33l417 -1044q26 -62 26 -116z" />
 <glyph unicode="&#xf019;" horiz-adv-x="1664" d="M1280 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 416v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h465l135 -136 q58 -56 136 -56t136 56l136 136h464q40 0 68 -28t28 -68zM1339 985q17 -41 -14 -70l-448 -448q-18 -19 -45 -19t-45 19l-448 448q-31 29 -14 70q17 39 59 39h256v448q0 26 19 45t45 19h256q26 0 45 -19t19 -45v-448h256q42 0 59 -39z" />
@@ -77,11 +78,11 @@
 <glyph unicode="&#xf02e;" horiz-adv-x="1280" d="M1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
 <glyph unicode="&#xf02f;" horiz-adv-x="1664" d="M384 0h896v256h-896v-256zM384 640h896v384h-160q-40 0 -68 28t-28 68v160h-640v-640zM1536 576q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 576v-416q0 -13 -9.5 -22.5t-22.5 -9.5h-224v-160q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68 v160h-224q-13 0 -22.5 9.5t-9.5 22.5v416q0 79 56.5 135.5t135.5 56.5h64v544q0 40 28 68t68 28h672q40 0 88 -20t76 -48l152 -152q28 -28 48 -76t20 -88v-256h64q79 0 135.5 -56.5t56.5 -135.5z" />
 <glyph unicode="&#xf030;" horiz-adv-x="1920" d="M960 864q119 0 203.5 -84.5t84.5 -203.5t-84.5 -203.5t-203.5 -84.5t-203.5 84.5t-84.5 203.5t84.5 203.5t203.5 84.5zM1664 1280q106 0 181 -75t75 -181v-896q0 -106 -75 -181t-181 -75h-1408q-106 0 -181 75t-75 181v896q0 106 75 181t181 75h224l51 136 q19 49 69.5 84.5t103.5 35.5h512q53 0 103.5 -35.5t69.5 -84.5l51 -136h224zM960 128q185 0 316.5 131.5t131.5 316.5t-131.5 316.5t-316.5 131.5t-316.5 -131.5t-131.5 -316.5t131.5 -316.5t316.5 -131.5z" />
-<glyph unicode="&#xf031;" horiz-adv-x="1664" d="M725 977l-170 -450q73 -1 153.5 -2t119 -1.5t52.5 -0.5l29 2q-32 95 -92 241q-53 132 -92 211zM21 -128h-21l2 79q22 7 80 18q89 16 110 31q20 16 48 68l237 616l280 724h75h53l11 -21l205 -480q103 -242 124 -297q39 -102 96 -235q26 -58 65 -164q24 -67 65 -149 q22 -49 35 -57q22 -19 69 -23q47 -6 103 -27q6 -39 6 -57q0 -14 -1 -26q-80 0 -192 8q-93 8 -189 8q-79 0 -135 -2l-200 -11l-58 -2q0 45 4 78l131 28q56 13 68 23q12 12 12 27t-6 32l-47 114l-92 228l-450 2q-29 -65 -104 -274q-23 -64 -23 -84q0 -31 17 -43 q26 -21 103 -32q3 0 13.5 -2t30 -5t40.5 -6q1 -28 1 -58q0 -17 -2 -27q-66 0 -349 20l-48 -8q-81 -14 -167 -14z" />
-<glyph unicode="&#xf032;" horiz-adv-x="1408" d="M555 15q76 -32 140 -32q131 0 216 41t122 113q38 70 38 181q0 114 -41 180q-58 94 -141 126q-80 32 -247 32q-74 0 -101 -10v-144l-1 -173l3 -270q0 -15 12 -44zM541 761q43 -7 109 -7q175 0 264 65t89 224q0 112 -85 187q-84 75 -255 75q-52 0 -130 -13q0 -44 2 -77 q7 -122 6 -279l-1 -98q0 -43 1 -77zM0 -128l2 94q45 9 68 12q77 12 123 31q17 27 21 51q9 66 9 194l-2 497q-5 256 -9 404q-1 87 -11 109q-1 4 -12 12q-18 12 -69 15q-30 2 -114 13l-4 83l260 6l380 13l45 1q5 0 14 0.5t14 0.5q1 0 21.5 -0.5t40.5 -0.5h74q88 0 191 -27 q43 -13 96 -39q57 -29 102 -76q44 -47 65 -104t21 -122q0 -70 -32 -128t-95 -105q-26 -20 -150 -77q177 -41 267 -146q92 -106 92 -236q0 -76 -29 -161q-21 -62 -71 -117q-66 -72 -140 -108q-73 -36 -203 -60q-82 -15 -198 -11l-197 4q-84 2 -298 -11q-33 -3 -272 -11z" />
-<glyph unicode="&#xf033;" horiz-adv-x="1024" d="M0 -126l17 85q4 1 77 20q76 19 116 39q29 37 41 101l27 139l56 268l12 64q8 44 17 84.5t16 67t12.5 46.5t9 30.5t3.5 11.5l29 157l16 63l22 135l8 50v38q-41 22 -144 28q-28 2 -38 4l19 103l317 -14q39 -2 73 -2q66 0 214 9q33 2 68 4.5t36 2.5q-2 -19 -6 -38 q-7 -29 -13 -51q-55 -19 -109 -31q-64 -16 -101 -31q-12 -31 -24 -88q-9 -44 -13 -82q-44 -199 -66 -306l-61 -311l-38 -158l-43 -235l-12 -45q-2 -7 1 -27q64 -15 119 -21q36 -5 66 -10q-1 -29 -7 -58q-7 -31 -9 -41q-18 0 -23 -1q-24 -2 -42 -2q-9 0 -28 3q-19 4 -145 17 l-198 2q-41 1 -174 -11q-74 -7 -98 -9z" />
-<glyph unicode="&#xf034;" horiz-adv-x="1792" d="M81 1407l54 -27q20 -5 211 -5h130l19 3l115 1l215 -1h293l34 -2q14 -1 28 7t21 16l7 8l42 1q15 0 28 -1v-104.5t1 -131.5l1 -100l-1 -58q0 -32 -4 -51q-39 -15 -68 -18q-25 43 -54 128q-8 24 -15.5 62.5t-11.5 65.5t-6 29q-13 15 -27 19q-7 2 -42.5 2t-103.5 -1t-111 -1 q-34 0 -67 -5q-10 -97 -8 -136l1 -152v-332l3 -359l-1 -147q-1 -46 11 -85q49 -25 89 -32q2 0 18 -5t44 -13t43 -12q30 -8 50 -18q5 -45 5 -50q0 -10 -3 -29q-14 -1 -34 -1q-110 0 -187 10q-72 8 -238 8q-88 0 -233 -14q-48 -4 -70 -4q-2 22 -2 26l-1 26v9q21 33 79 49 q139 38 159 50q9 21 12 56q8 192 6 433l-5 428q-1 62 -0.5 118.5t0.5 102.5t-2 57t-6 15q-6 5 -14 6q-38 6 -148 6q-43 0 -100 -13.5t-73 -24.5q-13 -9 -22 -33t-22 -75t-24 -84q-6 -19 -19.5 -32t-20.5 -13q-44 27 -56 44v297v86zM1744 128q33 0 42 -18.5t-11 -44.5 l-126 -162q-20 -26 -49 -26t-49 26l-126 162q-20 26 -11 44.5t42 18.5h80v1024h-80q-33 0 -42 18.5t11 44.5l126 162q20 26 49 26t49 -26l126 -162q20 -26 11 -44.5t-42 -18.5h-80v-1024h80z" />
-<glyph unicode="&#xf035;" d="M81 1407l54 -27q20 -5 211 -5h130l19 3l115 1l446 -1h318l34 -2q14 -1 28 7t21 16l7 8l42 1q15 0 28 -1v-104.5t1 -131.5l1 -100l-1 -58q0 -32 -4 -51q-39 -15 -68 -18q-25 43 -54 128q-8 24 -15.5 62.5t-11.5 65.5t-6 29q-13 15 -27 19q-7 2 -58.5 2t-138.5 -1t-128 -1 q-94 0 -127 -5q-10 -97 -8 -136l1 -152v52l3 -359l-1 -147q-1 -46 11 -85q49 -25 89 -32q2 0 18 -5t44 -13t43 -12q30 -8 50 -18q5 -45 5 -50q0 -10 -3 -29q-14 -1 -34 -1q-110 0 -187 10q-72 8 -238 8q-82 0 -233 -13q-45 -5 -70 -5q-2 22 -2 26l-1 26v9q21 33 79 49 q139 38 159 50q9 21 12 56q6 137 6 433l-5 44q0 265 -2 278q-2 11 -6 15q-6 5 -14 6q-38 6 -148 6q-50 0 -168.5 -14t-132.5 -24q-13 -9 -22 -33t-22 -75t-24 -84q-6 -19 -19.5 -32t-20.5 -13q-44 27 -56 44v297v86zM1505 113q26 -20 26 -49t-26 -49l-162 -126 q-26 -20 -44.5 -11t-18.5 42v80h-1024v-80q0 -33 -18.5 -42t-44.5 11l-162 126q-26 20 -26 49t26 49l162 126q26 20 44.5 11t18.5 -42v-80h1024v80q0 33 18.5 42t44.5 -11z" />
+<glyph unicode="&#xf031;" horiz-adv-x="1664" d="M725 977l-170 -450q33 0 136.5 -2t160.5 -2q19 0 57 2q-87 253 -184 452zM0 -128l2 79q23 7 56 12.5t57 10.5t49.5 14.5t44.5 29t31 50.5l237 616l280 724h75h53q8 -14 11 -21l205 -480q33 -78 106 -257.5t114 -274.5q15 -34 58 -144.5t72 -168.5q20 -45 35 -57 q19 -15 88 -29.5t84 -20.5q6 -38 6 -57q0 -4 -0.5 -13t-0.5 -13q-63 0 -190 8t-191 8q-76 0 -215 -7t-178 -8q0 43 4 78l131 28q1 0 12.5 2.5t15.5 3.5t14.5 4.5t15 6.5t11 8t9 11t2.5 14q0 16 -31 96.5t-72 177.5t-42 100l-450 2q-26 -58 -76.5 -195.5t-50.5 -162.5 q0 -22 14 -37.5t43.5 -24.5t48.5 -13.5t57 -8.5t41 -4q1 -19 1 -58q0 -9 -2 -27q-58 0 -174.5 10t-174.5 10q-8 0 -26.5 -4t-21.5 -4q-80 -14 -188 -14z" />
+<glyph unicode="&#xf032;" horiz-adv-x="1408" d="M555 15q74 -32 140 -32q376 0 376 335q0 114 -41 180q-27 44 -61.5 74t-67.5 46.5t-80.5 25t-84 10.5t-94.5 2q-73 0 -101 -10q0 -53 -0.5 -159t-0.5 -158q0 -8 -1 -67.5t-0.5 -96.5t4.5 -83.5t12 -66.5zM541 761q42 -7 109 -7q82 0 143 13t110 44.5t74.5 89.5t25.5 142 q0 70 -29 122.5t-79 82t-108 43.5t-124 14q-50 0 -130 -13q0 -50 4 -151t4 -152q0 -27 -0.5 -80t-0.5 -79q0 -46 1 -69zM0 -128l2 94q15 4 85 16t106 27q7 12 12.5 27t8.5 33.5t5.5 32.5t3 37.5t0.5 34v35.5v30q0 982 -22 1025q-4 8 -22 14.5t-44.5 11t-49.5 7t-48.5 4.5 t-30.5 3l-4 83q98 2 340 11.5t373 9.5q23 0 68.5 -0.5t67.5 -0.5q70 0 136.5 -13t128.5 -42t108 -71t74 -104.5t28 -137.5q0 -52 -16.5 -95.5t-39 -72t-64.5 -57.5t-73 -45t-84 -40q154 -35 256.5 -134t102.5 -248q0 -100 -35 -179.5t-93.5 -130.5t-138 -85.5t-163.5 -48.5 t-176 -14q-44 0 -132 3t-132 3q-106 0 -307 -11t-231 -12z" />
+<glyph unicode="&#xf033;" horiz-adv-x="1024" d="M0 -126l17 85q6 2 81.5 21.5t111.5 37.5q28 35 41 101q1 7 62 289t114 543.5t52 296.5v25q-24 13 -54.5 18.5t-69.5 8t-58 5.5l19 103q33 -2 120 -6.5t149.5 -7t120.5 -2.5q48 0 98.5 2.5t121 7t98.5 6.5q-5 -39 -19 -89q-30 -10 -101.5 -28.5t-108.5 -33.5 q-8 -19 -14 -42.5t-9 -40t-7.5 -45.5t-6.5 -42q-27 -148 -87.5 -419.5t-77.5 -355.5q-2 -9 -13 -58t-20 -90t-16 -83.5t-6 -57.5l1 -18q17 -4 185 -31q-3 -44 -16 -99q-11 0 -32.5 -1.5t-32.5 -1.5q-29 0 -87 10t-86 10q-138 2 -206 2q-51 0 -143 -9t-121 -11z" />
+<glyph unicode="&#xf034;" horiz-adv-x="1792" d="M1744 128q33 0 42 -18.5t-11 -44.5l-126 -162q-20 -26 -49 -26t-49 26l-126 162q-20 26 -11 44.5t42 18.5h80v1024h-80q-33 0 -42 18.5t11 44.5l126 162q20 26 49 26t49 -26l126 -162q20 -26 11 -44.5t-42 -18.5h-80v-1024h80zM81 1407l54 -27q12 -5 211 -5q44 0 132 2 t132 2q36 0 107.5 -0.5t107.5 -0.5h293q6 0 21 -0.5t20.5 0t16 3t17.5 9t15 17.5l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 48t-14.5 73.5t-7.5 35.5q-6 8 -12 12.5t-15.5 6t-13 2.5t-18 0.5t-16.5 -0.5 q-17 0 -66.5 0.5t-74.5 0.5t-64 -2t-71 -6q-9 -81 -8 -136q0 -94 2 -388t2 -455q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q19 42 19 383q0 101 -3 303t-3 303v117q0 2 0.5 15.5t0.5 25t-1 25.5t-3 24t-5 14q-11 12 -162 12q-33 0 -93 -12t-80 -26q-19 -13 -34 -72.5t-31.5 -111t-42.5 -53.5q-42 26 -56 44v383z" />
+<glyph unicode="&#xf035;" d="M81 1407l54 -27q12 -5 211 -5q44 0 132 2t132 2q70 0 246.5 1t304.5 0.5t247 -4.5q33 -1 56 31l42 1q4 0 14 -0.5t14 -0.5q2 -112 2 -336q0 -80 -5 -109q-39 -14 -68 -18q-25 44 -54 128q-3 9 -11 47.5t-15 73.5t-7 36q-10 13 -27 19q-5 2 -66 2q-30 0 -93 1t-103 1 t-94 -2t-96 -7q-9 -81 -8 -136l1 -152v52q0 -55 1 -154t1.5 -180t0.5 -153q0 -16 -2.5 -71.5t0 -91.5t12.5 -69q40 -21 124 -42.5t120 -37.5q5 -40 5 -50q0 -14 -3 -29l-34 -1q-76 -2 -218 8t-207 10q-50 0 -151 -9t-152 -9q-3 51 -3 52v9q17 27 61.5 43t98.5 29t78 27 q7 16 11.5 74t6 145.5t1.5 155t-0.5 153.5t-0.5 89q0 7 -2.5 21.5t-2.5 22.5q0 7 0.5 44t1 73t0 76.5t-3 67.5t-6.5 32q-11 12 -162 12q-41 0 -163 -13.5t-138 -24.5q-19 -12 -34 -71.5t-31.5 -111.5t-42.5 -54q-42 26 -56 44v383zM1310 125q12 0 42 -19.5t57.5 -41.5 t59.5 -49t36 -30q26 -21 26 -49t-26 -49q-4 -3 -36 -30t-59.5 -49t-57.5 -41.5t-42 -19.5q-13 0 -20.5 10.5t-10 28.5t-2.5 33.5t1.5 33t1.5 19.5h-1024q0 -2 1.5 -19.5t1.5 -33t-2.5 -33.5t-10 -28.5t-20.5 -10.5q-12 0 -42 19.5t-57.5 41
 .5t-59.5 49t-36 30q-26 21 -26 49 t26 49q4 3 36 30t59.5 49t57.5 41.5t42 19.5q13 0 20.5 -10.5t10 -28.5t2.5 -33.5t-1.5 -33t-1.5 -19.5h1024q0 2 -1.5 19.5t-1.5 33t2.5 33.5t10 28.5t20.5 10.5z" />
 <glyph unicode="&#xf036;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
 <glyph unicode="&#xf037;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1408 576v-128q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h896q26 0 45 -19t19 -45zM1664 960v-128q0 -26 -19 -45t-45 -19 h-1408q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1408q26 0 45 -19t19 -45zM1280 1344v-128q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h640q26 0 45 -19t19 -45z" />
 <glyph unicode="&#xf038;" horiz-adv-x="1792" d="M1792 192v-128q0 -26 -19 -45t-45 -19h-1664q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1664q26 0 45 -19t19 -45zM1792 576v-128q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1280q26 0 45 -19t19 -45zM1792 960v-128q0 -26 -19 -45 t-45 -19h-1536q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1536q26 0 45 -19t19 -45zM1792 1344v-128q0 -26 -19 -45t-45 -19h-1152q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
@@ -109,8 +110,8 @@
 <glyph unicode="&#xf050;" horiz-adv-x="1792" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v710q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19l-710 -710 q-19 -19 -32 -13t-13 32v710q-5 -10 -13 -19z" />
 <glyph unicode="&#xf051;" horiz-adv-x="1024" d="M45 -115q-19 -19 -32 -13t-13 32v1472q0 26 13 32t32 -13l710 -710q8 -8 13 -19v678q0 26 19 45t45 19h128q26 0 45 -19t19 -45v-1408q0 -26 -19 -45t-45 -19h-128q-26 0 -45 19t-19 45v678q-5 -10 -13 -19z" />
 <glyph unicode="&#xf052;" horiz-adv-x="1538" d="M14 557l710 710q19 19 45 19t45 -19l710 -710q19 -19 13 -32t-32 -13h-1472q-26 0 -32 13t13 32zM1473 0h-1408q-26 0 -45 19t-19 45v256q0 26 19 45t45 19h1408q26 0 45 -19t19 -45v-256q0 -26 -19 -45t-45 -19z" />
-<glyph unicode="&#xf053;" horiz-adv-x="1152" d="M742 -37l-652 651q-37 37 -37 90.5t37 90.5l652 651q37 37 90.5 37t90.5 -37l75 -75q37 -37 37 -90.5t-37 -90.5l-486 -486l486 -485q37 -38 37 -91t-37 -90l-75 -75q-37 -37 -90.5 -37t-90.5 37z" />
-<glyph unicode="&#xf054;" horiz-adv-x="1152" d="M1099 704q0 -52 -37 -91l-652 -651q-37 -37 -90 -37t-90 37l-76 75q-37 39 -37 91q0 53 37 90l486 486l-486 485q-37 39 -37 91q0 53 37 90l76 75q36 38 90 38t90 -38l652 -651q37 -37 37 -90z" />
+<glyph unicode="&#xf053;" horiz-adv-x="1280" d="M1171 1235l-531 -531l531 -531q19 -19 19 -45t-19 -45l-166 -166q-19 -19 -45 -19t-45 19l-742 742q-19 19 -19 45t19 45l742 742q19 19 45 19t45 -19l166 -166q19 -19 19 -45t-19 -45z" />
+<glyph unicode="&#xf054;" horiz-adv-x="1280" d="M1107 659l-742 -742q-19 -19 -45 -19t-45 19l-166 166q-19 19 -19 45t19 45l531 531l-531 531q-19 19 -19 45t19 45l166 166q19 19 45 19t45 -19l742 -742q19 -19 19 -45t-19 -45z" />
 <glyph unicode="&#xf055;" d="M1216 576v128q0 26 -19 45t-45 19h-256v256q0 26 -19 45t-45 19h-128q-26 0 -45 -19t-19 -45v-256h-256q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h256v-256q0 -26 19 -45t45 -19h128q26 0 45 19t19 45v256h256q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5 t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
 <glyph unicode="&#xf056;" d="M1216 576v128q0 26 -19 45t-45 19h-768q-26 0 -45 -19t-19 -45v-128q0 -26 19 -45t45 -19h768q26 0 45 19t19 45zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5 t103 -385.5z" />
 <glyph unicode="&#xf057;" d="M1149 414q0 26 -19 45l-181 181l181 181q19 19 19 45q0 27 -19 46l-90 90q-19 19 -46 19q-26 0 -45 -19l-181 -181l-181 181q-19 19 -45 19q-27 0 -46 -19l-90 -90q-19 -19 -19 -46q0 -26 19 -45l181 -181l-181 -181q-19 -19 -19 -45q0 -27 19 -46l90 -90q19 -19 46 -19 q26 0 45 19l181 181l181 -181q19 -19 45 -19q27 0 46 19l90 90q19 19 19 46zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
@@ -143,17 +144,17 @@
 <glyph unicode="&#xf074;" horiz-adv-x="1792" d="M666 1055q-60 -92 -137 -273q-22 45 -37 72.5t-40.5 63.5t-51 56.5t-63 35t-81.5 14.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q250 0 410 -225zM1792 256q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5v192q-32 0 -85 -0.5t-81 -1t-73 1 t-71 5t-64 10.5t-63 18.5t-58 28.5t-59 40t-55 53.5t-56 69.5q59 93 136 273q22 -45 37 -72.5t40.5 -63.5t51 -56.5t63 -35t81.5 -14.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23zM1792 1152q0 -14 -9 -23l-320 -320q-9 -9 -23 -9q-13 0 -22.5 9.5t-9.5 22.5 v192h-256q-48 0 -87 -15t-69 -45t-51 -61.5t-45 -77.5q-32 -62 -78 -171q-29 -66 -49.5 -111t-54 -105t-64 -100t-74 -83t-90 -68.5t-106.5 -42t-128 -16.5h-224q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h224q48 0 87 15t69 45t51 61.5t45 77.5q32 62 78 171q29 66 49.5 111 t54 105t64 100t74 83t90 68.5t106.5 42t128 16.5h256v192q0 14 9 23t23 9q12 0 24 -10l319 -319q9 -9 9 -23z" />
 <glyph unicode="&#xf075;" horiz-adv-x="1792" d="M1792 640q0 -174 -120 -321.5t-326 -233t-450 -85.5q-70 0 -145 8q-198 -175 -460 -242q-49 -14 -114 -22q-17 -2 -30.5 9t-17.5 29v1q-3 4 -0.5 12t2 10t4.5 9.5l6 9t7 8.5t8 9q7 8 31 34.5t34.5 38t31 39.5t32.5 51t27 59t26 76q-157 89 -247.5 220t-90.5 281 q0 130 71 248.5t191 204.5t286 136.5t348 50.5q244 0 450 -85.5t326 -233t120 -321.5z" />
 <glyph unicode="&#xf076;" d="M1536 704v-128q0 -201 -98.5 -362t-274 -251.5t-395.5 -90.5t-395.5 90.5t-274 251.5t-98.5 362v128q0 26 19 45t45 19h384q26 0 45 -19t19 -45v-128q0 -52 23.5 -90t53.5 -57t71 -30t64 -13t44 -2t44 2t64 13t71 30t53.5 57t23.5 90v128q0 26 19 45t45 19h384 q26 0 45 -19t19 -45zM512 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45zM1536 1344v-384q0 -26 -19 -45t-45 -19h-384q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h384q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf077;" horiz-adv-x="1664" d="M1611 320q0 -53 -37 -90l-75 -75q-38 -38 -91 -38q-54 0 -90 38l-486 485l-486 -485q-36 -38 -90 -38t-90 38l-75 75q-38 36 -38 90q0 53 38 91l651 651q37 37 90 37q52 0 91 -37l650 -651q38 -38 38 -91z" />
-<glyph unicode="&#xf078;" horiz-adv-x="1664" d="M1611 832q0 -53 -37 -90l-651 -651q-38 -38 -91 -38q-54 0 -90 38l-651 651q-38 36 -38 90q0 53 38 91l74 75q39 37 91 37q53 0 90 -37l486 -486l486 486q37 37 90 37q52 0 91 -37l75 -75q37 -39 37 -91z" />
+<glyph unicode="&#xf077;" horiz-adv-x="1792" d="M1683 205l-166 -165q-19 -19 -45 -19t-45 19l-531 531l-531 -531q-19 -19 -45 -19t-45 19l-166 165q-19 19 -19 45.5t19 45.5l742 741q19 19 45 19t45 -19l742 -741q19 -19 19 -45.5t-19 -45.5z" />
+<glyph unicode="&#xf078;" horiz-adv-x="1792" d="M1683 728l-742 -741q-19 -19 -45 -19t-45 19l-742 741q-19 19 -19 45.5t19 45.5l166 165q19 19 45 19t45 -19l531 -531l531 531q19 19 45 19t45 -19l166 -165q19 -19 19 -45.5t-19 -45.5z" />
 <glyph unicode="&#xf079;" horiz-adv-x="1920" d="M1280 32q0 -13 -9.5 -22.5t-22.5 -9.5h-960q-8 0 -13.5 2t-9 7t-5.5 8t-3 11.5t-1 11.5v13v11v160v416h-192q-26 0 -45 19t-19 45q0 24 15 41l320 384q19 22 49 22t49 -22l320 -384q15 -17 15 -41q0 -26 -19 -45t-45 -19h-192v-384h576q16 0 25 -11l160 -192q7 -11 7 -21 zM1920 448q0 -24 -15 -41l-320 -384q-20 -23 -49 -23t-49 23l-320 384q-15 17 -15 41q0 26 19 45t45 19h192v384h-576q-16 0 -25 12l-160 192q-7 9 -7 20q0 13 9.5 22.5t22.5 9.5h960q8 0 13.5 -2t9 -7t5.5 -8t3 -11.5t1 -11.5v-13v-11v-160v-416h192q26 0 45 -19t19 -45z " />
-<glyph unicode="&#xf07a;" horiz-adv-x="1664" d="M640 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1536 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1664 1088v-512q0 -24 -16 -42.5t-41 -21.5 l-1044 -122q1 -7 4.5 -21.5t6 -26.5t2.5 -22q0 -16 -24 -64h920q26 0 45 -19t19 -45t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 14 11 39.5t29.5 59.5t20.5 38l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t20 -15.5t13 -24.5t7.5 -26.5 t5.5 -29.5t4.5 -25.5h1201q26 0 45 -19t19 -45z" />
+<glyph unicode="&#xf07a;" horiz-adv-x="1664" d="M640 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1536 0q0 -52 -38 -90t-90 -38t-90 38t-38 90t38 90t90 38t90 -38t38 -90zM1664 1088v-512q0 -24 -16.5 -42.5t-40.5 -21.5l-1044 -122q13 -60 13 -70q0 -16 -24 -64h920q26 0 45 -19t19 -45 t-19 -45t-45 -19h-1024q-26 0 -45 19t-19 45q0 11 8 31.5t16 36t21.5 40t15.5 29.5l-177 823h-204q-26 0 -45 19t-19 45t19 45t45 19h256q16 0 28.5 -6.5t19.5 -15.5t13 -24.5t8 -26t5.5 -29.5t4.5 -26h1201q26 0 45 -19t19 -45z" />
 <glyph unicode="&#xf07b;" horiz-adv-x="1664" d="M1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" />
 <glyph unicode="&#xf07c;" horiz-adv-x="1920" d="M1879 584q0 -31 -31 -66l-336 -396q-43 -51 -120.5 -86.5t-143.5 -35.5h-1088q-34 0 -60.5 13t-26.5 43q0 31 31 66l336 396q43 51 120.5 86.5t143.5 35.5h1088q34 0 60.5 -13t26.5 -43zM1536 928v-160h-832q-94 0 -197 -47.5t-164 -119.5l-337 -396l-5 -6q0 4 -0.5 12.5 t-0.5 12.5v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158z" />
 <glyph unicode="&#xf07d;" horiz-adv-x="768" d="M704 1216q0 -26 -19 -45t-45 -19h-128v-1024h128q26 0 45 -19t19 -45t-19 -45l-256 -256q-19 -19 -45 -19t-45 19l-256 256q-19 19 -19 45t19 45t45 19h128v1024h-128q-26 0 -45 19t-19 45t19 45l256 256q19 19 45 19t45 -19l256 -256q19 -19 19 -45z" />
 <glyph unicode="&#xf07e;" horiz-adv-x="1792" d="M1792 640q0 -26 -19 -45l-256 -256q-19 -19 -45 -19t-45 19t-19 45v128h-1024v-128q0 -26 -19 -45t-45 -19t-45 19l-256 256q-19 19 -19 45t19 45l256 256q19 19 45 19t45 -19t19 -45v-128h1024v128q0 26 19 45t45 19t45 -19l256 -256q19 -19 19 -45z" />
-<glyph unicode="&#xf080;" horiz-adv-x="1920" d="M512 512v-384h-256v384h256zM896 1024v-896h-256v896h256zM1280 768v-640h-256v640h256zM1664 1152v-1024h-256v1024h256zM1792 32v1216q0 13 -9.5 22.5t-22.5 9.5h-1600q-13 0 -22.5 -9.5t-9.5 -22.5v-1216q0 -13 9.5 -22.5t22.5 -9.5h1600q13 0 22.5 9.5t9.5 22.5z M1920 1248v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600q66 0 113 -47t47 -113z" />
+<glyph unicode="&#xf080;" horiz-adv-x="2048" d="M640 640v-512h-256v512h256zM1024 1152v-1024h-256v1024h256zM2048 0v-128h-2048v1536h128v-1408h1920zM1408 896v-768h-256v768h256zM1792 1280v-1152h-256v1152h256z" />
 <glyph unicode="&#xf081;" d="M1280 926q-56 -25 -121 -34q68 40 93 117q-65 -38 -134 -51q-61 66 -153 66q-87 0 -148.5 -61.5t-61.5 -148.5q0 -29 5 -48q-129 7 -242 65t-192 155q-29 -50 -29 -106q0 -114 91 -175q-47 1 -100 26v-2q0 -75 50 -133.5t123 -72.5q-29 -8 -51 -8q-13 0 -39 4 q21 -63 74.5 -104t121.5 -42q-116 -90 -261 -90q-26 0 -50 3q148 -94 322 -94q112 0 210 35.5t168 95t120.5 137t75 162t24.5 168.5q0 18 -1 27q63 45 105 109zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5 t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf082;" d="M1307 618l23 219h-198v109q0 49 15.5 68.5t71.5 19.5h110v219h-175q-152 0 -218 -72t-66 -213v-131h-131v-219h131v-635h262v635h175zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960 q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf082;" d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-188v595h199l30 232h-229v148q0 56 23.5 84t91.5 28l122 1v207q-63 9 -178 9q-136 0 -217.5 -80t-81.5 -226v-171h-200v-232h200v-595h-532q-119 0 -203.5 84.5t-84.5 203.5v960 q0 119 84.5 203.5t203.5 84.5h960z" />
 <glyph unicode="&#xf083;" horiz-adv-x="1792" d="M928 704q0 14 -9 23t-23 9q-66 0 -113 -47t-47 -113q0 -14 9 -23t23 -9t23 9t9 23q0 40 28 68t68 28q14 0 23 9t9 23zM1152 574q0 -106 -75 -181t-181 -75t-181 75t-75 181t75 181t181 75t181 -75t75 -181zM128 0h1536v128h-1536v-128zM1280 574q0 159 -112.5 271.5 t-271.5 112.5t-271.5 -112.5t-112.5 -271.5t112.5 -271.5t271.5 -112.5t271.5 112.5t112.5 271.5zM256 1216h384v128h-384v-128zM128 1024h1536v118v138h-828l-64 -128h-644v-128zM1792 1280v-1280q0 -53 -37.5 -90.5t-90.5 -37.5h-1536q-53 0 -90.5 37.5t-37.5 90.5v1280 q0 53 37.5 90.5t90.5 37.5h1536q53 0 90.5 -37.5t37.5 -90.5z" />
 <glyph unicode="&#xf084;" horiz-adv-x="1792" d="M832 1024q0 80 -56 136t-136 56t-136 -56t-56 -136q0 -42 19 -83q-41 19 -83 19q-80 0 -136 -56t-56 -136t56 -136t136 -56t136 56t56 136q0 42 -19 83q41 -19 83 -19q80 0 136 56t56 136zM1683 320q0 -17 -49 -66t-66 -49q-9 0 -28.5 16t-36.5 33t-38.5 40t-24.5 26 l-96 -96l220 -220q28 -28 28 -68q0 -42 -39 -81t-81 -39q-40 0 -68 28l-671 671q-176 -131 -365 -131q-163 0 -265.5 102.5t-102.5 265.5q0 160 95 313t248 248t313 95q163 0 265.5 -102.5t102.5 -265.5q0 -189 -131 -365l355 -355l96 96q-3 3 -26 24.5t-40 38.5t-33 36.5 t-16 28.5q0 17 49 66t66 49q13 0 23 -10q6 -6 46 -44.5t82 -79.5t86.5 -86t73 -78t28.5 -41z" />
 <glyph unicode="&#xf085;" horiz-adv-x="1920" d="M896 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1664 128q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM1664 1152q0 52 -38 90t-90 38t-90 -38t-38 -90q0 -53 37.5 -90.5t90.5 -37.5 t90.5 37.5t37.5 90.5zM1280 731v-185q0 -10 -7 -19.5t-16 -10.5l-155 -24q-11 -35 -32 -76q34 -48 90 -115q7 -10 7 -20q0 -12 -7 -19q-23 -30 -82.5 -89.5t-78.5 -59.5q-11 0 -21 7l-115 90q-37 -19 -77 -31q-11 -108 -23 -155q-7 -24 -30 -24h-186q-11 0 -20 7.5t-10 17.5 l-23 153q-34 10 -75 31l-118 -89q-7 -7 -20 -7q-11 0 -21 8q-144 133 -144 160q0 9 7 19q10 14 41 53t47 61q-23 44 -35 82l-152 24q-10 1 -17 9.5t-7 19.5v185q0 10 7 19.5t16 10.5l155 24q11 35 32 76q-34 48 -90 115q-7 11 -7 20q0 12 7 20q22 30 82 89t79 59q11 0 21 -7 l115 -90q34 18 77 32q11 108 23 154q7 24 30 24h186q11 0 20 -7.5t10 -17.5l23 -153q34 -10 75 -31l118 89q8 7 20 7q11 0 21 -8q144 -133 144 -160q0 -9 -7 -19q-12 -16 -42 -54t-45 -60q23 -48 34 -82l152 
 -23q10 -2 17 -10.5t7 -19.5zM1920 198v-140q0 -16 -149 -31 q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20 t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31zM1920 1222v-140q0 -16 -149 -31q-12 -27 -30 -52q51 -113 51 -138q0 -4 -4 -7q-122 -71 -124 -71q-8 0 -46 47t-52 68 q-20 -2 -30 -2t-30 2q-14 -21 -52 -68t-46 -47q-2 0 -124 71q-4 3 -4 7q0 25 51 138q-18 25 -30 52q-149 15 -149 31v140q0 16 149 31q13 29 30 52q-51 113 -51 138q0 4 4 7q4 2 35 20t59 34t30 16q8 0 46 -46.5t52 -67.5q20 2 30 2t30 -2q51 71 92 112l6 2q4 0 124 -70 q4 -3 4 -7q0 -25 -51 -138q17 -23 30 -52q149 -15 149 -31z" />
@@ -168,7 +169,7 @@
 <glyph unicode="&#xf08e;" horiz-adv-x="1792" d="M1408 608v-320q0 -119 -84.5 -203.5t-203.5 -84.5h-832q-119 0 -203.5 84.5t-84.5 203.5v832q0 119 84.5 203.5t203.5 84.5h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-66 0 -113 -47t-47 -113v-832q0 -66 47 -113t113 -47h832q66 0 113 47t47 113v320 q0 14 9 23t23 9h64q14 0 23 -9t9 -23zM1792 1472v-512q0 -26 -19 -45t-45 -19t-45 19l-176 176l-652 -652q-10 -10 -23 -10t-23 10l-114 114q-10 10 -10 23t10 23l652 652l-176 176q-19 19 -19 45t19 45t45 19h512q26 0 45 -19t19 -45z" />
 <glyph unicode="&#xf090;" d="M1184 640q0 -26 -19 -45l-544 -544q-19 -19 -45 -19t-45 19t-19 45v288h-448q-26 0 -45 19t-19 45v384q0 26 19 45t45 19h448v288q0 26 19 45t45 19t45 -19l544 -544q19 -19 19 -45zM1536 992v-704q0 -119 -84.5 -203.5t-203.5 -84.5h-320q-13 0 -22.5 9.5t-9.5 22.5 q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q66 0 113 47t47 113v704q0 66 -47 113t-113 47h-288h-11h-13t-11.5 1t-11.5 3t-8 5.5t-7 9t-2 13.5q0 4 -1 20t-0.5 26.5t3 23.5t10 19.5t20.5 6.5h320q119 0 203.5 -84.5t84.5 -203.5z" />
 <glyph unicode="&#xf091;" horiz-adv-x="1664" d="M458 653q-74 162 -74 371h-256v-96q0 -78 94.5 -162t235.5 -113zM1536 928v96h-256q0 -209 -74 -371q141 29 235.5 113t94.5 162zM1664 1056v-128q0 -71 -41.5 -143t-112 -130t-173 -97.5t-215.5 -44.5q-42 -54 -95 -95q-38 -34 -52.5 -72.5t-14.5 -89.5q0 -54 30.5 -91 t97.5 -37q75 0 133.5 -45.5t58.5 -114.5v-64q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v64q0 69 58.5 114.5t133.5 45.5q67 0 97.5 37t30.5 91q0 51 -14.5 89.5t-52.5 72.5q-53 41 -95 95q-113 5 -215.5 44.5t-173 97.5t-112 130t-41.5 143v128q0 40 28 68t68 28h288v96 q0 66 47 113t113 47h576q66 0 113 -47t47 -113v-96h288q40 0 68 -28t28 -68z" />
-<glyph unicode="&#xf092;" d="M394 184q-8 -9 -20 3q-13 11 -4 19q8 9 20 -3q12 -11 4 -19zM352 245q9 -12 0 -19q-8 -6 -17 7t0 18q9 7 17 -6zM291 305q-5 -7 -13 -2q-10 5 -7 12q3 5 13 2q10 -5 7 -12zM322 271q-6 -7 -16 3q-9 11 -2 16q6 6 16 -3q9 -11 2 -16zM451 159q-4 -12 -19 -6q-17 4 -13 15 t19 7q16 -5 13 -16zM514 154q0 -11 -16 -11q-17 -2 -17 11q0 11 16 11q17 2 17 -11zM572 164q2 -10 -14 -14t-18 8t14 15q16 2 18 -9zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-224q-16 0 -24.5 1t-19.5 5t-16 14.5t-5 27.5v239q0 97 -52 142q57 6 102.5 18t94 39 t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103 q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -103t0.5 
 -68q0 -22 -11 -33.5t-22 -13t-33 -1.5 h-224q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf092;" d="M519 336q4 6 -3 13q-9 7 -14 2q-4 -6 3 -13q9 -7 14 -2zM491 377q-5 7 -12 4q-6 -4 0 -12q7 -8 12 -5q6 4 0 13zM450 417q2 4 -5 8q-7 2 -8 -2q-3 -5 4 -8q8 -2 9 2zM471 394q2 1 1.5 4.5t-3.5 5.5q-6 7 -10 3t1 -11q6 -6 11 -2zM557 319q2 7 -9 11q-9 3 -13 -4 q-2 -7 9 -11q9 -3 13 4zM599 316q0 8 -12 8q-10 0 -10 -8t11 -8t11 8zM638 323q-2 7 -13 5t-9 -9q2 -8 12 -6t10 10zM1280 640q0 212 -150 362t-362 150t-362 -150t-150 -362q0 -167 98 -300.5t252 -185.5q18 -3 26.5 5t8.5 20q0 52 -1 95q-6 -1 -15.5 -2.5t-35.5 -2t-48 4 t-43.5 20t-29.5 41.5q-23 59 -57 74q-2 1 -4.5 3.5l-8 8t-7 9.5t4 7.5t19.5 3.5q6 0 15 -2t30 -15.5t33 -35.5q16 -28 37.5 -42t43.5 -14t38 3.5t30 9.5q7 47 33 69q-49 6 -86 18.5t-73 39t-55.5 76t-19.5 119.5q0 79 53 137q-24 62 5 136q19 6 54.5 -7.5t60.5 -29.5l26 -16 q58 17 128 17t128 -17q11 7 28.5 18t55.5 26t57 9q29 -74 5 -136q53 -58 53 -137q0 -57 -14 -100.5t-35.5 -70t-53.5 -44.5t-62.5 -26t-68.5 -12q35 -31 35 -95q0 -40 -0.5 -89t-0.5 -51q0 -12 8.5 -20t26.5 -5q154 52 252 185.5t98 
 300.5zM1536 1120v-960 q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
 <glyph unicode="&#xf093;" horiz-adv-x="1664" d="M1280 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1536 64q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 288v-320q0 -40 -28 -68t-68 -28h-1472q-40 0 -68 28t-28 68v320q0 40 28 68t68 28h427q21 -56 70.5 -92 t110.5 -36h256q61 0 110.5 36t70.5 92h427q40 0 68 -28t28 -68zM1339 936q-17 -40 -59 -40h-256v-448q0 -26 -19 -45t-45 -19h-256q-26 0 -45 19t-19 45v448h-256q-42 0 -59 40q-17 39 14 69l448 448q18 19 45 19t45 -19l448 -448q31 -30 14 -69z" />
 <glyph unicode="&#xf094;" d="M1407 710q0 44 -7 113.5t-18 96.5q-12 30 -17 44t-9 36.5t-4 48.5q0 23 5 68.5t5 67.5q0 37 -10 55q-4 1 -13 1q-19 0 -58 -4.5t-59 -4.5q-60 0 -176 24t-175 24q-43 0 -94.5 -11.5t-85 -23.5t-89.5 -34q-137 -54 -202 -103q-96 -73 -159.5 -189.5t-88 -236t-24.5 -248.5 q0 -40 12.5 -120t12.5 -121q0 -23 -11 -66.5t-11 -65.5t12 -36.5t34 -14.5q24 0 72.5 11t73.5 11q57 0 169.5 -15.5t169.5 -15.5q181 0 284 36q129 45 235.5 152.5t166 245.5t59.5 275zM1535 712q0 -165 -70 -327.5t-196 -288t-281 -180.5q-124 -44 -326 -44 q-57 0 -170 14.5t-169 14.5q-24 0 -72.5 -14.5t-73.5 -14.5q-73 0 -123.5 55.5t-50.5 128.5q0 24 11 68t11 67q0 40 -12.5 120.5t-12.5 121.5q0 111 18 217.5t54.5 209.5t100.5 194t150 156q78 59 232 120q194 78 316 78q60 0 175.5 -24t173.5 -24q19 0 57 5t58 5 q81 0 118 -50.5t37 -134.5q0 -23 -5 -68t-5 -68q0 -10 1 -18.5t3 -17t4 -13.5t6.5 -16t6.5 -17q16 -40 25 -118.5t9 -136.5z" />
 <glyph unicode="&#xf095;" horiz-adv-x="1408" d="M1408 296q0 -27 -10 -70.5t-21 -68.5q-21 -50 -122 -106q-94 -51 -186 -51q-27 0 -52.5 3.5t-57.5 12.5t-47.5 14.5t-55.5 20.5t-49 18q-98 35 -175 83q-128 79 -264.5 215.5t-215.5 264.5q-48 77 -83 175q-3 9 -18 49t-20.5 55.5t-14.5 47.5t-12.5 57.5t-3.5 52.5 q0 92 51 186q56 101 106 122q25 11 68.5 21t70.5 10q14 0 21 -3q18 -6 53 -76q11 -19 30 -54t35 -63.5t31 -53.5q3 -4 17.5 -25t21.5 -35.5t7 -28.5q0 -20 -28.5 -50t-62 -55t-62 -53t-28.5 -46q0 -9 5 -22.5t8.5 -20.5t14 -24t11.5 -19q76 -137 174 -235t235 -174 q2 -1 19 -11.5t24 -14t20.5 -8.5t22.5 -5q18 0 46 28.5t53 62t55 62t50 28.5q14 0 28.5 -7t35.5 -21.5t25 -17.5q25 -15 53.5 -31t63.5 -35t54 -30q70 -35 76 -53q3 -7 3 -21z" />
@@ -176,14 +177,14 @@
 <glyph unicode="&#xf097;" horiz-adv-x="1280" d="M1152 1280h-1024v-1242l423 406l89 85l89 -85l423 -406v1242zM1164 1408q23 0 44 -9q33 -13 52.5 -41t19.5 -62v-1289q0 -34 -19.5 -62t-52.5 -41q-19 -8 -44 -8q-48 0 -83 32l-441 424l-441 -424q-36 -33 -83 -33q-23 0 -44 9q-33 13 -52.5 41t-19.5 62v1289 q0 34 19.5 62t52.5 41q21 9 44 9h1048z" />
 <glyph unicode="&#xf098;" d="M1280 343q0 11 -2 16q-3 8 -38.5 29.5t-88.5 49.5l-53 29q-5 3 -19 13t-25 15t-21 5q-18 0 -47 -32.5t-57 -65.5t-44 -33q-7 0 -16.5 3.5t-15.5 6.5t-17 9.5t-14 8.5q-99 55 -170.5 126.5t-126.5 170.5q-2 3 -8.5 14t-9.5 17t-6.5 15.5t-3.5 16.5q0 13 20.5 33.5t45 38.5 t45 39.5t20.5 36.5q0 10 -5 21t-15 25t-13 19q-3 6 -15 28.5t-25 45.5t-26.5 47.5t-25 40.5t-16.5 18t-16 2q-48 0 -101 -22q-46 -21 -80 -94.5t-34 -130.5q0 -16 2.5 -34t5 -30.5t9 -33t10 -29.5t12.5 -33t11 -30q60 -164 216.5 -320.5t320.5 -216.5q6 -2 30 -11t33 -12.5 t29.5 -10t33 -9t30.5 -5t34 -2.5q57 0 130.5 34t94.5 80q22 53 22 101zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
 <glyph unicode="&#xf099;" horiz-adv-x="1664" d="M1620 1128q-67 -98 -162 -167q1 -14 1 -42q0 -130 -38 -259.5t-115.5 -248.5t-184.5 -210.5t-258 -146t-323 -54.5q-271 0 -496 145q35 -4 78 -4q225 0 401 138q-105 2 -188 64.5t-114 159.5q33 -5 61 -5q43 0 85 11q-112 23 -185.5 111.5t-73.5 205.5v4q68 -38 146 -41 q-66 44 -105 115t-39 154q0 88 44 163q121 -149 294.5 -238.5t371.5 -99.5q-8 38 -8 74q0 134 94.5 228.5t228.5 94.5q140 0 236 -102q109 21 205 78q-37 -115 -142 -178q93 10 186 50z" />
-<glyph unicode="&#xf09a;" horiz-adv-x="768" d="M511 980h257l-30 -284h-227v-824h-341v824h-170v284h170v171q0 182 86 275.5t283 93.5h227v-284h-142q-39 0 -62.5 -6.5t-34 -23.5t-13.5 -34.5t-3 -49.5v-142z" />
-<glyph unicode="&#xf09b;" d="M1536 640q0 -251 -146.5 -451.5t-378.5 -277.5q-27 -5 -39.5 7t-12.5 30v211q0 97 -52 142q57 6 102.5 18t94 39t81 66.5t53 105t20.5 150.5q0 121 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-86 13.5 q-44 -113 -7 -204q-79 -85 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-40 -36 -49 -103q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23 q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -89t0.5 -54q0 -18 -13 -30t-40 -7q-232 77 -378.5 277.5t-146.5 451.5q0 209 103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
+<glyph unicode="&#xf09a;" horiz-adv-x="1024" d="M959 1524v-264h-157q-86 0 -116 -36t-30 -108v-189h293l-39 -296h-254v-759h-306v759h-255v296h255v218q0 186 104 288.5t277 102.5q147 0 228 -12z" />
+<glyph unicode="&#xf09b;" d="M768 1408q209 0 385.5 -103t279.5 -279.5t103 -385.5q0 -251 -146.5 -451.5t-378.5 -277.5q-27 -5 -40 7t-13 30q0 3 0.5 76.5t0.5 134.5q0 97 -52 142q57 6 102.5 18t94 39t81 66.5t53 105t20.5 150.5q0 119 -79 206q37 91 -8 204q-28 9 -81 -11t-92 -44l-38 -24 q-93 26 -192 26t-192 -26q-16 11 -42.5 27t-83.5 38.5t-85 13.5q-45 -113 -8 -204q-79 -87 -79 -206q0 -85 20.5 -150t52.5 -105t80.5 -67t94 -39t102.5 -18q-39 -36 -49 -103q-21 -10 -45 -15t-57 -5t-65.5 21.5t-55.5 62.5q-19 32 -48.5 52t-49.5 24l-20 3q-21 0 -29 -4.5 t-5 -11.5t9 -14t13 -12l7 -5q22 -10 43.5 -38t31.5 -51l10 -23q13 -38 44 -61.5t67 -30t69.5 -7t55.5 3.5l23 4q0 -38 0.5 -88.5t0.5 -54.5q0 -18 -13 -30t-40 -7q-232 77 -378.5 277.5t-146.5 451.5q0 209 103 385.5t279.5 279.5t385.5 103zM291 305q3 7 -7 12 q-10 3 -13 -2q-3 -7 7 -12q9 -6 13 2zM322 271q7 5 -2 16q-10 9 -16 3q-7 -5 2 -16q10 -10 16 -3zM352 226q9 7 0 19q-8 13 -17 6q-9 -5 0 -18t17 -7zM394 184q8 8 -4 19q-12 12 -20 3q-9 -8 4 -19q12 -12 20 -3zM451 159q3 11 -13 16q-15 4 -
 19 -7t13 -15q15 -6 19 6z M514 154q0 13 -17 11q-16 0 -16 -11q0 -13 17 -11q16 0 16 11zM572 164q-2 11 -18 9q-16 -3 -14 -15t18 -8t14 14z" />
 <glyph unicode="&#xf09c;" horiz-adv-x="1664" d="M1664 960v-256q0 -26 -19 -45t-45 -19h-64q-26 0 -45 19t-19 45v256q0 106 -75 181t-181 75t-181 -75t-75 -181v-192h96q40 0 68 -28t28 -68v-576q0 -40 -28 -68t-68 -28h-960q-40 0 -68 28t-28 68v576q0 40 28 68t68 28h672v192q0 185 131.5 316.5t316.5 131.5 t316.5 -131.5t131.5 -316.5z" />
 <glyph unicode="&#xf09d;" horiz-adv-x="1920" d="M1760 1408q66 0 113 -47t47 -113v-1216q0 -66 -47 -113t-113 -47h-1600q-66 0 -113 47t-47 113v1216q0 66 47 113t113 47h1600zM160 1280q-13 0 -22.5 -9.5t-9.5 -22.5v-224h1664v224q0 13 -9.5 22.5t-22.5 9.5h-1600zM1760 0q13 0 22.5 9.5t9.5 22.5v608h-1664v-608 q0 -13 9.5 -22.5t22.5 -9.5h1600zM256 128v128h256v-128h-256zM640 128v128h384v-128h-384z" />
 <glyph unicode="&#xf09e;" horiz-adv-x="1408" d="M384 192q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM896 69q2 -28 -17 -48q-18 -21 -47 -21h-135q-25 0 -43 16.5t-20 41.5q-22 229 -184.5 391.5t-391.5 184.5q-25 2 -41.5 20t-16.5 43v135q0 29 21 47q17 17 43 17h5q160 -13 306 -80.5 t259 -181.5q114 -113 181.5 -259t80.5 -306zM1408 67q2 -27 -18 -47q-18 -20 -46 -20h-143q-26 0 -44.5 17.5t-19.5 42.5q-12 215 -101 408.5t-231.5 336t-336 231.5t-408.5 102q-25 1 -42.5 19.5t-17.5 43.5v143q0 28 20 46q18 18 44 18h3q262 -13 501.5 -120t425.5 -294 q187 -186 294 -425.5t120 -501.5z" />
 <glyph unicode="&#xf0a0;" d="M1040 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1296 320q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5zM1408 160v320q0 13 -9.5 22.5t-22.5 9.5 h-1216q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h1216q13 0 22.5 9.5t9.5 22.5zM178 640h1180l-157 482q-4 13 -16 21.5t-26 8.5h-782q-14 0 -26 -8.5t-16 -21.5zM1536 480v-320q0 -66 -47 -113t-113 -47h-1216q-66 0 -113 47t-47 113v320q0 25 16 75 l197 606q17 53 63 86t101 33h782q55 0 101 -33t63 -86l197 -606q16 -50 16 -75z" />
 <glyph unicode="&#xf0a1;" horiz-adv-x="1792" d="M1664 896q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5v-384q0 -52 -38 -90t-90 -38q-417 347 -812 380q-58 -19 -91 -66t-31 -100.5t40 -92.5q-20 -33 -23 -65.5t6 -58t33.5 -55t48 -50t61.5 -50.5q-29 -58 -111.5 -83t-168.5 -11.5t-132 55.5q-7 23 -29.5 87.5 t-32 94.5t-23 89t-15 101t3.5 98.5t22 110.5h-122q-66 0 -113 47t-47 113v192q0 66 47 113t113 47h480q435 0 896 384q52 0 90 -38t38 -90v-384zM1536 292v954q-394 -302 -768 -343v-270q377 -42 768 -341z" />
-<glyph unicode="&#xf0a2;" horiz-adv-x="1664" d="M848 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM183 128h1298q-164 181 -246.5 411.5t-82.5 484.5q0 256 -320 256t-320 -256q0 -254 -82.5 -484.5t-246.5 -411.5zM1664 128q0 -52 -38 -90t-90 -38 h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q190 161 287 397.5t97 498.5q0 165 96 262t264 117q-8 18 -8 37q0 40 28 68t68 28t68 -28t28 -68q0 -19 -8 -37q168 -20 264 -117t96 -262q0 -262 97 -498.5t287 -397.5z" />
+<glyph unicode="&#xf0a2;" horiz-adv-x="1792" d="M912 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM246 128h1300q-266 300 -266 832q0 51 -24 105t-69 103t-121.5 80.5t-169.5 31.5t-169.5 -31.5t-121.5 -80.5t-69 -103t-24 -105q0 -532 -266 -832z M1728 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q50 42 91 88t85 119.5t74.5 158.5t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q190 -28 307 -158.5 t117 -282.5q0 -139 19.5 -260t50 -206t74.5 -158.5t85 -119.5t91 -88z" />
 <glyph unicode="&#xf0a3;" d="M1376 640l138 -135q30 -28 20 -70q-12 -41 -52 -51l-188 -48l53 -186q12 -41 -19 -70q-29 -31 -70 -19l-186 53l-48 -188q-10 -40 -51 -52q-12 -2 -19 -2q-31 0 -51 22l-135 138l-135 -138q-28 -30 -70 -20q-41 11 -51 52l-48 188l-186 -53q-41 -12 -70 19q-31 29 -19 70 l53 186l-188 48q-40 10 -52 51q-10 42 20 70l138 135l-138 135q-30 28 -20 70q12 41 52 51l188 48l-53 186q-12 41 19 70q29 31 70 19l186 -53l48 188q10 41 51 51q41 12 70 -19l135 -139l135 139q29 30 70 19q41 -10 51 -51l48 -188l186 53q41 12 70 -19q31 -29 19 -70 l-53 -186l188 -48q40 -10 52 -51q10 -42 -20 -70z" />
 <glyph unicode="&#xf0a4;" horiz-adv-x="1792" d="M256 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1664 768q0 51 -39 89.5t-89 38.5h-576q0 20 15 48.5t33 55t33 68t15 84.5q0 67 -44.5 97.5t-115.5 30.5q-24 0 -90 -139q-24 -44 -37 -65q-40 -64 -112 -145q-71 -81 -101 -106 q-69 -57 -140 -57h-32v-640h32q72 0 167 -32t193.5 -64t179.5 -32q189 0 189 167q0 26 -5 56q30 16 47.5 52.5t17.5 73.5t-18 69q53 50 53 119q0 25 -10 55.5t-25 47.5h331q52 0 90 38t38 90zM1792 769q0 -105 -75.5 -181t-180.5 -76h-169q-4 -62 -37 -119q3 -21 3 -43 q0 -101 -60 -178q1 -139 -85 -219.5t-227 -80.5q-133 0 -322 69q-164 59 -223 59h-288q-53 0 -90.5 37.5t-37.5 90.5v640q0 53 37.5 90.5t90.5 37.5h288q10 0 21.5 4.5t23.5 14t22.5 18t24 22.5t20.5 21.5t19 21.5t14 17q65 74 100 129q13 21 33 62t37 72t40.5 63t55 49.5 t69.5 17.5q125 0 206.5 -67t81.5 -189q0 -68 -22 -128h374q104 0 180 -76t76 -179z" />
 <glyph unicode="&#xf0a5;" horiz-adv-x="1792" d="M1376 128h32v640h-32q-35 0 -67.5 12t-62.5 37t-50 46t-49 54q-2 3 -3.5 4.5t-4 4.5t-4.5 5q-72 81 -112 145q-14 22 -38 68q-1 3 -10.5 22.5t-18.5 36t-20 35.5t-21.5 30.5t-18.5 11.5q-71 0 -115.5 -30.5t-44.5 -97.5q0 -43 15 -84.5t33 -68t33 -55t15 -48.5h-576 q-50 0 -89 -38.5t-39 -89.5q0 -52 38 -90t90 -38h331q-15 -17 -25 -47.5t-10 -55.5q0 -69 53 -119q-18 -32 -18 -69t17.5 -73.5t47.5 -52.5q-4 -24 -4 -56q0 -85 48.5 -126t135.5 -41q84 0 183 32t194 64t167 32zM1664 192q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45 t45 -19t45 19t19 45zM1792 768v-640q0 -53 -37.5 -90.5t-90.5 -37.5h-288q-59 0 -223 -59q-190 -69 -317 -69q-142 0 -230 77.5t-87 217.5l1 5q-61 76 -61 178q0 22 3 43q-33 57 -37 119h-169q-105 0 -180.5 76t-75.5 181q0 103 76 179t180 76h374q-22 60 -22 128 q0 122 81.5 189t206.5 67q38 0 69.5 -17.5t55 -49.5t40.5 -63t37 -72t33 -62q35 -55 100 -129q2 -3 14 -17t19 -21.5t20.5 -21.5t24 -22.5t22.5 -18t23.5 -14t21.5 -4.5h288q53 0 90.5 -37.5t37.5 -90.5z" />
@@ -218,8 +219,8 @@
 <glyph unicode="&#xf0d1;" horiz-adv-x="1792" d="M640 128q0 52 -38 90t-90 38t-90 -38t-38 -90t38 -90t90 -38t90 38t38 90zM256 640h384v256h-158q-13 0 -22 -9l-195 -195q-9 -9 -9 -22v-30zM1536 128q0 52 -38 90t-90 38t-90 -38t-38 -90t38 -90t90 -38t90 38t38 90zM1792 1216v-1024q0 -15 -4 -26.5t-13.5 -18.5 t-16.5 -11.5t-23.5 -6t-22.5 -2t-25.5 0t-22.5 0.5q0 -106 -75 -181t-181 -75t-181 75t-75 181h-384q0 -106 -75 -181t-181 -75t-181 75t-75 181h-64q-3 0 -22.5 -0.5t-25.5 0t-22.5 2t-23.5 6t-16.5 11.5t-13.5 18.5t-4 26.5q0 26 19 45t45 19v320q0 8 -0.5 35t0 38 t2.5 34.5t6.5 37t14 30.5t22.5 30l198 198q19 19 50.5 32t58.5 13h160v192q0 26 19 45t45 19h1024q26 0 45 -19t19 -45z" />
 <glyph unicode="&#xf0d2;" d="M1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103q-111 0 -218 32q59 93 78 164q9 34 54 211q20 -39 73 -67.5t114 -28.5q121 0 216 68.5t147 188.5t52 270q0 114 -59.5 214t-172.5 163t-255 63q-105 0 -196 -29t-154.5 -77t-109 -110.5t-67 -129.5t-21.5 -134 q0 -104 40 -183t117 -111q30 -12 38 20q2 7 8 31t8 30q6 23 -11 43q-51 61 -51 151q0 151 104.5 259.5t273.5 108.5q151 0 235.5 -82t84.5 -213q0 -170 -68.5 -289t-175.5 -119q-61 0 -98 43.5t-23 104.5q8 35 26.5 93.5t30 103t11.5 75.5q0 50 -27 83t-77 33 q-62 0 -105 -57t-43 -142q0 -73 25 -122l-99 -418q-17 -70 -13 -177q-206 91 -333 281t-127 423q0 209 103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
 <glyph unicode="&#xf0d3;" d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-725q85 122 108 210q9 34 53 209q21 -39 73.5 -67t112.5 -28q181 0 295.5 147.5t114.5 373.5q0 84 -35 162.5t-96.5 139t-152.5 97t-197 36.5q-104 0 -194.5 -28.5t-153 -76.5 t-107.5 -109.5t-66.5 -128t-21.5 -132.5q0 -102 39.5 -180t116.5 -110q13 -5 23.5 0t14.5 19q10 44 15 61q6 23 -11 42q-50 62 -50 150q0 150 103.5 256.5t270.5 106.5q149 0 232.5 -81t83.5 -210q0 -168 -67.5 -286t-173.5 -118q-60 0 -97 43.5t-23 103.5q8 34 26.5 92.5 t29.5 102t11 74.5q0 49 -26.5 81.5t-75.5 32.5q-61 0 -103.5 -56.5t-42.5 -139.5q0 -72 24 -121l-98 -414q-24 -100 -7 -254h-183q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960z" />
-<glyph unicode="&#xf0d4;" d="M678 -57q0 -38 -10 -71h-380q-95 0 -171.5 56.5t-103.5 147.5q24 45 69 77.5t100 49.5t107 24t107 7q32 0 49 -2q6 -4 30.5 -21t33 -23t31 -23t32 -25.5t27.5 -25.5t26.5 -29.5t21 -30.5t17.5 -34.5t9.5 -36t4.5 -40.5zM385 294q-234 -7 -385 -85v433q103 -118 273 -118 q32 0 70 5q-21 -61 -21 -86q0 -67 63 -149zM558 805q0 -100 -43.5 -160.5t-140.5 -60.5q-51 0 -97 26t-78 67.5t-56 93.5t-35.5 104t-11.5 99q0 96 51.5 165t144.5 69q66 0 119 -41t84 -104t47 -130t16 -128zM1536 896v-736q0 -119 -84.5 -203.5t-203.5 -84.5h-468 q39 73 39 157q0 66 -22 122.5t-55.5 93t-72 71t-72 59.5t-55.5 54.5t-22 59.5q0 36 23 68t56 61.5t65.5 64.5t55.5 93t23 131t-26.5 145.5t-75.5 118.5q-6 6 -14 11t-12.5 7.5t-10 9.5t-10.5 17h135l135 64h-437q-138 0 -244.5 -38.5t-182.5 -133.5q0 126 81 213t207 87h960 q119 0 203.5 -84.5t84.5 -203.5v-96h-256v256h-128v-256h-256v-128h256v-256h128v256h256z" />
-<glyph unicode="&#xf0d5;" horiz-adv-x="1664" d="M876 71q0 21 -4.5 40.5t-9.5 36t-17.5 34.5t-21 30.5t-26.5 29.5t-27.5 25.5t-32 25.5t-31 23t-33 23t-30.5 21q-17 2 -50 2q-54 0 -106 -7t-108 -25t-98 -46t-69 -75t-27 -107q0 -68 35.5 -121.5t93 -84t120.5 -45.5t127 -15q59 0 112.5 12.5t100.5 39t74.5 73.5 t27.5 110zM756 933q0 60 -16.5 127.5t-47 130.5t-84 104t-119.5 41q-93 0 -144 -69t-51 -165q0 -47 11.5 -99t35.5 -104t56 -93.5t78 -67.5t97 -26q97 0 140.5 60.5t43.5 160.5zM625 1408h437l-135 -79h-135q71 -45 110 -126t39 -169q0 -74 -23 -131.5t-56 -92.5t-66 -64.5 t-56 -61t-23 -67.5q0 -26 16.5 -51t43 -48t58.5 -48t64 -55.5t58.5 -66t43 -85t16.5 -106.5q0 -160 -140 -282q-152 -131 -420 -131q-59 0 -119.5 10t-122 33.5t-108.5 58t-77 89t-30 121.5q0 61 37 135q32 64 96 110.5t145 71t155 36t150 13.5q-64 83 -64 149q0 12 2 23.5 t5 19.5t8 21.5t7 21.5q-40 -5 -70 -5q-149 0 -255.5 98t-106.5 246q0 140 95 250.5t234 141.5q94 20 187 20zM1664 1152v-128h-256v-256h-128v256h-256v128h256v256h128v-256h256z" />
+<glyph unicode="&#xf0d4;" d="M917 631q0 26 -6 64h-362v-132h217q-3 -24 -16.5 -50t-37.5 -53t-66.5 -44.5t-96.5 -17.5q-99 0 -169 71t-70 171t70 171t169 71q92 0 153 -59l104 101q-108 100 -257 100q-160 0 -272 -112.5t-112 -271.5t112 -271.5t272 -112.5q165 0 266.5 105t101.5 270zM1262 585 h109v110h-109v110h-110v-110h-110v-110h110v-110h110v110zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf0d5;" horiz-adv-x="2304" d="M1437 623q0 -208 -87 -370.5t-248 -254t-369 -91.5q-149 0 -285 58t-234 156t-156 234t-58 285t58 285t156 234t234 156t285 58q286 0 491 -192l-199 -191q-117 113 -292 113q-123 0 -227.5 -62t-165.5 -168.5t-61 -232.5t61 -232.5t165.5 -168.5t227.5 -62 q83 0 152.5 23t114.5 57.5t78.5 78.5t49 83t21.5 74h-416v252h692q12 -63 12 -122zM2304 745v-210h-209v-209h-210v209h-209v210h209v209h210v-209h209z" />
 <glyph unicode="&#xf0d6;" horiz-adv-x="1920" d="M768 384h384v96h-128v448h-114l-148 -137l77 -80q42 37 55 57h2v-288h-128v-96zM1280 640q0 -70 -21 -142t-59.5 -134t-101.5 -101t-138 -39t-138 39t-101.5 101t-59.5 134t-21 142t21 142t59.5 134t101.5 101t138 39t138 -39t101.5 -101t59.5 -134t21 -142zM1792 384 v512q-106 0 -181 75t-75 181h-1152q0 -106 -75 -181t-181 -75v-512q106 0 181 -75t75 -181h1152q0 106 75 181t181 75zM1920 1216v-1152q0 -26 -19 -45t-45 -19h-1792q-26 0 -45 19t-19 45v1152q0 26 19 45t45 19h1792q26 0 45 -19t19 -45z" />
 <glyph unicode="&#xf0d7;" horiz-adv-x="1024" d="M1024 832q0 -26 -19 -45l-448 -448q-19 -19 -45 -19t-45 19l-448 448q-19 19 -19 45t19 45t45 19h896q26 0 45 -19t19 -45z" />
 <glyph unicode="&#xf0d8;" horiz-adv-x="1024" d="M1024 320q0 -26 -19 -45t-45 -19h-896q-26 0 -45 19t-19 45t19 45l448 448q19 19 45 19t45 -19l448 -448q19 -19 19 -45z" />
@@ -247,10 +248,10 @@
 <glyph unicode="&#xf0f0;" horiz-adv-x="1408" d="M384 192q0 -26 -19 -45t-45 -19t-45 19t-19 45t19 45t45 19t45 -19t19 -45zM1408 131q0 -121 -73 -190t-194 -69h-874q-121 0 -194 69t-73 190q0 68 5.5 131t24 138t47.5 132.5t81 103t120 60.5q-22 -52 -22 -120v-203q-58 -20 -93 -70t-35 -111q0 -80 56 -136t136 -56 t136 56t56 136q0 61 -35.5 111t-92.5 70v203q0 62 25 93q132 -104 295 -104t295 104q25 -31 25 -93v-64q-106 0 -181 -75t-75 -181v-89q-32 -29 -32 -71q0 -40 28 -68t68 -28t68 28t28 68q0 42 -32 71v89q0 52 38 90t90 38t90 -38t38 -90v-89q-32 -29 -32 -71q0 -40 28 -68 t68 -28t68 28t28 68q0 42 -32 71v89q0 68 -34.5 127.5t-93.5 93.5q0 10 0.5 42.5t0 48t-2.5 41.5t-7 47t-13 40q68 -15 120 -60.5t81 -103t47.5 -132.5t24 -138t5.5 -131zM1088 1024q0 -159 -112.5 -271.5t-271.5 -112.5t-271.5 112.5t-112.5 271.5t112.5 271.5t271.5 112.5 t271.5 -112.5t112.5 -271.5z" />
 <glyph unicode="&#xf0f1;" horiz-adv-x="1408" d="M1280 832q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 832q0 -62 -35.5 -111t-92.5 -70v-395q0 -159 -131.5 -271.5t-316.5 -112.5t-316.5 112.5t-131.5 271.5v132q-164 20 -274 128t-110 252v512q0 26 19 45t45 19q6 0 16 -2q17 30 47 48 t65 18q53 0 90.5 -37.5t37.5 -90.5t-37.5 -90.5t-90.5 -37.5q-33 0 -64 18v-402q0 -106 94 -181t226 -75t226 75t94 181v402q-31 -18 -64 -18q-53 0 -90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5q35 0 65 -18t47 -48q10 2 16 2q26 0 45 -19t19 -45v-512q0 -144 -110 -252 t-274 -128v-132q0 -106 94 -181t226 -75t226 75t94 181v395q-57 21 -92.5 70t-35.5 111q0 80 56 136t136 56t136 -56t56 -136z" />
 <glyph unicode="&#xf0f2;" horiz-adv-x="1792" d="M640 1152h512v128h-512v-128zM288 1152v-1280h-64q-92 0 -158 66t-66 158v832q0 92 66 158t158 66h64zM1408 1152v-1280h-1024v1280h128v160q0 40 28 68t68 28h576q40 0 68 -28t28 -68v-160h128zM1792 928v-832q0 -92 -66 -158t-158 -66h-64v1280h64q92 0 158 -66 t66 -158z" />
-<glyph unicode="&#xf0f3;" horiz-adv-x="1664" d="M848 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM1664 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q190 161 287 397.5t97 498.5 q0 165 96 262t264 117q-8 18 -8 37q0 40 28 68t68 28t68 -28t28 -68q0 -19 -8 -37q168 -20 264 -117t96 -262q0 -262 97 -498.5t287 -397.5z" />
+<glyph unicode="&#xf0f3;" horiz-adv-x="1792" d="M912 -160q0 16 -16 16q-59 0 -101.5 42.5t-42.5 101.5q0 16 -16 16t-16 -16q0 -73 51.5 -124.5t124.5 -51.5q16 0 16 16zM1728 128q0 -52 -38 -90t-90 -38h-448q0 -106 -75 -181t-181 -75t-181 75t-75 181h-448q-52 0 -90 38t-38 90q50 42 91 88t85 119.5t74.5 158.5 t50 206t19.5 260q0 152 117 282.5t307 158.5q-8 19 -8 39q0 40 28 68t68 28t68 -28t28 -68q0 -20 -8 -39q190 -28 307 -158.5t117 -282.5q0 -139 19.5 -260t50 -206t74.5 -158.5t85 -119.5t91 -88z" />
 <glyph unicode="&#xf0f4;" horiz-adv-x="1920" d="M1664 896q0 80 -56 136t-136 56h-64v-384h64q80 0 136 56t56 136zM0 128h1792q0 -106 -75 -181t-181 -75h-1280q-106 0 -181 75t-75 181zM1856 896q0 -159 -112.5 -271.5t-271.5 -112.5h-64v-32q0 -92 -66 -158t-158 -66h-704q-92 0 -158 66t-66 158v736q0 26 19 45 t45 19h1152q159 0 271.5 -112.5t112.5 -271.5z" />
 <glyph unicode="&#xf0f5;" horiz-adv-x="1408" d="M640 1472v-640q0 -61 -35.5 -111t-92.5 -70v-779q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v779q-57 20 -92.5 70t-35.5 111v640q0 26 19 45t45 19t45 -19t19 -45v-416q0 -26 19 -45t45 -19t45 19t19 45v416q0 26 19 45t45 19t45 -19t19 -45v-416q0 -26 19 -45 t45 -19t45 19t19 45v416q0 26 19 45t45 19t45 -19t19 -45zM1408 1472v-1600q0 -52 -38 -90t-90 -38h-128q-52 0 -90 38t-38 90v512h-224q-13 0 -22.5 9.5t-9.5 22.5v800q0 132 94 226t226 94h256q26 0 45 -19t19 -45z" />
-<glyph unicode="&#xf0f6;" horiz-adv-x="1280" d="M1024 352v-64q0 -14 -9 -23t-23 -9h-704q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h704q14 0 23 -9t9 -23zM1024 608v-64q0 -14 -9 -23t-23 -9h-704q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h704q14 0 23 -9t9 -23zM128 0h1024v768h-416q-40 0 -68 28t-28 68v416h-512v-1280z M768 896h376q-10 29 -22 41l-313 313q-12 12 -41 22v-376zM1280 864v-896q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h640q40 0 88 -20t76 -48l312 -312q28 -28 48 -76t20 -88z" />
+<glyph unicode="&#xf0f6;" d="M1468 1156q28 -28 48 -76t20 -88v-1152q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h896q40 0 88 -20t76 -48zM1024 1400v-376h376q-10 29 -22 41l-313 313q-12 12 -41 22zM1408 -128v1024h-416q-40 0 -68 28t-28 68v416h-768v-1536h1280z M384 736q0 14 9 23t23 9h704q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-14 0 -23 9t-9 23v64zM1120 512q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h704zM1120 256q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-704 q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h704z" />
 <glyph unicode="&#xf0f7;" horiz-adv-x="1408" d="M384 224v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M640 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M1152 224v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM896 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M640 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 992v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5
 v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M1152 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM896 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M640 992v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 1248v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M1152 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM896 992v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M640 1248v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1152 992v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9
 .5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M896 1248v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1152 1248v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M896 -128h384v1536h-1152v-1536h384v224q0 13 9.5 22.5t22.5 9.5h320q13 0 22.5 -9.5t9.5 -22.5v-224zM1408 1472v-1664q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v1664q0 26 19 45t45 19h1280q26 0 45 -19t19 -45z" />
 <glyph unicode="&#xf0f8;" horiz-adv-x="1408" d="M384 224v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M640 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM384 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M1152 224v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM896 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M640 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1152 480v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.
 5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M896 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5zM1152 736v-64q0 -13 -9.5 -22.5t-22.5 -9.5h-64q-13 0 -22.5 9.5t-9.5 22.5v64q0 13 9.5 22.5t22.5 9.5h64q13 0 22.5 -9.5t9.5 -22.5z M896 -128h384v1152h-256v-32q0 -40 -28 -68t-68 -28h-448q-40 0 -68 28t-28 68v32h-256v-1152h384v224q0 13 9.5 22.5t22.5 9.5h320q13 0 22.5 -9.5t9.5 -22.5v-224zM896 1056v320q0 13 -9.5 22.5t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-96h-128v96q0 13 -9.5 22.5 t-22.5 9.5h-64q-13 0 -22.5 -9.5t-9.5 -22.5v-320q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5v96h128v-96q0 -13 9.5 -22.5t22.5 -9.5h64q13 0 22.5 9.5t9.5 22.5zM1408 1088v-1280q0 -26 -19 -45t-45 -19h-1280q-26 0 -45 19t-19 45v1280q0 26 19 45t45 19h320 v288q0 40 28 68t68 28h448q40 0 68 -28t28 -68v-288h320q26 0 45 -19t19 -45z" />
 <glyph unicode="&#xf0f9;" horiz-adv-x="1920" d="M640 128q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5t90.5 37.5t37.5 90.5zM256 640h384v256h-158q-14 -2 -22 -9l-195 -195q-7 -12 -9 -22v-30zM1536 128q0 53 -37.5 90.5t-90.5 37.5t-90.5 -37.5t-37.5 -90.5t37.5 -90.5t90.5 -37.5 t90.5 37.5t37.5 90.5zM1664 800v192q0 14 -9 23t-23 9h-224v224q0 14 -9 23t-23 9h-192q-14 0 -23 -9t-9 -23v-224h-224q-14 0 -23 -9t-9 -23v-192q0 -14 9 -23t23 -9h224v-224q0 -14 9 -23t23 -9h192q14 0 23 9t9 23v224h224q14 0 23 9t9 23zM1920 1344v-1152 q0 -26 -19 -45t-45 -19h-192q0 -106 -75 -181t-181 -75t-181 75t-75 181h-384q0 -106 -75 -181t-181 -75t-181 75t-75 181h-128q-26 0 -45 19t-19 45t19 45t45 19v416q0 26 13 58t32 51l198 198q19 19 51 32t58 13h160v320q0 26 19 45t45 19h1152q26 0 45 -19t19 -45z" />
@@ -274,14 +275,14 @@
 <glyph unicode="&#xf10c;" d="M768 1184q-148 0 -273 -73t-198 -198t-73 -273t73 -273t198 -198t273 -73t273 73t198 198t73 273t-73 273t-198 198t-273 73zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103 t279.5 -279.5t103 -385.5z" />
 <glyph unicode="&#xf10d;" horiz-adv-x="1664" d="M768 576v-384q0 -80 -56 -136t-136 -56h-384q-80 0 -136 56t-56 136v704q0 104 40.5 198.5t109.5 163.5t163.5 109.5t198.5 40.5h64q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-64q-106 0 -181 -75t-75 -181v-32q0 -40 28 -68t68 -28h224q80 0 136 -56t56 -136z M1664 576v-384q0 -80 -56 -136t-136 -56h-384q-80 0 -136 56t-56 136v704q0 104 40.5 198.5t109.5 163.5t163.5 109.5t198.5 40.5h64q26 0 45 -19t19 -45v-128q0 -26 -19 -45t-45 -19h-64q-106 0 -181 -75t-75 -181v-32q0 -40 28 -68t68 -28h224q80 0 136 -56t56 -136z" />
 <glyph unicode="&#xf10e;" horiz-adv-x="1664" d="M768 1216v-704q0 -104 -40.5 -198.5t-109.5 -163.5t-163.5 -109.5t-198.5 -40.5h-64q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h64q106 0 181 75t75 181v32q0 40 -28 68t-68 28h-224q-80 0 -136 56t-56 136v384q0 80 56 136t136 56h384q80 0 136 -56t56 -136zM1664 1216 v-704q0 -104 -40.5 -198.5t-109.5 -163.5t-163.5 -109.5t-198.5 -40.5h-64q-26 0 -45 19t-19 45v128q0 26 19 45t45 19h64q106 0 181 75t75 181v32q0 40 -28 68t-68 28h-224q-80 0 -136 56t-56 136v384q0 80 56 136t136 56h384q80 0 136 -56t56 -136z" />
-<glyph unicode="&#xf110;" horiz-adv-x="1568" d="M496 192q0 -60 -42.5 -102t-101.5 -42q-60 0 -102 42t-42 102t42 102t102 42q59 0 101.5 -42t42.5 -102zM928 0q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM320 640q0 -66 -47 -113t-113 -47t-113 47t-47 113 t47 113t113 47t113 -47t47 -113zM1360 192q0 -46 -33 -79t-79 -33t-79 33t-33 79t33 79t79 33t79 -33t33 -79zM528 1088q0 -73 -51.5 -124.5t-124.5 -51.5t-124.5 51.5t-51.5 124.5t51.5 124.5t124.5 51.5t124.5 -51.5t51.5 -124.5zM992 1280q0 -80 -56 -136t-136 -56 t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1536 640q0 -40 -28 -68t-68 -28t-68 28t-28 68t28 68t68 28t68 -28t28 -68zM1328 1088q0 -33 -23.5 -56.5t-56.5 -23.5t-56.5 23.5t-23.5 56.5t23.5 56.5t56.5 23.5t56.5 -23.5t23.5 -56.5z" />
+<glyph unicode="&#xf110;" horiz-adv-x="1792" d="M526 142q0 -53 -37.5 -90.5t-90.5 -37.5q-52 0 -90 38t-38 90q0 53 37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1024 -64q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM320 640q0 -53 -37.5 -90.5t-90.5 -37.5 t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1522 142q0 -52 -38 -90t-90 -38q-53 0 -90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM558 1138q0 -66 -47 -113t-113 -47t-113 47t-47 113t47 113t113 47t113 -47t47 -113z M1728 640q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1088 1344q0 -80 -56 -136t-136 -56t-136 56t-56 136t56 136t136 56t136 -56t56 -136zM1618 1138q0 -93 -66 -158.5t-158 -65.5q-93 0 -158.5 65.5t-65.5 158.5 q0 92 65.5 158t158.5 66q92 0 158 -66t66 -158z" />
 <glyph unicode="&#xf111;" d="M1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
 <glyph unicode="&#xf112;" horiz-adv-x="1792" d="M1792 416q0 -166 -127 -451q-3 -7 -10.5 -24t-13.5 -30t-13 -22q-12 -17 -28 -17q-15 0 -23.5 10t-8.5 25q0 9 2.5 26.5t2.5 23.5q5 68 5 123q0 101 -17.5 181t-48.5 138.5t-80 101t-105.5 69.5t-133 42.5t-154 21.5t-175.5 6h-224v-256q0 -26 -19 -45t-45 -19t-45 19 l-512 512q-19 19 -19 45t19 45l512 512q19 19 45 19t45 -19t19 -45v-256h224q713 0 875 -403q53 -134 53 -333z" />
 <glyph unicode="&#xf113;" horiz-adv-x="1664" d="M640 320q0 -40 -12.5 -82t-43 -76t-72.5 -34t-72.5 34t-43 76t-12.5 82t12.5 82t43 76t72.5 34t72.5 -34t43 -76t12.5 -82zM1280 320q0 -40 -12.5 -82t-43 -76t-72.5 -34t-72.5 34t-43 76t-12.5 82t12.5 82t43 76t72.5 34t72.5 -34t43 -76t12.5 -82zM1440 320 q0 120 -69 204t-187 84q-41 0 -195 -21q-71 -11 -157 -11t-157 11q-152 21 -195 21q-118 0 -187 -84t-69 -204q0 -88 32 -153.5t81 -103t122 -60t140 -29.5t149 -7h168q82 0 149 7t140 29.5t122 60t81 103t32 153.5zM1664 496q0 -207 -61 -331q-38 -77 -105.5 -133t-141 -86 t-170 -47.5t-171.5 -22t-167 -4.5q-78 0 -142 3t-147.5 12.5t-152.5 30t-137 51.5t-121 81t-86 115q-62 123 -62 331q0 237 136 396q-27 82 -27 170q0 116 51 218q108 0 190 -39.5t189 -123.5q147 35 309 35q148 0 280 -32q105 82 187 121t189 39q51 -102 51 -218 q0 -87 -27 -168q136 -160 136 -398z" />
 <glyph unicode="&#xf114;" horiz-adv-x="1664" d="M1536 224v704q0 40 -28 68t-68 28h-704q-40 0 -68 28t-28 68v64q0 40 -28 68t-68 28h-320q-40 0 -68 -28t-28 -68v-960q0 -40 28 -68t68 -28h1216q40 0 68 28t28 68zM1664 928v-704q0 -92 -66 -158t-158 -66h-1216q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320 q92 0 158 -66t66 -158v-32h672q92 0 158 -66t66 -158z" />
 <glyph unicode="&#xf115;" horiz-adv-x="1920" d="M1781 605q0 35 -53 35h-1088q-40 0 -85.5 -21.5t-71.5 -52.5l-294 -363q-18 -24 -18 -40q0 -35 53 -35h1088q40 0 86 22t71 53l294 363q18 22 18 39zM640 768h768v160q0 40 -28 68t-68 28h-576q-40 0 -68 28t-28 68v64q0 40 -28 68t-68 28h-320q-40 0 -68 -28t-28 -68 v-853l256 315q44 53 116 87.5t140 34.5zM1909 605q0 -62 -46 -120l-295 -363q-43 -53 -116 -87.5t-140 -34.5h-1088q-92 0 -158 66t-66 158v960q0 92 66 158t158 66h320q92 0 158 -66t66 -158v-32h544q92 0 158 -66t66 -158v-160h192q54 0 99 -24.5t67 -70.5q15 -32 15 -68z " />
-<glyph unicode="&#xf116;" horiz-adv-x="1152" d="M896 608v-64q0 -14 -9 -23t-23 -9h-224v-224q0 -14 -9 -23t-23 -9h-64q-14 0 -23 9t-9 23v224h-224q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h224v224q0 14 9 23t23 9h64q14 0 23 -9t9 -23v-224h224q14 0 23 -9t9 -23zM1024 224v704q0 40 -28 68t-68 28h-704q-40 0 -68 -28 t-28 -68v-704q0 -40 28 -68t68 -28h704q40 0 68 28t28 68zM1152 928v-704q0 -92 -65.5 -158t-158.5 -66h-704q-93 0 -158.5 66t-65.5 158v704q0 93 65.5 158.5t158.5 65.5h704q93 0 158.5 -65.5t65.5 -158.5z" />
-<glyph unicode="&#xf117;" horiz-adv-x="1152" d="M928 1152q93 0 158.5 -65.5t65.5 -158.5v-704q0 -92 -65.5 -158t-158.5 -66h-704q-93 0 -158.5 66t-65.5 158v704q0 93 65.5 158.5t158.5 65.5h704zM1024 224v704q0 40 -28 68t-68 28h-704q-40 0 -68 -28t-28 -68v-704q0 -40 28 -68t68 -28h704q40 0 68 28t28 68z M864 640q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-576q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h576z" />
+<glyph unicode="&#xf116;" horiz-adv-x="1792" />
+<glyph unicode="&#xf117;" horiz-adv-x="1792" />
 <glyph unicode="&#xf118;" d="M1134 461q-37 -121 -138 -195t-228 -74t-228 74t-138 195q-8 25 4 48.5t38 31.5q25 8 48.5 -4t31.5 -38q25 -80 92.5 -129.5t151.5 -49.5t151.5 49.5t92.5 129.5q8 26 32 38t49 4t37 -31.5t4 -48.5zM640 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5 t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1152 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248.5 -51t-204 -136.5t-136.5 -204t-51 -248.5 t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 204t51 248.5zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
 <glyph unicode="&#xf119;" d="M1134 307q8 -25 -4 -48.5t-37 -31.5t-49 4t-32 38q-25 80 -92.5 129.5t-151.5 49.5t-151.5 -49.5t-92.5 -129.5q-8 -26 -31.5 -38t-48.5 -4q-26 8 -38 31.5t-4 48.5q37 121 138 195t228 74t228 -74t138 -195zM640 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5 t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1152 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248.5 -51t-204 -136.5t-136.5 -204 t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 204t51 248.5zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
 <glyph unicode="&#xf11a;" d="M1152 448q0 -26 -19 -45t-45 -19h-640q-26 0 -45 19t-19 45t19 45t45 19h640q26 0 45 -19t19 -45zM640 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1152 896q0 -53 -37.5 -90.5t-90.5 -37.5t-90.5 37.5 t-37.5 90.5t37.5 90.5t90.5 37.5t90.5 -37.5t37.5 -90.5zM1408 640q0 130 -51 248.5t-136.5 204t-204 136.5t-248.5 51t-248.5 -51t-204 -136.5t-136.5 -204t-51 -248.5t51 -248.5t136.5 -204t204 -136.5t248.5 -51t248.5 51t204 136.5t136.5 204t51 248.5zM1536 640 q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
@@ -310,7 +311,7 @@
 <glyph unicode="&#xf133;" horiz-adv-x="1664" d="M128 -128h1408v1024h-1408v-1024zM512 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1280 1088v288q0 14 -9 23t-23 9h-64q-14 0 -23 -9t-9 -23v-288q0 -14 9 -23t23 -9h64q14 0 23 9t9 23zM1664 1152v-1280 q0 -52 -38 -90t-90 -38h-1408q-52 0 -90 38t-38 90v1280q0 52 38 90t90 38h128v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h384v96q0 66 47 113t113 47h64q66 0 113 -47t47 -113v-96h128q52 0 90 -38t38 -90z" />
 <glyph unicode="&#xf134;" horiz-adv-x="1408" d="M512 1344q0 26 -19 45t-45 19t-45 -19t-19 -45t19 -45t45 -19t45 19t19 45zM1408 1376v-320q0 -16 -12 -25q-8 -7 -20 -7q-4 0 -7 1l-448 96q-11 2 -18 11t-7 20h-256v-102q111 -23 183.5 -111t72.5 -203v-800q0 -26 -19 -45t-45 -19h-512q-26 0 -45 19t-19 45v800 q0 106 62.5 190.5t161.5 114.5v111h-32q-59 0 -115 -23.5t-91.5 -53t-66 -66.5t-40.5 -53.5t-14 -24.5q-17 -35 -57 -35q-16 0 -29 7q-23 12 -31.5 37t3.5 49q5 10 14.5 26t37.5 53.5t60.5 70t85 67t108.5 52.5q-25 42 -25 86q0 66 47 113t113 47t113 -47t47 -113 q0 -33 -14 -64h302q0 11 7 20t18 11l448 96q3 1 7 1q12 0 20 -7q12 -9 12 -25z" />
 <glyph unicode="&#xf135;" horiz-adv-x="1664" d="M1440 1088q0 40 -28 68t-68 28t-68 -28t-28 -68t28 -68t68 -28t68 28t28 68zM1664 1376q0 -249 -75.5 -430.5t-253.5 -360.5q-81 -80 -195 -176l-20 -379q-2 -16 -16 -26l-384 -224q-7 -4 -16 -4q-12 0 -23 9l-64 64q-13 14 -8 32l85 276l-281 281l-276 -85q-3 -1 -9 -1 q-14 0 -23 9l-64 64q-17 19 -5 39l224 384q10 14 26 16l379 20q96 114 176 195q188 187 358 258t431 71q14 0 24 -9.5t10 -22.5z" />
-<glyph unicode="&#xf136;" horiz-adv-x="1792" d="M1708 881l-188 -881h-304l181 849q4 21 1 43q-4 20 -16 35q-10 14 -28 24q-18 9 -40 9h-197l-205 -960h-303l204 960h-304l-205 -960h-304l272 1280h1139q157 0 245 -118q86 -116 52 -281z" />
+<glyph unicode="&#xf136;" horiz-adv-x="1792" d="M1745 763l-164 -763h-334l178 832q13 56 -15 88q-27 33 -83 33h-169l-204 -953h-334l204 953h-286l-204 -953h-334l204 953l-153 327h1276q101 0 189.5 -40.5t147.5 -113.5q60 -73 81 -168.5t0 -194.5z" />
 <glyph unicode="&#xf137;" d="M909 141l102 102q19 19 19 45t-19 45l-307 307l307 307q19 19 19 45t-19 45l-102 102q-19 19 -45 19t-45 -19l-454 -454q-19 -19 -19 -45t19 -45l454 -454q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
 <glyph unicode="&#xf138;" d="M717 141l454 454q19 19 19 45t-19 45l-454 454q-19 19 -45 19t-45 -19l-102 -102q-19 -19 -19 -45t19 -45l307 -307l-307 -307q-19 -19 -19 -45t19 -45l102 -102q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
 <glyph unicode="&#xf139;" d="M1165 397l102 102q19 19 19 45t-19 45l-454 454q-19 19 -45 19t-45 -19l-454 -454q-19 -19 -19 -45t19 -45l102 -102q19 -19 45 -19t45 19l307 307l307 -307q19 -19 45 -19t45 19zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5 t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
@@ -342,11 +343,11 @@
 <glyph unicode="&#xf155;" horiz-adv-x="1024" d="M978 351q0 -153 -99.5 -263.5t-258.5 -136.5v-175q0 -14 -9 -23t-23 -9h-135q-13 0 -22.5 9.5t-9.5 22.5v175q-66 9 -127.5 31t-101.5 44.5t-74 48t-46.5 37.5t-17.5 18q-17 21 -2 41l103 135q7 10 23 12q15 2 24 -9l2 -2q113 -99 243 -125q37 -8 74 -8q81 0 142.5 43 t61.5 122q0 28 -15 53t-33.5 42t-58.5 37.5t-66 32t-80 32.5q-39 16 -61.5 25t-61.5 26.5t-62.5 31t-56.5 35.5t-53.5 42.5t-43.5 49t-35.5 58t-21 66.5t-8.5 78q0 138 98 242t255 134v180q0 13 9.5 22.5t22.5 9.5h135q14 0 23 -9t9 -23v-176q57 -6 110.5 -23t87 -33.5 t63.5 -37.5t39 -29t15 -14q17 -18 5 -38l-81 -146q-8 -15 -23 -16q-14 -3 -27 7q-3 3 -14.5 12t-39 26.5t-58.5 32t-74.5 26t-85.5 11.5q-95 0 -155 -43t-60 -111q0 -26 8.5 -48t29.5 -41.5t39.5 -33t56 -31t60.5 -27t70 -27.5q53 -20 81 -31.5t76 -35t75.5 -42.5t62 -50 t53 -63.5t31.5 -76.5t13 -94z" />
 <glyph unicode="&#xf156;" horiz-adv-x="898" d="M898 1066v-102q0 -14 -9 -23t-23 -9h-168q-23 -144 -129 -234t-276 -110q167 -178 459 -536q14 -16 4 -34q-8 -18 -29 -18h-195q-16 0 -25 12q-306 367 -498 571q-9 9 -9 22v127q0 13 9.5 22.5t22.5 9.5h112q132 0 212.5 43t102.5 125h-427q-14 0 -23 9t-9 23v102 q0 14 9 23t23 9h413q-57 113 -268 113h-145q-13 0 -22.5 9.5t-9.5 22.5v133q0 14 9 23t23 9h832q14 0 23 -9t9 -23v-102q0 -14 -9 -23t-23 -9h-233q47 -61 64 -144h171q14 0 23 -9t9 -23z" />
 <glyph unicode="&#xf157;" horiz-adv-x="1027" d="M603 0h-172q-13 0 -22.5 9t-9.5 23v330h-288q-13 0 -22.5 9t-9.5 23v103q0 13 9.5 22.5t22.5 9.5h288v85h-288q-13 0 -22.5 9t-9.5 23v104q0 13 9.5 22.5t22.5 9.5h214l-321 578q-8 16 0 32q10 16 28 16h194q19 0 29 -18l215 -425q19 -38 56 -125q10 24 30.5 68t27.5 61 l191 420q8 19 29 19h191q17 0 27 -16q9 -14 1 -31l-313 -579h215q13 0 22.5 -9.5t9.5 -22.5v-104q0 -14 -9.5 -23t-22.5 -9h-290v-85h290q13 0 22.5 -9.5t9.5 -22.5v-103q0 -14 -9.5 -23t-22.5 -9h-290v-330q0 -13 -9.5 -22.5t-22.5 -9.5z" />
-<glyph unicode="&#xf158;" horiz-adv-x="1664" d="M1664 352v-32q0 -132 -94 -226t-226 -94h-128q-132 0 -226 94t-94 226v480h-224q-2 -102 -14.5 -190.5t-30.5 -156t-48.5 -126.5t-57 -99.5t-67.5 -77.5t-69.5 -58.5t-74 -44t-69 -32t-65.5 -25.5q-4 -2 -32 -13q-8 -2 -12 -2q-22 0 -30 20l-71 178q-5 13 0 25t17 17 q7 3 20 7.5t18 6.5q31 12 46.5 18.5t44.5 20t45.5 26t42 32.5t40.5 42.5t34.5 53.5t30.5 68.5t22.5 83.5t17 103t6.5 123h-256q-14 0 -23 9t-9 23v160q0 14 9 23t23 9h1216q14 0 23 -9t9 -23v-160q0 -14 -9 -23t-23 -9h-224v-512q0 -26 19 -45t45 -19h128q26 0 45 19t19 45 v64q0 14 9 23t23 9h192q14 0 23 -9t9 -23zM1280 1376v-160q0 -14 -9 -23t-23 -9h-960q-14 0 -23 9t-9 23v160q0 14 9 23t23 9h960q14 0 23 -9t9 -23z" />
+<glyph unicode="&#xf158;" horiz-adv-x="1280" d="M1043 971q0 100 -65 162t-171 62h-320v-448h320q106 0 171 62t65 162zM1280 971q0 -193 -126.5 -315t-326.5 -122h-340v-118h505q14 0 23 -9t9 -23v-128q0 -14 -9 -23t-23 -9h-505v-192q0 -14 -9.5 -23t-22.5 -9h-167q-14 0 -23 9t-9 23v192h-224q-14 0 -23 9t-9 23v128 q0 14 9 23t23 9h224v118h-224q-14 0 -23 9t-9 23v149q0 13 9 22.5t23 9.5h224v629q0 14 9 23t23 9h539q200 0 326.5 -122t126.5 -315z" />
 <glyph unicode="&#xf159;" horiz-adv-x="1792" d="M514 341l81 299h-159l75 -300q1 -1 1 -3t1 -3q0 1 0.5 3.5t0.5 3.5zM630 768l35 128h-292l32 -128h225zM822 768h139l-35 128h-70zM1271 340l78 300h-162l81 -299q0 -1 0.5 -3.5t1.5 -3.5q0 1 0.5 3t0.5 3zM1382 768l33 128h-297l34 -128h230zM1792 736v-64q0 -14 -9 -23 t-23 -9h-213l-164 -616q-7 -24 -31 -24h-159q-24 0 -31 24l-166 616h-209l-167 -616q-7 -24 -31 -24h-159q-11 0 -19.5 7t-10.5 17l-160 616h-208q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h175l-33 128h-142q-14 0 -23 9t-9 23v64q0 14 9 23t23 9h109l-89 344q-5 15 5 28 q10 12 26 12h137q26 0 31 -24l90 -360h359l97 360q7 24 31 24h126q24 0 31 -24l98 -360h365l93 360q5 24 31 24h137q16 0 26 -12q10 -13 5 -28l-91 -344h111q14 0 23 -9t9 -23v-64q0 -14 -9 -23t-23 -9h-145l-34 -128h179q14 0 23 -9t9 -23z" />
 <glyph unicode="&#xf15a;" horiz-adv-x="1280" d="M1167 896q18 -182 -131 -258q117 -28 175 -103t45 -214q-7 -71 -32.5 -125t-64.5 -89t-97 -58.5t-121.5 -34.5t-145.5 -15v-255h-154v251q-80 0 -122 1v-252h-154v255q-18 0 -54 0.5t-55 0.5h-200l31 183h111q50 0 58 51v402h16q-6 1 -16 1v287q-13 68 -89 68h-111v164 l212 -1q64 0 97 1v252h154v-247q82 2 122 2v245h154v-252q79 -7 140 -22.5t113 -45t82.5 -78t36.5 -114.5zM952 351q0 36 -15 64t-37 46t-57.5 30.5t-65.5 18.5t-74 9t-69 3t-64.5 -1t-47.5 -1v-338q8 0 37 -0.5t48 -0.5t53 1.5t58.5 4t57 8.5t55.5 14t47.5 21t39.5 30 t24.5 40t9.5 51zM881 827q0 33 -12.5 58.5t-30.5 42t-48 28t-55 16.5t-61.5 8t-58 2.5t-54 -1t-39.5 -0.5v-307q5 0 34.5 -0.5t46.5 0t50 2t55 5.5t51.5 11t48.5 18.5t37 27t27 38.5t9 51z" />
-<glyph unicode="&#xf15b;" horiz-adv-x="1280" d="M1280 768v-800q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28t-28 68v1344q0 40 28 68t68 28h544v-544q0 -40 28 -68t68 -28h544zM1277 896h-509v509q82 -15 132 -65l312 -312q50 -50 65 -132z" />
-<glyph unicode="&#xf15c;" horiz-adv-x="1280" d="M1024 160v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704q14 0 23 9t9 23zM1024 416v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704q14 0 23 9t9 23zM1280 768v-800q0 -40 -28 -68t-68 -28h-1088q-40 0 -68 28 t-28 68v1344q0 40 28 68t68 28h544v-544q0 -40 28 -68t68 -28h544zM1277 896h-509v509q82 -15 132 -65l312 -312q50 -50 65 -132z" />
+<glyph unicode="&#xf15b;" d="M1024 1024v472q22 -14 36 -28l408 -408q14 -14 28 -36h-472zM896 992q0 -40 28 -68t68 -28h544v-1056q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h800v-544z" />
+<glyph unicode="&#xf15c;" d="M1468 1060q14 -14 28 -36h-472v472q22 -14 36 -28zM992 896h544v-1056q0 -40 -28 -68t-68 -28h-1344q-40 0 -68 28t-28 68v1600q0 40 28 68t68 28h800v-544q0 -40 28 -68t68 -28zM1152 160v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704 q14 0 23 9t9 23zM1152 416v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704q14 0 23 9t9 23zM1152 672v64q0 14 -9 23t-23 9h-704q-14 0 -23 -9t-9 -23v-64q0 -14 9 -23t23 -9h704q14 0 23 9t9 23z" />
 <glyph unicode="&#xf15d;" horiz-adv-x="1664" d="M1191 1128h177l-72 218l-12 47q-2 16 -2 20h-4l-3 -20q0 -1 -3.5 -18t-7.5 -29zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23zM1572 -23 v-233h-584v90l369 529q12 18 21 27l11 9v3q-2 0 -6.5 -0.5t-7.5 -0.5q-12 -3 -30 -3h-232v-115h-120v229h567v-89l-369 -530q-6 -8 -21 -26l-11 -11v-2l14 2q9 2 30 2h248v119h121zM1661 874v-106h-288v106h75l-47 144h-243l-47 -144h75v-106h-287v106h70l230 662h162 l230 -662h70z" />
 <glyph unicode="&#xf15e;" horiz-adv-x="1664" d="M1191 104h177l-72 218l-12 47q-2 16 -2 20h-4l-3 -20q0 -1 -3.5 -18t-7.5 -29zM736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23zM1661 -150 v-106h-288v106h75l-47 144h-243l-47 -144h75v-106h-287v106h70l230 662h162l230 -662h70zM1572 1001v-233h-584v90l369 529q12 18 21 27l11 9v3q-2 0 -6.5 -0.5t-7.5 -0.5q-12 -3 -30 -3h-232v-115h-120v229h567v-89l-369 -530q-6 -8 -21 -26l-11 -10v-3l14 3q9 1 30 1h248 v119h121z" />
 <glyph unicode="&#xf160;" horiz-adv-x="1792" d="M736 96q0 -12 -10 -24l-319 -319q-10 -9 -23 -9q-12 0 -23 9l-320 320q-15 16 -7 35q8 20 30 20h192v1376q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1376h192q14 0 23 -9t9 -23zM1792 -32v-192q0 -14 -9 -23t-23 -9h-832q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h832 q14 0 23 -9t9 -23zM1600 480v-192q0 -14 -9 -23t-23 -9h-640q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h640q14 0 23 -9t9 -23zM1408 992v-192q0 -14 -9 -23t-23 -9h-448q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h448q14 0 23 -9t9 -23zM1216 1504v-192q0 -14 -9 -23t-23 -9h-256 q-14 0 -23 9t-9 23v192q0 14 9 23t23 9h256q14 0 23 -9t9 -23z" />
@@ -361,14 +362,14 @@
 <glyph unicode="&#xf169;" d="M685 771q0 1 -126 222q-21 34 -52 34h-184q-18 0 -26 -11q-7 -12 1 -29l125 -216v-1l-196 -346q-9 -14 0 -28q8 -13 24 -13h185q31 0 50 36zM1309 1268q-7 12 -24 12h-187q-30 0 -49 -35l-411 -729q1 -2 262 -481q20 -35 52 -35h184q18 0 25 12q8 13 -1 28l-260 476v1 l409 723q8 16 0 28zM1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
 <glyph unicode="&#xf16a;" horiz-adv-x="1792" d="M1280 640q0 37 -30 54l-512 320q-31 20 -65 2q-33 -18 -33 -56v-640q0 -38 33 -56q16 -8 31 -8q20 0 34 10l512 320q30 17 30 54zM1792 640q0 -96 -1 -150t-8.5 -136.5t-22.5 -147.5q-16 -73 -69 -123t-124 -58q-222 -25 -671 -25t-671 25q-71 8 -124.5 58t-69.5 123 q-14 65 -21.5 147.5t-8.5 136.5t-1 150t1 150t8.5 136.5t22.5 147.5q16 73 69 123t124 58q222 25 671 25t671 -25q71 -8 124.5 -58t69.5 -123q14 -65 21.5 -147.5t8.5 -136.5t1 -150z" />
 <glyph unicode="&#xf16b;" horiz-adv-x="1792" d="M402 829l494 -305l-342 -285l-490 319zM1388 274v-108l-490 -293v-1l-1 1l-1 -1v1l-489 293v108l147 -96l342 284v2l1 -1l1 1v-2l343 -284zM554 1418l342 -285l-494 -304l-338 270zM1390 829l338 -271l-489 -319l-343 285zM1239 1418l489 -319l-338 -270l-494 304z" />
-<glyph unicode="&#xf16c;" horiz-adv-x="1408" d="M928 135v-151l-707 -1v151zM1169 481v-701l-1 -35v-1h-1132l-35 1h-1v736h121v-618h928v618h120zM241 393l704 -65l-13 -150l-705 65zM309 709l683 -183l-39 -146l-683 183zM472 1058l609 -360l-77 -130l-609 360zM832 1389l398 -585l-124 -85l-399 584zM1285 1536 l121 -697l-149 -26l-121 697z" />
-<glyph unicode="&#xf16d;" d="M1362 110v648h-135q20 -63 20 -131q0 -126 -64 -232.5t-174 -168.5t-240 -62q-197 0 -337 135.5t-140 327.5q0 68 20 131h-141v-648q0 -26 17.5 -43.5t43.5 -17.5h1069q25 0 43 17.5t18 43.5zM1078 643q0 124 -90.5 211.5t-218.5 87.5q-127 0 -217.5 -87.5t-90.5 -211.5 t90.5 -211.5t217.5 -87.5q128 0 218.5 87.5t90.5 211.5zM1362 1003v165q0 28 -20 48.5t-49 20.5h-174q-29 0 -49 -20.5t-20 -48.5v-165q0 -29 20 -49t49 -20h174q29 0 49 20t20 49zM1536 1211v-1142q0 -81 -58 -139t-139 -58h-1142q-81 0 -139 58t-58 139v1142q0 81 58 139 t139 58h1142q81 0 139 -58t58 -139z" />
+<glyph unicode="&#xf16c;" d="M1289 -96h-1118v480h-160v-640h1438v640h-160v-480zM347 428l33 157l783 -165l-33 -156zM450 802l67 146l725 -339l-67 -145zM651 1158l102 123l614 -513l-102 -123zM1048 1536l477 -641l-128 -96l-477 641zM330 65v159h800v-159h-800z" />
+<glyph unicode="&#xf16d;" d="M1024 640q0 106 -75 181t-181 75t-181 -75t-75 -181t75 -181t181 -75t181 75t75 181zM1162 640q0 -164 -115 -279t-279 -115t-279 115t-115 279t115 279t279 115t279 -115t115 -279zM1270 1050q0 -38 -27 -65t-65 -27t-65 27t-27 65t27 65t65 27t65 -27t27 -65zM768 1270 q-7 0 -76.5 0.5t-105.5 0t-96.5 -3t-103 -10t-71.5 -18.5q-50 -20 -88 -58t-58 -88q-11 -29 -18.5 -71.5t-10 -103t-3 -96.5t0 -105.5t0.5 -76.5t-0.5 -76.5t0 -105.5t3 -96.5t10 -103t18.5 -71.5q20 -50 58 -88t88 -58q29 -11 71.5 -18.5t103 -10t96.5 -3t105.5 0t76.5 0.5 t76.5 -0.5t105.5 0t96.5 3t103 10t71.5 18.5q50 20 88 58t58 88q11 29 18.5 71.5t10 103t3 96.5t0 105.5t-0.5 76.5t0.5 76.5t0 105.5t-3 96.5t-10 103t-18.5 71.5q-20 50 -58 88t-88 58q-29 11 -71.5 18.5t-103 10t-96.5 3t-105.5 0t-76.5 -0.5zM1536 640q0 -229 -5 -317 q-10 -208 -124 -322t-322 -124q-88 -5 -317 -5t-317 5q-208 10 -322 124t-124 322q-5 88 -5 317t5 317q10 208 124 322t322 124q88 5 317 5t317 -5q208 -10 322 -124t124 -322q5 -88 5 -317z" />
 <glyph unicode="&#xf16e;" d="M1248 1408q119 0 203.5 -84.5t84.5 -203.5v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960zM698 640q0 88 -62 150t-150 62t-150 -62t-62 -150t62 -150t150 -62t150 62t62 150zM1262 640q0 88 -62 150 t-150 62t-150 -62t-62 -150t62 -150t150 -62t150 62t62 150z" />
 <glyph unicode="&#xf170;" d="M768 914l201 -306h-402zM1133 384h94l-459 691l-459 -691h94l104 160h522zM1536 640q0 -209 -103 -385.5t-279.5 -279.5t-385.5 -103t-385.5 103t-279.5 279.5t-103 385.5t103 385.5t279.5 279.5t385.5 103t385.5 -103t279.5 -279.5t103 -385.5z" />
 <glyph unicode="&#xf171;" horiz-adv-x="1408" d="M815 677q8 -63 -50.5 -101t-111.5 -6q-39 17 -53.5 58t-0.5 82t52 58q36 18 72.5 12t64 -35.5t27.5 -67.5zM926 698q-14 107 -113 164t-197 13q-63 -28 -100.5 -88.5t-34.5 -129.5q4 -91 77.5 -155t165.5 -56q91 8 152 84t50 168zM1165 1240q-20 27 -56 44.5t-58 22 t-71 12.5q-291 47 -566 -2q-43 -7 -66 -12t-55 -22t-50 -43q30 -28 76 -45.5t73.5 -22t87.5 -11.5q228 -29 448 -1q63 8 89.5 12t72.5 21.5t75 46.5zM1222 205q-8 -26 -15.5 -76.5t-14 -84t-28.5 -70t-58 -56.5q-86 -48 -189.5 -71.5t-202 -22t-201.5 18.5q-46 8 -81.5 18 t-76.5 27t-73 43.5t-52 61.5q-25 96 -57 292l6 16l18 9q223 -148 506.5 -148t507.5 148q21 -6 24 -23t-5 -45t-8 -37zM1403 1166q-26 -167 -111 -655q-5 -30 -27 -56t-43.5 -40t-54.5 -31q-252 -126 -610 -88q-248 27 -394 139q-15 12 -25.5 26.5t-17 35t-9 34t-6 39.5 t-5.5 35q-9 50 -26.5 150t-28 161.5t-23.5 147.5t-22 158q3 26 17.5 48.5t31.5 37.5t45 30t46 22.5t48 18.5q125 46 313 64q379 37 676 -50q155 -46 215 -122q16 -20 16.5 -51t-5.5 -54z" />
 <glyph unicode="&#xf172;" d="M848 666q0 43 -41 66t-77 1q-43 -20 -42.5 -72.5t43.5 -70.5q39 -23 81 4t36 72zM928 682q8 -66 -36 -121t-110 -61t-119 40t-56 113q-2 49 25.5 93t72.5 64q70 31 141.5 -10t81.5 -118zM1100 1073q-20 -21 -53.5 -34t-53 -16t-63.5 -8q-155 -20 -324 0q-44 6 -63 9.5 t-52.5 16t-54.5 32.5q13 19 36 31t40 15.5t47 8.5q198 35 408 1q33 -5 51 -8.5t43 -16t39 -31.5zM1142 327q0 7 5.5 26.5t3 32t-17.5 16.5q-161 -106 -365 -106t-366 106l-12 -6l-5 -12q26 -154 41 -210q47 -81 204 -108q249 -46 428 53q34 19 49 51.5t22.5 85.5t12.5 71z M1272 1020q9 53 -8 75q-43 55 -155 88q-216 63 -487 36q-132 -12 -226 -46q-38 -15 -59.5 -25t-47 -34t-29.5 -54q8 -68 19 -138t29 -171t24 -137q1 -5 5 -31t7 -36t12 -27t22 -28q105 -80 284 -100q259 -28 440 63q24 13 39.5 23t31 29t19.5 40q48 267 80 473zM1536 1120 v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
-<glyph unicode="&#xf173;" horiz-adv-x="1024" d="M390 1408h219v-388h364v-241h-364v-394q0 -136 14 -172q13 -37 52 -60q50 -31 117 -31q117 0 232 76v-242q-102 -48 -178 -65q-77 -19 -173 -19q-105 0 -186 27q-78 25 -138 75q-58 51 -79 105q-22 54 -22 161v539h-170v217q91 30 155 84q64 55 103 132q39 78 54 196z " />
-<glyph unicode="&#xf174;" d="M1123 127v181q-88 -56 -174 -56q-51 0 -88 23q-29 17 -39 45q-11 30 -11 129v295h274v181h-274v291h-164q-11 -90 -40 -147t-78 -99q-48 -40 -116 -63v-163h127v-404q0 -78 17 -121q17 -42 59 -78q43 -37 104 -57q62 -20 140 -20q67 0 129 14q57 13 134 49zM1536 1120 v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
+<glyph unicode="&#xf173;" horiz-adv-x="1024" d="M944 207l80 -237q-23 -35 -111 -66t-177 -32q-104 -2 -190.5 26t-142.5 74t-95 106t-55.5 120t-16.5 118v544h-168v215q72 26 129 69.5t91 90t58 102t34 99t15 88.5q1 5 4.5 8.5t7.5 3.5h244v-424h333v-252h-334v-518q0 -30 6.5 -56t22.5 -52.5t49.5 -41.5t81.5 -14 q78 2 134 29z" />
+<glyph unicode="&#xf174;" d="M1136 75l-62 183q-44 -22 -103 -22q-36 -1 -62 10.5t-38.5 31.5t-17.5 40.5t-5 43.5v398h257v194h-256v326h-188q-8 0 -9 -10q-5 -44 -17.5 -87t-39 -95t-77 -95t-118.5 -68v-165h130v-418q0 -57 21.5 -115t65 -111t121 -85.5t176.5 -30.5q69 1 136.5 25t85.5 50z M1536 1120v-960q0 -119 -84.5 -203.5t-203.5 -84.5h-960q-119 0 -203.5 84.5t-84.5 203.5v960q0 119 84.5 203.5t203.5 84.5h960q119 0 203.5 -84.5t84.5 -203.5z" />
 <glyph unicode="&#xf175;" horiz-adv-x="768" d="M765 237q8 -19 -5 -35l-350 -384q-10 -10 -23 -10q-14 0 -24 10l-355 384q-13 16 -5 35q9 19 29 19h224v1248q0 14 9 23t23 9h192q14 0 23 -9t9 -23v-1248h224q21 0 29 -19z" />
 <glyph unicode="&#xf176;" 

<TRUNCATED>


[23/50] [abbrv] incubator-geode git commit: RemoveAll

Posted by ji...@apache.org.
RemoveAll


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

Branch: refs/heads/feature/GEODE-1571
Commit: 4ac886e65543ff251872a945fbc7ff2558d43eea
Parents: baea171
Author: Kevin J. Duling <kd...@pivotal.io>
Authored: Fri Jun 24 10:51:10 2016 -0700
Committer: Kevin J. Duling <kd...@pivotal.io>
Committed: Fri Jun 24 10:51:10 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/command/RemoveAll.java   |  5 +-
 ...ratedClientRemoveAllAuthDistributedTest.java | 56 ++++++++++++++++++++
 .../internal/security/clientServer.json         |  8 +++
 3 files changed, 67 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4ac886e6/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
index ac546fc..6ed5d2f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RemoveAll.java
@@ -23,7 +23,6 @@ package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Collection;
 
 import com.gemstone.gemfire.cache.DynamicRegionFactory;
 import com.gemstone.gemfire.cache.RegionDestroyedException;
@@ -50,6 +49,7 @@ import com.gemstone.gemfire.internal.cache.versions.VersionTag;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.internal.util.Breadcrumbs;
 
 public class RemoveAll extends BaseCommand {
@@ -210,7 +210,8 @@ public class RemoveAll extends BaseCommand {
         servConn.setRequestSpecificTimeout(timeout);
       }
 
-      
+      GeodeSecurityUtil.authorizeDataWrite();
+
       AuthorizeRequest authzRequest = servConn.getAuthzRequest();
       if (authzRequest != null) {
         // TODO SW: This is to handle DynamicRegionFactory create

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4ac886e6/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
new file mode 100644
index 0000000..527972c
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.security;
+
+import static org.junit.Assert.assertFalse;
+
+import java.util.Arrays;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+@Category(DistributedTest.class)
+public class IntegratedClientRemoveAllAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
+
+  @Test
+  public void testRemoveAll() throws InterruptedException {
+
+    AsyncInvocation ai1 = client1.invokeAsync(() -> {
+      Cache cache = SecurityTestUtils.createCacheClient("dataReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      assertNotAuthorized(() -> region.removeAll(Arrays.asList("key1", "key2", "key3", "key4")), "DATA:WRITE");
+    });
+
+    AsyncInvocation ai2 = client2.invokeAsync(() -> {
+      Cache cache = SecurityTestUtils.createCacheClient("dataUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      region.removeAll(Arrays.asList("key1", "key2", "key3", "key4"));
+      assertFalse(region.containsKey("key1"));
+      assertFalse(region.containsKeyOnServer("key1"));
+    });
+    ai1.join();
+    ai2.join();
+    ai1.checkException();
+    ai2.checkException();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/4ac886e6/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
index c659709..8dedd78 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
@@ -136,6 +136,14 @@
       ]
     },
     {
+      "name": "dataUser",
+      "password": "1234567",
+      "roles": [
+        "data-read",
+        "data-write"
+      ]
+    },
+    {
       "name":"authRegionUser",
       "password": "1234567",
       "roles": [


[34/50] [abbrv] incubator-geode git commit: GEODE-1576: Website needs to be upgraded to include WAN and CQ

Posted by ji...@apache.org.
GEODE-1576: Website needs to be upgraded to include WAN and CQ

fixed icons in the header


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

Branch: refs/heads/feature/GEODE-1571
Commit: 7ad9cc9451212f1c8a0acba3ec78a9eb562d3780
Parents: 95eff48
Author: Swapnil Bawaskar <sb...@pivotal.io>
Authored: Fri Jun 24 16:02:56 2016 -0700
Committer: Swapnil Bawaskar <sb...@pivotal.io>
Committed: Fri Jun 24 16:05:36 2016 -0700

----------------------------------------------------------------------
 geode-site/website/content/css/geode-site.css | 41 ++++++++++------------
 geode-site/website/layouts/header.html        | 18 +++++-----
 2 files changed, 27 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ad9cc94/geode-site/website/content/css/geode-site.css
----------------------------------------------------------------------
diff --git a/geode-site/website/content/css/geode-site.css b/geode-site/website/content/css/geode-site.css
index 136a757..bfb9ba9 100644
--- a/geode-site/website/content/css/geode-site.css
+++ b/geode-site/website/content/css/geode-site.css
@@ -1291,17 +1291,17 @@ h2.icns-tools:before{
 /* Phone and below */
 @media screen and (max-width: 768px) {
   #home-logo { opacity: 1; height: 25px;}
-  span.icon-book:after {
+  span.fa-book:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Docs";
   }
-  span.icon-group:after {
+  span.fa-users:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Community";
   }
-  span.icon-github-sign:after {
+  span.fa-github-square:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Github";
@@ -1311,12 +1311,12 @@ h2.icns-tools:before{
     font-weight: 600;
     content: " Trello";
   }
-  span.icon-bug:after {
+  span.fa-bug:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " JIRA";
   }
-  span.icon-stackexchange:after {
+  span.fa-stack-overflow:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " StackOverflow";
@@ -1326,17 +1326,17 @@ h2.icns-tools:before{
     font-weight: 600;
     content: " HipChat";
   }
-  span.icon-twitter:after {
+  span.fa-twitter:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Twitter";
   }
-  span.icon-edit:after {
+  span.fa-pencil-square-o:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: "  Wiki";
   }
-  span.icon-releases:after {
+  span.fa-arrow-circle-o-down:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Releases";
@@ -1346,16 +1346,11 @@ h2.icns-tools:before{
     font-weight: 600;
     content: " Chat";
   }
-  span.icon-envelope:after {
+  span.fa-envelope:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Mailing Lists";
   }
-  span.icon-cloud-download:after {
-    font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
-    font-weight: 600;
-    content: " Releases";
-  }
   span.icon-flag-checkered:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
@@ -1495,17 +1490,17 @@ h2.icns-tools:before{
 
 /* Tablets/desktops and up */
 @media screen and (min-width: 1200px) {
-  span.icon-book:after {
+  span.fa-book:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Docs";
   }
-  span.icon-group:after {
+  span.fa-users:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Community";
   }
-  span.icon-github-sign:after {
+  span.fa-github-square:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Github";
@@ -1515,12 +1510,12 @@ h2.icns-tools:before{
     font-weight: 600;
     content: " Trello";
   }
-  span.icon-bug:after {
+  span.fa-bug:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " JIRA";
   }
-  span.icon-stackexchange:after {
+  span.fa-stack-overflow:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " StackOverflow";
@@ -1530,17 +1525,17 @@ h2.icns-tools:before{
     font-weight: 600;
     content: " HipChat";
   }
-  span.icon-twitter:after {
+  span.fa-twitter:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Twitter";
   }
-  span.icon-edit:after {
+  span.fa-pencil-square-o:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: "  Wiki";
   }
-  span.icon-releases:after {
+  span.fa-arrow-circle-o-down:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Releases";
@@ -1550,7 +1545,7 @@ h2.icns-tools:before{
     font-weight: 600;
     content: " Chat";
   }
-  span.icon-envelope:after {
+  span.fa-envelope:after {
     font-family: 'klavika-web', 'Open Sans', 'Helvetica Neue', Helvetica, Arial, sans-serif;
     font-weight: 600;
     content: " Mailing Lists";

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/7ad9cc94/geode-site/website/layouts/header.html
----------------------------------------------------------------------
diff --git a/geode-site/website/layouts/header.html b/geode-site/website/layouts/header.html
index b5a4ca7..843c689 100644
--- a/geode-site/website/layouts/header.html
+++ b/geode-site/website/layouts/header.html
@@ -245,18 +245,18 @@
             </a>
         <nav class="collapse navbar-collapse bf-navbar-collapse" role="navigation">
             <ul class="nav navbar-nav navbar-right">
-                <li class="<%= 'active' if @community %>"><a href="/community/"><span class="icns icon-group"></span></a></li>
-                <li class="<%= 'active' if @docs %>"><a href="/docs/"><span class="icns icon-book"></span></a></li>
-                <li><a href="http://github.com/apache/incubator-geode" target="_blank"><span class="icns icon-github-sign"></span></a></li>
+                <li class="<%= 'active' if @community %>"><a href="/community/"><span class="fa fa-users"></span></a></li>
+                <li class="<%= 'active' if @docs %>"><a href="/docs/"><span class="fa fa-book"></span></a></li>
+                <li><a href="http://github.com/apache/incubator-geode" target="_blank"><span class="fa fa-github-square"></span></a></li>
                 <!--<li><a href="https://trello.com/b/exQmJIOn/usergrid" target="_blank"><span class="icns icon-trello"></span></a></li>-->
                 <li><a href="https://issues.apache.org/jira/browse/GEODE/"
-                       target="_blank"><span class="icns icon-bug"></span></a></li>
-                <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank"><span class="icns icon-stackexchange"></span></a></li>
-                <li><a href="/community/#mailing-lists"><span class="icns icon-envelope"></span></a></li>
+                       target="_blank"><span class="fa fa-bug"></span></a></li>
+                <li><a href="http://stackoverflow.com/search?q=Apache%20Geode" target="_blank"><span class="fa fa-stack-overflow"></span></a></li>
+                <li><a href="/community/#mailing-lists"><span class="fa fa-envelope"></span></a></li>
                 <!-- <li><a href="/community/#live"><span class="icns icon-comments"></span></a></li> -->
-                <li><a href="https://twitter.com/apachegeode" target="_blank"><span class="icns icon-twitter"></span></a></li>
-                <li><a href="https://cwiki.apache.org/confluence/display/geode/" target="_blank"><span class="icns icon-edit"></span></a></li>
-                <li><a href="/releases/"><span class="icns icon-releases"></span></a></li>
+                <li><a href="https://twitter.com/apachegeode" target="_blank"><span class="fa fa-twitter"></span></a></li>
+                <li><a href="https://cwiki.apache.org/confluence/display/geode/" target="_blank"><span class="fa fa-pencil-square-o"></span></a></li>
+                <li><a href="/releases/"><span class="fa fa-arrow-circle-o-down"></span></a></li>
             </ul>
         </nav>
     </div>


[13/50] [abbrv] incubator-geode git commit: Merge branch 'develop' into feature/GEODE-1571

Posted by ji...@apache.org.
Merge branch 'develop' into feature/GEODE-1571


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

Branch: refs/heads/feature/GEODE-1571
Commit: a4c2dada36a07f29d3acbece4c149e7e96867562
Parents: e9d9144 5882693
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Wed Jun 22 11:06:43 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Wed Jun 22 11:06:43 2016 -0700

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/Statistics.java   |  32 +--
 .../gemfire/internal/StatSamplerStats.java      |   2 +-
 .../management/internal/cli/GfshParser.java     |   4 +-
 .../cli/functions/DataCommandFunction.java      |   9 +-
 .../cli/functions/RegionCreateFunction.java     |   5 +-
 .../cli/parser/jopt/JoptOptionParser.java       |   4 +-
 .../internal/cli/remote/CommandProcessor.java   |   1 -
 .../internal/cli/util/HyphenFormatter.java      |  92 +++++++++
 .../internal/cli/util/OptionJFormatter.java     | 115 -----------
 .../internal/cli/GfshParserIntegrationTest.java |  38 ++++
 .../commands/GemfireDataCommandsDUnitTest.java  | 106 +++++++---
 .../internal/cli/util/HyphenFormatterTest.java  | 201 +++++++++++++++++++
 .../internal/cli/util/OptionJFormatterTest.java | 189 -----------------
 .../gemfire/cache/lucene/LuceneQueriesBase.java |  36 +++-
 .../lucene/LuceneQueriesClientDUnitTest.java    |  70 +++++++
 .../src/main/resources/gemfire.properties       |   2 +-
 geode-pulse/src/main/webapp/DataBrowser.html    |   2 +-
 geode-pulse/src/main/webapp/MemberDetails.html  |   2 +-
 .../src/main/webapp/QueryStatistics.html        |   2 +-
 geode-pulse/src/main/webapp/clusterDetail.html  |   2 +-
 .../main/webapp/properties/gemfire.properties   |   2 +-
 .../webapp/properties/gemfire_en.properties     |   2 +-
 geode-pulse/src/main/webapp/regionDetail.html   |   2 +-
 .../cli/commands/CommandOverHttpDUnitTest.java  |   2 +-
 24 files changed, 541 insertions(+), 381 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/a4c2dada/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
----------------------------------------------------------------------


[03/50] [abbrv] incubator-geode git commit: GEODE-117 - gfsh put ignores --skip-if-exists flag

Posted by ji...@apache.org.
GEODE-117 - gfsh put ignores --skip-if-exists flag

Fixed test, which erroneously reported that the --skip-if-exists flag is honored.
Implemented check for flag within the put operation.

This closes #166


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

Branch: refs/heads/feature/GEODE-1571
Commit: 91f92a49726be20dd3000738f77b3b383518adc2
Parents: 83c7c66
Author: Kevin J. Duling <kd...@pivotal.io>
Authored: Fri Jun 17 11:07:49 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Tue Jun 21 09:45:20 2016 -0700

----------------------------------------------------------------------
 .../cli/functions/DataCommandFunction.java      |  9 ++-
 .../commands/GemfireDataCommandsDUnitTest.java  | 66 +++++++++++++-------
 2 files changed, 51 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/91f92a49/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
index 48cc83a..927aea8 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/functions/DataCommandFunction.java
@@ -609,9 +609,12 @@ public class DataCommandFunction extends FunctionAdapter implements  InternalEnt
         valueObject = getClassObject(value,valueClass);
       }catch(ClassNotFoundException e){
         return DataCommandResult.createPutResult(key, null, null, "ClassNotFoundException " + valueClass, false); 
-      }      
-      
-      Object returnValue = region.put(keyObject,valueObject);
+      }
+      Object returnValue;
+      if (putIfAbsent && region.containsKey(keyObject))
+        returnValue = region.get(keyObject);
+      else
+        returnValue = region.put(keyObject,valueObject);
       Object array[] = getJSONForNonPrimitiveObject(returnValue);             
       DataCommandResult result = DataCommandResult.createPutResult(key, array[1], null, null, true);
       if(array[0]!=null)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/91f92a49/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
index 553b7a4..68fe0ae 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
@@ -16,7 +16,32 @@
  */
 package com.gemstone.gemfire.management.internal.cli.commands;
 
-import com.gemstone.gemfire.cache.*;
+import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
+import static com.gemstone.gemfire.test.dunit.Assert.*;
+import static com.gemstone.gemfire.test.dunit.IgnoredException.*;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
+import com.gemstone.gemfire.cache.PartitionAttributes;
+import com.gemstone.gemfire.cache.PartitionAttributesFactory;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
 import com.gemstone.gemfire.cache.query.QueryInvalidException;
 import com.gemstone.gemfire.cache.query.data.Portfolio;
 import com.gemstone.gemfire.cache.query.internal.CompiledValue;
@@ -32,7 +57,11 @@ import com.gemstone.gemfire.management.cli.Result;
 import com.gemstone.gemfire.management.internal.cli.CliUtil;
 import com.gemstone.gemfire.management.internal.cli.HeadlessGfsh;
 import com.gemstone.gemfire.management.internal.cli.domain.DataCommandRequest;
-import com.gemstone.gemfire.management.internal.cli.dto.*;
+import com.gemstone.gemfire.management.internal.cli.dto.Car;
+import com.gemstone.gemfire.management.internal.cli.dto.Key1;
+import com.gemstone.gemfire.management.internal.cli.dto.ObjectWithCharAttr;
+import com.gemstone.gemfire.management.internal.cli.dto.Value1;
+import com.gemstone.gemfire.management.internal.cli.dto.Value2;
 import com.gemstone.gemfire.management.internal.cli.i18n.CliStrings;
 import com.gemstone.gemfire.management.internal.cli.json.GfJsonArray;
 import com.gemstone.gemfire.management.internal.cli.json.GfJsonException;
@@ -42,22 +71,18 @@ import com.gemstone.gemfire.management.internal.cli.result.CompositeResultData.S
 import com.gemstone.gemfire.management.internal.cli.result.ResultData;
 import com.gemstone.gemfire.management.internal.cli.result.TabularResultData;
 import com.gemstone.gemfire.management.internal.cli.util.CommandStringBuilder;
-import com.gemstone.gemfire.test.dunit.*;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.IgnoredException;
+import com.gemstone.gemfire.test.dunit.SerializableCallable;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
+
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.File;
-import java.io.IOException;
-import java.util.*;
-
-import static com.gemstone.gemfire.test.dunit.Assert.*;
-import static com.gemstone.gemfire.test.dunit.IgnoredException.addIgnoredException;
-import static com.gemstone.gemfire.test.dunit.LogWriterUtils.getLogWriter;
-import static com.gemstone.gemfire.test.dunit.Wait.waitForCriterion;
-import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
-
 /**
  * Dunit class for testing gemfire data commands : get, put, remove, select, rebalance
  */
@@ -815,7 +840,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     final VM vm1 = Host.getHost(0).getVM(1);
     final VM vm2 = Host.getHost(0).getVM(2);
 
-
+    // Seed the region with some keys
     SerializableRunnable putKeys = new SerializableRunnable() {
       @Override
       public void run() {
@@ -833,6 +858,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     };
     vm1.invoke(putKeys);
 
+    // Now try to replace all existing keys with new values to test --skip-if-exists.  Values should not be replaced if the key is present.
     for (int i = 0; i < COUNT; i++) {
       String command = "put";
       String key = keyPrefix + i;
@@ -844,20 +870,18 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
       validateResult(cmdResult, true);
     }
 
+    // Verify that none of the values were replaced
     SerializableRunnable checkPutIfAbsentKeys = new SerializableRunnable() {
       @Override
       public void run() {
         Cache cache = getCache();
         Region region = cache.getRegion(DATA_REGION_NAME_PATH);
         assertNotNull(region);
-        for (int i = COUNT + 1; i < COUNT; i++) {
+        for (int i = 0; i < COUNT; i++) {
           String key = keyPrefix + i;
-          String notExpectedvalue = valuePrefix + i + i;
-          String expectedvalue = valuePrefix + i;
-          String value = (String) region.get(key);
-          assertNotNull(value);
-          assertEquals(value, expectedvalue);
-          if (value.equals(notExpectedvalue)) fail("Value is overriden even if put-If-absent was true");
+          String expected = valuePrefix + i;
+          String actual = (String) region.get(key);
+          assertEquals("--skip-if-exists=true failed to preserve value", expected, actual);
         }
       }
     };


[20/50] [abbrv] incubator-geode git commit: Add missing return before security checks

Posted by ji...@apache.org.
Add missing return before security checks


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

Branch: refs/heads/feature/GEODE-1571
Commit: 6dc3e506929856a1309dbc4f1c28a2dcb6ec4217
Parents: 2c09032
Author: gmeilen <gr...@gmail.com>
Authored: Thu Jun 23 15:50:49 2016 -0700
Committer: gmeilen <gr...@gmail.com>
Committed: Thu Jun 23 15:50:49 2016 -0700

----------------------------------------------------------------------
 .../sockets/command/RegisterInterest61.java     |   1 +
 .../sockets/command/RegisterInterestList.java   |   1 +
 .../sockets/command/RegisterInterestList61.java | 158 ++++++++++---------
 .../sockets/command/UnregisterInterest.java     |   1 +
 4 files changed, 83 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6dc3e506/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
index a797a6c..e6c946b 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
@@ -180,6 +180,7 @@ public class RegisterInterest61 extends BaseCommand {
       writeChunkedErrorResponse(msg, MessageType.REGISTER_INTEREST_DATA_ERROR,
           message.toLocalizedString(), servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
 
     if(interestType == InterestType.REGULAR_EXPRESSION)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6dc3e506/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList.java
index 71b29fe..bdc6f67 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList.java
@@ -161,6 +161,7 @@ public class RegisterInterestList extends BaseCommand {
       writeChunkedErrorResponse(msg, MessageType.REGISTER_INTEREST_DATA_ERROR,
           s, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
 
     GeodeSecurityUtil.authorizeRegionRead(regionName);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6dc3e506/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList61.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList61.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList61.java
index 102e168..5931ebb 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList61.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList61.java
@@ -32,6 +32,7 @@ import com.gemstone.gemfire.cache.DynamicRegionFactory;
 import com.gemstone.gemfire.cache.InterestResultPolicy;
 import com.gemstone.gemfire.cache.operations.RegisterInterestOperationContext;
 import com.gemstone.gemfire.i18n.StringId;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -171,94 +172,95 @@ public class RegisterInterestList61 extends BaseCommand {
       writeChunkedErrorResponse(msg, MessageType.REGISTER_INTEREST_DATA_ERROR,
           s, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else { // key not null
 
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        logger.info(LocalizedMessage.create(LocalizedStrings.RegisterInterestList_0_REGION_NAMED_1_WAS_NOT_FOUND_DURING_REGISTER_INTEREST_LIST_REQUEST, new Object[]{servConn.getName(), regionName}));
-        // writeChunkedErrorResponse(msg,
-        // MessageType.REGISTER_INTEREST_DATA_ERROR, message);
-        // responded = true;
-      } // else { // region not null
-      try {
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        if (authzRequest != null) {
-          // TODO SW: This is a workaround for DynamicRegionFactory
-          // registerInterest calls. Remove this when the semantics of
-          // DynamicRegionFactory are cleaned up.
-          if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-            RegisterInterestOperationContext registerContext = authzRequest
-                .registerInterestListAuthorize(regionName, keys, policy);
-            keys = (List)registerContext.getKey();
-          }
+    GeodeSecurityUtil.authorizeRegionRead(regionName);
+
+    LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
+    if (region == null) {
+      logger.info(LocalizedMessage.create(LocalizedStrings.RegisterInterestList_0_REGION_NAMED_1_WAS_NOT_FOUND_DURING_REGISTER_INTEREST_LIST_REQUEST, new Object[]{servConn.getName(), regionName}));
+      // writeChunkedErrorResponse(msg,
+      // MessageType.REGISTER_INTEREST_DATA_ERROR, message);
+      // responded = true;
+    } // else { // region not null
+    try {
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      if (authzRequest != null) {
+        // TODO SW: This is a workaround for DynamicRegionFactory
+        // registerInterest calls. Remove this when the semantics of
+        // DynamicRegionFactory are cleaned up.
+        if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
+          RegisterInterestOperationContext registerContext = authzRequest
+              .registerInterestListAuthorize(regionName, keys, policy);
+          keys = (List)registerContext.getKey();
         }
-        // Register interest
-        servConn.getAcceptor().getCacheClientNotifier()
-            .registerClientInterest(regionName, keys, servConn.getProxyID(),
-                isDurable, sendUpdatesAsInvalidates,
-                true, regionDataPolicyPartBytes[0], true);
-      }
-      catch (Exception ex) {
-        // If an interrupted exception is thrown , rethrow it
-        checkForInterrupt(servConn, ex);
-        // Otherwise, write an exception message and continue
-        writeChunkedException(msg, ex, false, servConn);
-        servConn.setAsTrue(RESPONDED);
-        return;
       }
+      // Register interest
+      servConn.getAcceptor().getCacheClientNotifier()
+          .registerClientInterest(regionName, keys, servConn.getProxyID(),
+              isDurable, sendUpdatesAsInvalidates,
+              true, regionDataPolicyPartBytes[0], true);
+    }
+    catch (Exception ex) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, ex);
+      // Otherwise, write an exception message and continue
+      writeChunkedException(msg, ex, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
 
-      // Update the statistics and write the reply
-      // bserverStats.incLong(processDestroyTimeId,
-      // DistributionStats.getStatTime() - start);
-      // start = DistributionStats.getStatTime();
+    // Update the statistics and write the reply
+    // bserverStats.incLong(processDestroyTimeId,
+    // DistributionStats.getStatTime() - start);
+    // start = DistributionStats.getStatTime();
 
-      boolean isPrimary = servConn.getAcceptor().getCacheClientNotifier()
-          .getClientProxy(servConn.getProxyID()).isPrimary();
-      if (!isPrimary) {
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_SECONDARY);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
-        chunkedResponseMsg.setLastChunk(true);
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Sending register interest response chunk from secondary for region: {} for key: {} chunk=<{}>", servConn.getName(), regionName, key, chunkedResponseMsg);
-        }
-        chunkedResponseMsg.sendChunk(servConn);
+    boolean isPrimary = servConn.getAcceptor().getCacheClientNotifier()
+        .getClientProxy(servConn.getProxyID()).isPrimary();
+    if (!isPrimary) {
+      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_SECONDARY);
+      chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+      chunkedResponseMsg.sendHeader();
+      chunkedResponseMsg.setLastChunk(true);
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: Sending register interest response chunk from secondary for region: {} for key: {} chunk=<{}>", servConn.getName(), regionName, key, chunkedResponseMsg);
       }
-      else { // isPrimary
-        // Send header which describes how many chunks will follow
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_PRIMARY);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
+      chunkedResponseMsg.sendChunk(servConn);
+    }
+    else { // isPrimary
+      // Send header which describes how many chunks will follow
+      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_PRIMARY);
+      chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+      chunkedResponseMsg.sendHeader();
 
-        // Send chunk response
-        try {
-          fillAndSendRegisterInterestResponseChunks(region, keys,
-              InterestType.KEY, policy, servConn);
-          servConn.setAsTrue(RESPONDED);
-        }
-        catch (Exception e) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, e);
+      // Send chunk response
+      try {
+        fillAndSendRegisterInterestResponseChunks(region, keys,
+            InterestType.KEY, policy, servConn);
+        servConn.setAsTrue(RESPONDED);
+      }
+      catch (Exception e) {
+        // If an interrupted exception is thrown , rethrow it
+        checkForInterrupt(servConn, e);
 
-          // otherwise send the exception back to client
-          writeChunkedException(msg, e, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
+        // otherwise send the exception back to client
+        writeChunkedException(msg, e, false, servConn);
+        servConn.setAsTrue(RESPONDED);
+        return;
+      }
 
-        if (logger.isDebugEnabled()) {
-          // logger.debug(getName() + ": Sent chunk (1 of 1) of register interest
-          // response (" + chunkedResponseMsg.getBufferLength() + " bytes) for
-          // region " + regionName + " key " + key);
-          logger.debug("{}: Sent register interest response for the following {} keys in region {}: {}", servConn.getName(), numberOfKeys, regionName, keys);
-        }
-        // bserverStats.incLong(writeDestroyResponseTimeId,
-        // DistributionStats.getStatTime() - start);
-        // bserverStats.incInt(destroyResponsesId, 1);
-      } // isPrimary
-      // } // region not null
-    } // key not null
+      if (logger.isDebugEnabled()) {
+        // logger.debug(getName() + ": Sent chunk (1 of 1) of register interest
+        // response (" + chunkedResponseMsg.getBufferLength() + " bytes) for
+        // region " + regionName + " key " + key);
+        logger.debug("{}: Sent register interest response for the following {} keys in region {}: {}", servConn.getName(), numberOfKeys, regionName, keys);
+      }
+      // bserverStats.incLong(writeDestroyResponseTimeId,
+      // DistributionStats.getStatTime() - start);
+      // bserverStats.incInt(destroyResponsesId, 1);
+    } // isPrimary
+    // } // region not null
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6dc3e506/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
index 9161d6d..eeaf286 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/UnregisterInterest.java
@@ -96,6 +96,7 @@ public class UnregisterInterest extends BaseCommand {
       writeErrorResponse(msg, MessageType.UNREGISTER_INTEREST_DATA_ERROR,
           s, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
 
     GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());


[31/50] [abbrv] incubator-geode git commit: GEODE-11: Added findResults() to LuceneQuery

Posted by ji...@apache.org.
GEODE-11: Added findResults() to LuceneQuery

Added findResults() to LuceneQuery which returns a list of LuceneResultStruct objects


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

Branch: refs/heads/feature/GEODE-1571
Commit: 2c4e17fbbf57daa123d623700489831e519d273c
Parents: 0010156
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Mon Jun 20 15:03:29 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri Jun 24 14:41:42 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneQuery.java        |  5 +++++
 .../cache/lucene/internal/LuceneQueryImpl.java   | 19 +++++++++++--------
 .../internal/LuceneQueryImplJUnitTest.java       |  9 +++++++++
 3 files changed, 25 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2c4e17fb/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
index 255bd79..7cc0977 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
@@ -17,6 +17,7 @@
 package com.gemstone.gemfire.cache.lucene;
 
 import java.util.Collection;
+import java.util.List;
 
 import com.gemstone.gemfire.annotations.Experimental;
 
@@ -39,6 +40,10 @@ public interface LuceneQuery<K, V> {
   public Collection<V> findValues() throws LuceneQueryException;
 
   /**
+   * Execute search and return list of LuceneResultStruct.
+   */
+  public List<LuceneResultStruct<K, V>> findResults() throws LuceneQueryException;
+  /**
    * Execute the search and get results. 
    */
   public PageableLuceneQueryResults<K, V> findPages() throws LuceneQueryException;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2c4e17fb/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
index 9de16dc..3125193 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
@@ -77,15 +77,22 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
 
   @Override
   public Collection<V> findValues() throws LuceneQueryException {
+    final List<LuceneResultStruct<K, V>> page = findResults();
+
+    return page.stream()
+      .map(entry -> entry.getValue())
+      .collect(Collectors.toList());
+  }
+
+  @Override
+  public List<LuceneResultStruct<K, V>> findResults() throws LuceneQueryException {
     PageableLuceneQueryResults<K, V> pages = findPages(0);
     final List<LuceneResultStruct<K, V>> page = pages.getNextPage();
+
     if(page == null) {
       return Collections.emptyList();
     }
-
-    return page.stream()
-      .map(entry -> entry.getValue())
-      .collect(Collectors.toList());
+    return page;
   }
 
   @Override
@@ -94,14 +101,10 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
   }
 
   private PageableLuceneQueryResults<K, V> findPages(int pageSize) throws LuceneQueryException {
-
     TopEntries<K> entries = findTopEntries();
-
     return new PageableLuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
   }
 
-
-
   private TopEntries<K> findTopEntries() throws LuceneQueryException {
     TopEntriesCollectorManager manager = new TopEntriesCollectorManager(null, limit);
     LuceneFunctionContext<TopEntriesCollector> context = new LuceneFunctionContext<>(query, indexName, manager, limit);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2c4e17fb/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
index 7b39e89..95371e4 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
@@ -57,6 +57,7 @@ import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntries;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollectorManager;
 import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector;
+import com.gemstone.gemfire.internal.cache.ValidatingDiskRegion;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
@@ -129,6 +130,14 @@ public class LuceneQueryImplJUnitTest {
   }
 
   @Test
+  public void shouldReturnLuceneResultStructFromFindResults() throws LuceneQueryException {
+    addValueToResults();
+    List<LuceneResultStruct<String,String>> result=new ArrayList<>();
+    result.add(new LuceneResultStructImpl("hi","value",5));
+    assertEquals(result,query.findResults());
+  }
+
+  @Test
   public void shouldInvokeLuceneFunctionWithCorrectArguments() throws Exception {
     addValueToResults();
     PageableLuceneQueryResults<Object, Object> results = query.findPages();


[17/50] [abbrv] incubator-geode git commit: Integrated client auth support for RegisterInterest, new dunit and refactoring of old dunit.

Posted by ji...@apache.org.
Integrated client auth support for RegisterInterest, new dunit and refactoring of old dunit.


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

Branch: refs/heads/feature/GEODE-1571
Commit: 46017f938f291140cdf43c02bb01580767510ca4
Parents: a4c2dad
Author: gmeilen <gr...@gmail.com>
Authored: Thu Jun 23 15:08:40 2016 -0700
Committer: gmeilen <gr...@gmail.com>
Committed: Thu Jun 23 15:08:40 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/BaseCommand.java         |  10 +
 .../tier/sockets/command/RegisterInterest.java  | 175 +++++++++---------
 .../sockets/command/RegisterInterest61.java     | 182 ++++++++++---------
 .../sockets/command/RegisterInterestList.java   |   6 +-
 .../sockets/command/RegisterInterestList66.java |   5 +-
 ...ractIntegratedClientAuthDistributedTest.java |  71 ++++++++
 .../security/IntegratedClientAuthDUnitTest.java |  55 +-----
 ...ientRegisterInterestAuthDistributedTest.java | 134 ++++++++++++++
 8 files changed, 413 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46017f93/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
index bdebb6b..b5bc033 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
@@ -41,8 +41,11 @@ import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.offheap.OffHeapHelper;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.internal.sequencelog.EntryLogger;
 import com.gemstone.gemfire.security.GemFireSecurityException;
+import com.gemstone.gemfire.security.NotAuthorizedException;
+
 import org.apache.logging.log4j.Logger;
 
 import java.io.*;
@@ -1218,6 +1221,13 @@ public abstract class BaseCommand implements Command {
           }
         }
 
+//        try { // TODO:SECURITY: Swapnil: do we need to throw or continue?
+//          GeodeSecurityUtil.authorizeRegionRead(region.getName(), key.toString());
+//        }
+//        catch (NotAuthorizedException e){
+//          continue;
+//        }
+
         ClientProxyMembershipID id = servConn == null ? null : servConn.getProxyID();
         data = region.get(key, null, true, true, true, id, versionHolder, true);
         versionTag = versionHolder.getVersionTag();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46017f93/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest.java
index 6defab0..3cf4bf4 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest.java
@@ -31,6 +31,7 @@ import com.gemstone.gemfire.cache.DynamicRegionFactory;
 import com.gemstone.gemfire.cache.InterestResultPolicy;
 import com.gemstone.gemfire.cache.operations.RegisterInterestOperationContext;
 import com.gemstone.gemfire.i18n.StringId;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 import java.io.IOException;
 
@@ -145,101 +146,105 @@ public class RegisterInterest extends BaseCommand {
       writeChunkedErrorResponse(msg, MessageType.REGISTER_INTEREST_DATA_ERROR,
           message.toLocalizedString(), servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else { // input key not null
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        logger.info(LocalizedMessage.create(LocalizedStrings.RegisterInterest_0_REGION_NAMED_1_WAS_NOT_FOUND_DURING_REGISTER_INTEREST_REQUEST, new Object[] {servConn.getName(), regionName}));
-        // writeChunkedErrorResponse(msg,
-        // MessageType.REGISTER_INTEREST_DATA_ERROR, message);
-        // responded = true;
-      }
-      // Register interest
-      try {
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        if (authzRequest != null) {
-          // TODO SW: This is a workaround for DynamicRegionFactory
-          // registerInterest calls. Remove this when the semantics of
-          // DynamicRegionFactory are cleaned up.
-          if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-            RegisterInterestOperationContext registerContext = authzRequest
-                .registerInterestAuthorize(regionName, key, interestType,
-                    policy);
-            key = registerContext.getKey();
-          }
+
+    // TODO: add security changes here
+    GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+
+    // input key not null
+    LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
+    if (region == null) {
+      logger.info(LocalizedMessage.create(LocalizedStrings.RegisterInterest_0_REGION_NAMED_1_WAS_NOT_FOUND_DURING_REGISTER_INTEREST_REQUEST, new Object[] {servConn.getName(), regionName}));
+      // writeChunkedErrorResponse(msg,
+      // MessageType.REGISTER_INTEREST_DATA_ERROR, message);
+      // responded = true;
+    }
+    // Register interest
+    try {
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      if (authzRequest != null) {
+        // TODO SW: This is a workaround for DynamicRegionFactory
+        // registerInterest calls. Remove this when the semantics of
+        // DynamicRegionFactory are cleaned up.
+        if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
+          RegisterInterestOperationContext registerContext = authzRequest
+              .registerInterestAuthorize(regionName, key, interestType,
+                  policy);
+          key = registerContext.getKey();
         }
-        servConn.getAcceptor().getCacheClientNotifier().registerClientInterest(
-            regionName, key, servConn.getProxyID(), interestType, isDurable,
-            sendUpdatesAsInvalidates, false, 0, true);
       }
-      catch (Exception e) {
-        // If an interrupted exception is thrown , rethrow it
-        checkForInterrupt(servConn, e);
-        // Otherwise, write an exception message and continue
-        writeChunkedException(msg, e, false, servConn);
-        servConn.setAsTrue(RESPONDED);
-        return;
+      servConn.getAcceptor().getCacheClientNotifier().registerClientInterest(
+          regionName, key, servConn.getProxyID(), interestType, isDurable,
+          sendUpdatesAsInvalidates, false, 0, true);
+    }
+    catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
+      // Otherwise, write an exception message and continue
+      writeChunkedException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    // System.out.println("Received register interest for " + regionName);
+
+    // Update the statistics and write the reply
+    // bserverStats.incLong(processDestroyTimeId,
+    // DistributionStats.getStatTime() - start);
+    // start = DistributionStats.getStatTime();
+
+    CacheClientProxy ccp = servConn.getAcceptor().getCacheClientNotifier()
+        .getClientProxy(servConn.getProxyID());
+    if (ccp == null) {
+      // fix for 37593
+      IOException ioex = new IOException(LocalizedStrings.RegisterInterest_CACHECLIENTPROXY_FOR_THIS_CLIENT_IS_NO_LONGER_ON_THE_SERVER_SO_REGISTERINTEREST_OPERATION_IS_UNSUCCESSFUL.toLocalizedString());
+      writeChunkedException(msg, ioex, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+    boolean isPrimary = ccp.isPrimary();
+    ChunkedMessage chunkedResponseMsg = servConn.getRegisterInterestResponseMessage();
+    if (!isPrimary) {
+      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_SECONDARY);
+      chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+      chunkedResponseMsg.sendHeader();
+      chunkedResponseMsg.setLastChunk(true);
+
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: Sending register interest response chunk from secondary for region: {} for key: {} chunk=<{}>", servConn.getName(), regionName, key, chunkedResponseMsg);
       }
+      chunkedResponseMsg.sendChunk(servConn);
+    } // !isPrimary
+    else { // isPrimary
 
-      // System.out.println("Received register interest for " + regionName);
+      // Send header which describes how many chunks will follow
+      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_PRIMARY);
+      chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+      chunkedResponseMsg.sendHeader();
 
-      // Update the statistics and write the reply
-      // bserverStats.incLong(processDestroyTimeId,
-      // DistributionStats.getStatTime() - start);
-      // start = DistributionStats.getStatTime();
-
-      CacheClientProxy ccp = servConn.getAcceptor().getCacheClientNotifier()
-          .getClientProxy(servConn.getProxyID());
-      if (ccp == null) {
-        // fix for 37593
-        IOException ioex = new IOException(LocalizedStrings.RegisterInterest_CACHECLIENTPROXY_FOR_THIS_CLIENT_IS_NO_LONGER_ON_THE_SERVER_SO_REGISTERINTEREST_OPERATION_IS_UNSUCCESSFUL.toLocalizedString());
-        writeChunkedException(msg, ioex, false, servConn);
+      // Send chunk response
+      try {
+        fillAndSendRegisterInterestResponseChunks(region, key, interestType,
+            policy, servConn);
+        servConn.setAsTrue(RESPONDED);
+      }
+      catch (Exception e) {
+        writeChunkedException(msg, e, false, servConn, chunkedResponseMsg);
         servConn.setAsTrue(RESPONDED);
         return;
       }
-      boolean isPrimary = ccp.isPrimary();
-      ChunkedMessage chunkedResponseMsg = servConn.getRegisterInterestResponseMessage();
-      if (!isPrimary) {
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_SECONDARY);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
-        chunkedResponseMsg.setLastChunk(true);
-
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Sending register interest response chunk from secondary for region: {} for key: {} chunk=<{}>", servConn.getName(), regionName, key, chunkedResponseMsg);
-        }
-        chunkedResponseMsg.sendChunk(servConn);
-      } // !isPrimary
-      else { // isPrimary
-
-        // Send header which describes how many chunks will follow
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_PRIMARY);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
-
-        // Send chunk response
-        try {
-          fillAndSendRegisterInterestResponseChunks(region, key, interestType,
-              policy, servConn);
-          servConn.setAsTrue(RESPONDED);
-        }
-        catch (Exception e) {
-          writeChunkedException(msg, e, false, servConn, chunkedResponseMsg);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
 
-        if (logger.isDebugEnabled()) {
-          // logger.debug(getName() + ": Sent chunk (1 of 1) of register interest
-          // response (" + chunkedResponseMsg.getBufferLength() + " bytes) for
-          // region " + regionName + " key " + key);
-          logger.debug("{}: Sent register interest response for region {} key {}", servConn.getName(), regionName, key);
-        }
-        // bserverStats.incLong(writeDestroyResponseTimeId,
-        // DistributionStats.getStatTime() - start);
-        // bserverStats.incInt(destroyResponsesId, 1);
-      } // isPrimary
-    } // input key not null
+      if (logger.isDebugEnabled()) {
+        // logger.debug(getName() + ": Sent chunk (1 of 1) of register interest
+        // response (" + chunkedResponseMsg.getBufferLength() + " bytes) for
+        // region " + regionName + " key " + key);
+        logger.debug("{}: Sent register interest response for region {} key {}", servConn.getName(), regionName, key);
+      }
+      // bserverStats.incLong(writeDestroyResponseTimeId,
+      // DistributionStats.getStatTime() - start);
+      // bserverStats.incInt(destroyResponsesId, 1);
+    } // isPrimary
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46017f93/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
index 5b4109c..a797a6c 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterest61.java
@@ -23,6 +23,7 @@ import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
+import com.gemstone.gemfire.internal.cache.tier.InterestType;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
 import com.gemstone.gemfire.internal.cache.tier.sockets.*;
 import com.gemstone.gemfire.internal.cache.vmotion.VMotionObserver;
@@ -34,6 +35,7 @@ import com.gemstone.gemfire.cache.DynamicRegionFactory;
 import com.gemstone.gemfire.cache.InterestResultPolicy;
 import com.gemstone.gemfire.cache.operations.RegisterInterestOperationContext;
 import com.gemstone.gemfire.i18n.StringId;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 import java.io.IOException;
 /**
@@ -179,103 +181,109 @@ public class RegisterInterest61 extends BaseCommand {
           message.toLocalizedString(), servConn);
       servConn.setAsTrue(RESPONDED);
     }
-    else { // input key not null
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        logger.info(LocalizedMessage.create(LocalizedStrings.RegisterInterest_0_REGION_NAMED_1_WAS_NOT_FOUND_DURING_REGISTER_INTEREST_REQUEST, new Object[] {servConn.getName(), regionName}));
-        // writeChunkedErrorResponse(msg,
-        // MessageType.REGISTER_INTEREST_DATA_ERROR, message);
-        // responded = true;
-      }
-      // Register interest
-      try {
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        if (authzRequest != null) {
-          // TODO SW: This is a workaround for DynamicRegionFactory
-          // registerInterest calls. Remove this when the semantics of
-          // DynamicRegionFactory are cleaned up.
-          if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
-            RegisterInterestOperationContext registerContext = authzRequest
-                .registerInterestAuthorize(regionName, key, interestType,
-                    policy);
-            key = registerContext.getKey();
-          }
+
+    if(interestType == InterestType.REGULAR_EXPRESSION)
+      GeodeSecurityUtil.authorizeRegionRead(regionName);
+    else
+      GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+
+    // input key not null
+    LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
+    if (region == null) {
+      logger.info(LocalizedMessage.create(LocalizedStrings.RegisterInterest_0_REGION_NAMED_1_WAS_NOT_FOUND_DURING_REGISTER_INTEREST_REQUEST, new Object[] {servConn.getName(), regionName}));
+      // writeChunkedErrorResponse(msg,
+      // MessageType.REGISTER_INTEREST_DATA_ERROR, message);
+      // responded = true;
+    }
+    // Register interest
+    try {
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      if (authzRequest != null) {
+        // TODO SW: This is a workaround for DynamicRegionFactory
+        // registerInterest calls. Remove this when the semantics of
+        // DynamicRegionFactory are cleaned up.
+        if (!DynamicRegionFactory.regionIsDynamicRegionList(regionName)) {
+          RegisterInterestOperationContext registerContext = authzRequest
+              .registerInterestAuthorize(regionName, key, interestType,
+                  policy);
+          key = registerContext.getKey();
         }
-        servConn.getAcceptor().getCacheClientNotifier()
-            .registerClientInterest(regionName, key, servConn.getProxyID(),
-                interestType, isDurable, sendUpdatesAsInvalidates, true,
-                regionDataPolicyPartBytes[0], true);
-      }
-      catch (Exception e) {
-        // If an interrupted exception is thrown , rethrow it
-        checkForInterrupt(servConn, e);
-        // Otherwise, write an exception message and continue
-        writeChunkedException(msg, e, false, servConn);
-        servConn.setAsTrue(RESPONDED);
-        return;
       }
+      servConn.getAcceptor().getCacheClientNotifier()
+          .registerClientInterest(regionName, key, servConn.getProxyID(),
+              interestType, isDurable, sendUpdatesAsInvalidates, true,
+              regionDataPolicyPartBytes[0], true);
+    }
+    catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
+      // Otherwise, write an exception message and continue
+      writeChunkedException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
 
-      // System.out.println("Received register interest for " + regionName);
+    // System.out.println("Received register interest for " + regionName);
 
-      // Update the statistics and write the reply
-      // bserverStats.incLong(processDestroyTimeId,
-      // DistributionStats.getStatTime() - start);
-      // start = DistributionStats.getStatTime();
+    // Update the statistics and write the reply
+    // bserverStats.incLong(processDestroyTimeId,
+    // DistributionStats.getStatTime() - start);
+    // start = DistributionStats.getStatTime();
 
-      CacheClientProxy ccp = servConn.getAcceptor().getCacheClientNotifier()
-          .getClientProxy(servConn.getProxyID());
-      if (ccp == null) {
-        // fix for 37593
-        IOException ioex = new IOException(
-            LocalizedStrings.RegisterInterest_CACHECLIENTPROXY_FOR_THIS_CLIENT_IS_NO_LONGER_ON_THE_SERVER_SO_REGISTERINTEREST_OPERATION_IS_UNSUCCESSFUL
-                .toLocalizedString());
-        writeChunkedException(msg, ioex, false, servConn);
-        servConn.setAsTrue(RESPONDED);
-        return;
+    CacheClientProxy ccp = servConn.getAcceptor().getCacheClientNotifier()
+        .getClientProxy(servConn.getProxyID());
+    if (ccp == null) {
+      // fix for 37593
+      IOException ioex = new IOException(
+          LocalizedStrings.RegisterInterest_CACHECLIENTPROXY_FOR_THIS_CLIENT_IS_NO_LONGER_ON_THE_SERVER_SO_REGISTERINTEREST_OPERATION_IS_UNSUCCESSFUL
+              .toLocalizedString());
+      writeChunkedException(msg, ioex, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+    boolean isPrimary = ccp.isPrimary();
+    ChunkedMessage chunkedResponseMsg = servConn.getRegisterInterestResponseMessage();
+    if (!isPrimary) {
+      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_SECONDARY);
+      chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+      chunkedResponseMsg.sendHeader();
+      chunkedResponseMsg.setLastChunk(true);
+
+      if (logger.isDebugEnabled()) {
+        logger.debug("{}: Sending register interest response chunk from secondary for region: {} for key: {} chunk=<{}>", servConn.getName(), regionName, key, chunkedResponseMsg);
       }
-      boolean isPrimary = ccp.isPrimary();
-      ChunkedMessage chunkedResponseMsg = servConn.getRegisterInterestResponseMessage();
-      if (!isPrimary) {
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_SECONDARY);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
-        chunkedResponseMsg.setLastChunk(true);
+      chunkedResponseMsg.sendChunk(servConn);
+    } // !isPrimary
+    else { // isPrimary
 
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Sending register interest response chunk from secondary for region: {} for key: {} chunk=<{}>", servConn.getName(), regionName, key, chunkedResponseMsg);
-        }
-        chunkedResponseMsg.sendChunk(servConn);
-      } // !isPrimary
-      else { // isPrimary
+      // Send header which describes how many chunks will follow
+      chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_PRIMARY);
+      chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+      chunkedResponseMsg.sendHeader();
 
-        // Send header which describes how many chunks will follow
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE_FROM_PRIMARY);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
+      // Send chunk response
+      try {
+        fillAndSendRegisterInterestResponseChunks(region, key, interestType,
+            serializeValues, policy, servConn);
+        servConn.setAsTrue(RESPONDED);
+      }
+      catch (Exception e) {
+        writeChunkedException(msg, e, false, servConn, chunkedResponseMsg);
+        servConn.setAsTrue(RESPONDED);
+        return;
+      }
 
-        // Send chunk response
-        try {
-          fillAndSendRegisterInterestResponseChunks(region, key, interestType,
-              serializeValues, policy, servConn);
-          servConn.setAsTrue(RESPONDED);
-        }
-        catch (Exception e) {
-          writeChunkedException(msg, e, false, servConn, chunkedResponseMsg);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
+      if (logger.isDebugEnabled()) {
+        // logger.debug(getName() + ": Sent chunk (1 of 1) of register interest
+        // response (" + chunkedResponseMsg.getBufferLength() + " bytes) for
+        // region " + regionName + " key " + key);
+        logger.debug("{}: Sent register interest response for region {} key {}", servConn.getName(), regionName, key);
+      }
+      // bserverStats.incLong(writeDestroyResponseTimeId,
+      // DistributionStats.getStatTime() - start);
+      // bserverStats.incInt(destroyResponsesId, 1);
+    } // isPrimary
 
-        if (logger.isDebugEnabled()) {
-          // logger.debug(getName() + ": Sent chunk (1 of 1) of register interest
-          // response (" + chunkedResponseMsg.getBufferLength() + " bytes) for
-          // region " + regionName + " key " + key);
-          logger.debug("{}: Sent register interest response for region {} key {}", servConn.getName(), regionName, key);
-        }
-        // bserverStats.incLong(writeDestroyResponseTimeId,
-        // DistributionStats.getStatTime() - start);
-        // bserverStats.incInt(destroyResponsesId, 1);
-      } // isPrimary
-    } // input key not null
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46017f93/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList.java
index c43c28e..71b29fe 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList.java
@@ -32,6 +32,7 @@ import com.gemstone.gemfire.cache.DynamicRegionFactory;
 import com.gemstone.gemfire.cache.InterestResultPolicy;
 import com.gemstone.gemfire.cache.operations.RegisterInterestOperationContext;
 import com.gemstone.gemfire.i18n.StringId;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -161,8 +162,10 @@ public class RegisterInterestList extends BaseCommand {
           s, servConn);
       servConn.setAsTrue(RESPONDED);
     }
-    else { // key not null
 
+    GeodeSecurityUtil.authorizeRegionRead(regionName);
+
+    // key not null
       LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
       if (region == null) {
         logger.info(LocalizedMessage.create(LocalizedStrings.RegisterInterestList_0_REGION_NAMED_1_WAS_NOT_FOUND_DURING_REGISTER_INTEREST_LIST_REQUEST, new Object[]{servConn.getName(), regionName}));
@@ -246,7 +249,6 @@ public class RegisterInterestList extends BaseCommand {
         // bserverStats.incInt(destroyResponsesId, 1);
       } // isPrimary
       // } // region not null
-    } // key not null
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46017f93/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList66.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList66.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList66.java
index 862e58b..264f1fb 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList66.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/RegisterInterestList66.java
@@ -33,6 +33,7 @@ import com.gemstone.gemfire.cache.DynamicRegionFactory;
 import com.gemstone.gemfire.cache.InterestResultPolicy;
 import com.gemstone.gemfire.cache.operations.RegisterInterestOperationContext;
 import com.gemstone.gemfire.i18n.StringId;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 import java.io.IOException;
 import java.util.List;
@@ -174,8 +175,9 @@ public class RegisterInterestList66 extends BaseCommand {
           s, servConn);
       servConn.setAsTrue(RESPONDED);
     }
-    else { // key not null
+    GeodeSecurityUtil.authorizeRegionRead(regionName);
 
+   // key not null
       LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
       if (region == null) {
         logger.info(LocalizedMessage.create(LocalizedStrings.RegisterInterestList_0_REGION_NAMED_1_WAS_NOT_FOUND_DURING_REGISTER_INTEREST_LIST_REQUEST, new Object[]{servConn.getName(), regionName}));
@@ -260,7 +262,6 @@ public class RegisterInterestList66 extends BaseCommand {
         // bserverStats.incInt(destroyResponsesId, 1);
       } // isPrimary
       // } // region not null
-    } // key not null
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46017f93/geode-core/src/test/java/com/gemstone/gemfire/security/AbstractIntegratedClientAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/AbstractIntegratedClientAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/AbstractIntegratedClientAuthDistributedTest.java
new file mode 100644
index 0000000..4a2db57
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/AbstractIntegratedClientAuthDistributedTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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.security;
+
+import static com.gemstone.gemfire.security.SecurityTestUtils.*;
+import static org.assertj.core.api.Assertions.*;
+import static org.junit.Assert.*;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.management.internal.security.JSONAuthorization;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+
+import org.assertj.core.api.ThrowableAssert.ThrowingCallable;
+import org.junit.After;
+import org.junit.Before;
+
+public class AbstractIntegratedClientAuthDistributedTest extends JUnit4DistributedTestCase {
+
+  protected VM client1 = null;
+  protected VM client2 = null;
+  protected VM client3 = null;
+  protected int serverPort;
+
+  @Before
+  public void before() throws Exception{
+    final Host host = Host.getHost(0);
+    client1 = host.getVM(1);
+    client2 = host.getVM(2);
+    client3 = host.getVM(3);
+
+    JSONAuthorization.setUpWithJsonFile("clientServer.json");
+    serverPort =  SecurityTestUtils.createCacheServer(JSONAuthorization.class.getName()+".create");
+    Region region = getCache().getRegion(SecurityTestUtils.REGION_NAME);
+    assertEquals(0, region.size());
+    for (int i = 0; i < 5; i++) {
+      String key = "key" + i;
+      String value = "value" + i;
+      region.put(key, value);
+    }
+    assertEquals(5, region.size());
+  }
+
+  @After
+  public void after(){
+    client1.invoke(() -> closeCache());
+    client2.invoke(() -> closeCache());
+    client3.invoke(() -> closeCache());
+    closeCache();
+  }
+
+  public static void assertNotAuthorized(ThrowingCallable shouldRaiseThrowable, String permString) {
+    assertThatThrownBy(shouldRaiseThrowable).hasMessageContaining(permString);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46017f93/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
index 212c9c9..506a171 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
@@ -17,8 +17,6 @@
 
 package com.gemstone.gemfire.security;
 
-import static com.gemstone.gemfire.security.SecurityTestUtils.*;
-import static org.assertj.core.api.Assertions.*;
 import static org.junit.Assert.*;
 
 import java.util.ArrayList;
@@ -27,55 +25,17 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.assertj.core.api.ThrowableAssert.ThrowingCallable;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.management.internal.security.JSONAuthorization;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
-import com.gemstone.gemfire.test.dunit.Host;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
-import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
-@Category(DistributedTest.class)
-public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
-  private VM client1 = null;
-  private VM client2 = null;
-  private VM client3 = null;
-  private int serverPort;
-
-  @Before
-  public void before() throws Exception{
-    final Host host = Host.getHost(0);
-    client1 = host.getVM(1);
-    client2 = host.getVM(2);
-    client3 = host.getVM(3);
-
-    JSONAuthorization.setUpWithJsonFile("clientServer.json");
-    serverPort =  SecurityTestUtils.createCacheServer(JSONAuthorization.class.getName()+".create");
-    Region region = getCache().getRegion(SecurityTestUtils.REGION_NAME);
-    assertEquals(0, region.size());
-    for (int i = 0; i < 5; i++) {
-      String key = "key" + i;
-      String value = "value" + i;
-      region.put(key, value);
-    }
-    assertEquals(5, region.size());
-  }
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
 
-  @After
-  public void after(){
-    client1.invoke(() -> closeCache());
-    client2.invoke(() -> closeCache());
-    client3.invoke(() -> closeCache());
-    closeCache();
-  }
+@Category(DistributedTest.class)
+public class IntegratedClientAuthDUnitTest extends AbstractIntegratedClientAuthDistributedTest {
 
   @Test
   public void testAuthentication(){
@@ -218,10 +178,6 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
     ai2.checkException();
   }
 
-  public static void assertNotAuthorized(ThrowingCallable shouldRaiseThrowable, String permString) {
-    assertThatThrownBy(shouldRaiseThrowable).hasMessageContaining(permString);
-  }
-
   @Test
   public void testRegionClear() throws InterruptedException {
     // Verify that an unauthorized user can't clear the region
@@ -246,5 +202,6 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
     };
     client2.invoke(clearAuthorized);
   }
-
 }
+
+

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/46017f93/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegisterInterestAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegisterInterestAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegisterInterestAuthDistributedTest.java
new file mode 100644
index 0000000..d533a20
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegisterInterestAuthDistributedTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.security;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(DistributedTest.class)
+public class IntegratedClientRegisterInterestAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
+
+  @Test
+  public void testRegisterInterest() throws InterruptedException {
+    // client1 connects to server as a user not authorized to do any operations
+    AsyncInvocation ai1 =  client1.invokeAsync(()->{
+      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      assertNotAuthorized(()->region.registerInterest("key3"), "DATA:READ:AuthRegion:key3");
+    });
+
+    // client2 connects to user as a user authorized to use AuthRegion region
+    AsyncInvocation ai2 =  client2.invokeAsync(()->{
+      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      region.registerInterest("key3");  //  DATA:READ:AuthRegion:key3;
+    });
+
+    // client3 connects to user as a user authorized to use key1 in AuthRegion region
+    AsyncInvocation ai3 =  client3.invokeAsync(()->{
+      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      assertNotAuthorized(()->region.registerInterest("key2"), "DATA:READ:AuthRegion:key2");
+    });
+
+    ai1.join();
+    ai2.join();
+    ai3.join();
+
+    ai1.checkException();
+    ai2.checkException();
+    ai3.checkException();
+  }
+
+  @Test
+  public void testRegisterInterestRegex() throws InterruptedException {
+    //client1 connects to server as a user not authorized to do any operations
+    AsyncInvocation ai1 =  client1.invokeAsync(()->{
+      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      assertNotAuthorized(()->region.registerInterestRegex("key.*"), "DATA:READ:AuthRegion");
+    });
+
+    // client2 connects to user as a user authorized to use AuthRegion region
+    AsyncInvocation ai2 =  client2.invokeAsync(()->{
+      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      region.registerInterestRegex("key[0-9]+");  //  DATA:READ:AuthRegion:key3;
+    });
+
+    // client3 connects to user as a user authorized to use key1 in AuthRegion region
+    AsyncInvocation ai3 =  client3.invokeAsync(()->{
+      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      assertNotAuthorized(()->region.registerInterestRegex("key[0-9]+"), "DATA:READ:AuthRegion");
+      assertNotAuthorized(()->region.registerInterestRegex("key1"), "DATA:READ:AuthRegion");
+    });
+
+    ai1.join();
+    ai2.join();
+    ai3.join();
+
+    ai1.checkException();
+    ai2.checkException();
+    ai3.checkException();
+  }
+
+  @Test
+  public void testRegisterInterestList() throws InterruptedException {
+    List<String> keys = new ArrayList<>();
+    keys.add("key1");
+    keys.add("key2");
+
+    //client1 connects to server as a user not authorized to do any operations
+    AsyncInvocation ai1 =  client1.invokeAsync(()->{
+      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      assertNotAuthorized(()->region.registerInterest(keys), "DATA:READ:AuthRegion");
+    });
+
+    // client2 connects to user as a user authorized to use AuthRegion region
+    AsyncInvocation ai2 =  client2.invokeAsync(()->{
+      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      region.registerInterest(keys);  //  DATA:READ:AuthRegion;
+    });
+
+    // client3 connects to user as a user authorized to use key1 in AuthRegion region
+    AsyncInvocation ai3 =  client3.invokeAsync(()->{
+      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      assertNotAuthorized(()->region.registerInterest(keys), "DATA:READ:AuthRegion");
+    });
+
+    ai1.join();
+    ai2.join();
+    ai3.join();
+
+    ai1.checkException();
+    ai2.checkException();
+    ai3.checkException();
+  }
+
+}



[22/50] [abbrv] incubator-geode git commit: containsKey

Posted by ji...@apache.org.
containsKey


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

Branch: refs/heads/feature/GEODE-1571
Commit: baea1716fd1eaf6cdac29a17bd7ff6bd59be487e
Parents: cb37021
Author: Kevin J. Duling <kd...@pivotal.io>
Authored: Fri Jun 24 10:10:44 2016 -0700
Committer: Kevin J. Duling <kd...@pivotal.io>
Committed: Fri Jun 24 10:11:48 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/command/ContainsKey.java | 120 ++++++++-------
 .../tier/sockets/command/ContainsKey66.java     | 150 ++++++++++---------
 ...tedClientContainsKeyAuthDistributedTest.java |  53 +++++++
 .../internal/security/clientServer.json         |   7 +
 4 files changed, 199 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/baea1716/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey.java
index 11a4dd8..3692fda 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey.java
@@ -15,38 +15,54 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.security.NotAuthorizedException;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-import java.io.IOException;
 
 
 public class ContainsKey extends BaseCommand {
 
   private final static ContainsKey singleton = new ContainsKey();
 
+  private ContainsKey() {
+  }
+
   public static Command getCommand() {
     return singleton;
   }
 
-  private ContainsKey() {
+  private static void writeContainsKeyResponse(boolean containsKey, Message origMsg, ServerConnection servConn)
+    throws IOException {
+    LogWriterI18n logger = servConn.getLogWriter();
+    Message responseMsg = servConn.getResponseMessage();
+    responseMsg.setMessageType(MessageType.RESPONSE);
+    responseMsg.setNumberOfParts(1);
+    responseMsg.setTransactionId(origMsg.getTransactionId());
+    responseMsg.addObjPart(containsKey ? Boolean.TRUE : Boolean.FALSE);
+    responseMsg.send(servConn);
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException {
     Part regionNamePart = null, keyPart = null;
     String regionName = null;
     Object key = null;
@@ -66,80 +82,68 @@ public class ContainsKey extends BaseCommand {
     regionName = regionNamePart.getString();
     try {
       key = keyPart.getStringOrObject();
-    }
-    catch (Exception e) {
+    } catch (Exception e) {
       writeException(msg, e, false, servConn);
       servConn.setAsTrue(RESPONDED);
       return;
     }
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received containsKey request ({} bytes) from {} for region {} key {}", servConn.getName(), msg.getPayloadLength(), servConn.getSocketString(), regionName, key);
+      logger.debug("{}: Received containsKey request ({} bytes) from {} for region {} key {}", servConn.getName(), msg.getPayloadLength(), servConn
+        .getSocketString(), regionName, key);
     }
 
     // Process the containsKey request
     if (key == null || regionName == null) {
       String errMessage = "";
       if (key == null) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ContainsKey_0_THE_INPUT_KEY_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL, servConn.getName()));
+        logger.warn(LocalizedMessage.create(LocalizedStrings.ContainsKey_0_THE_INPUT_KEY_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL, servConn
+          .getName()));
         errMessage = LocalizedStrings.ContainsKey_THE_INPUT_KEY_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL.toLocalizedString();
       }
       if (regionName == null) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ContainsKey_0_THE_INPUT_REGION_NAME_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL, servConn.getName()));
+        logger.warn(LocalizedMessage.create(LocalizedStrings.ContainsKey_0_THE_INPUT_REGION_NAME_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL, servConn
+          .getName()));
         errMessage = LocalizedStrings.ContainsKey_THE_INPUT_REGION_NAME_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL.toLocalizedString();
       }
-      writeErrorResponse(msg, MessageType.CONTAINS_KEY_DATA_ERROR, errMessage,
-          servConn);
+      writeErrorResponse(msg, MessageType.CONTAINS_KEY_DATA_ERROR, errMessage, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = LocalizedStrings.ContainsKey_WAS_NOT_FOUND_DURING_CONTAINSKEY_REQUEST.toLocalizedString();
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      }
-      else {
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        if (authzRequest != null) {
-          try {
-            authzRequest.containsKeyAuthorize(regionName, key);
-          }
-          catch (NotAuthorizedException ex) {
-            writeException(msg, ex, false, servConn);
-            servConn.setAsTrue(RESPONDED);
-            return;
-          }
-        }
-        // Execute the containsKey
-        boolean containsKey = region.containsKey(key);
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = LocalizedStrings.ContainsKey_WAS_NOT_FOUND_DURING_CONTAINSKEY_REQUEST.toLocalizedString();
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    GeodeSecurityUtil.authorizeDataRead();
 
-        // Update the statistics and write the reply
-        {
-          long oldStart = start;
-          start = DistributionStats.getStatTime();
-          stats.incProcessContainsKeyTime(start - oldStart);
-        }
-        writeContainsKeyResponse(containsKey, msg, servConn);
+    AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+    if (authzRequest != null) {
+      try {
+        authzRequest.containsKeyAuthorize(regionName, key);
+      } catch (NotAuthorizedException ex) {
+        writeException(msg, ex, false, servConn);
         servConn.setAsTrue(RESPONDED);
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Sent containsKey response for region {} key {}", servConn.getName(), regionName, key);
-        }
-        stats.incWriteContainsKeyResponseTime(DistributionStats.getStatTime()
-            - start);
+        return;
       }
     }
+    // Execute the containsKey
+    boolean containsKey = region.containsKey(key);
 
-  }
-
-  private static void writeContainsKeyResponse(boolean containsKey, Message origMsg,
-      ServerConnection servConn) throws IOException {
-    LogWriterI18n logger = servConn.getLogWriter();
-    Message responseMsg = servConn.getResponseMessage();
-    responseMsg.setMessageType(MessageType.RESPONSE);
-    responseMsg.setNumberOfParts(1);
-    responseMsg.setTransactionId(origMsg.getTransactionId());
-    responseMsg.addObjPart(containsKey ? Boolean.TRUE : Boolean.FALSE);
-    responseMsg.send(servConn);
+    // Update the statistics and write the reply
+    {
+      long oldStart = start;
+      start = DistributionStats.getStatTime();
+      stats.incProcessContainsKeyTime(start - oldStart);
+    }
+    writeContainsKeyResponse(containsKey, msg, servConn);
+    servConn.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Sent containsKey response for region {} key {}", servConn.getName(), regionName, key);
+    }
+    stats.incWriteContainsKeyResponseTime(DistributionStats.getStatTime() - start);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/baea1716/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey66.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey66.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey66.java
index 16b8703..5f11033 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey66.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey66.java
@@ -15,39 +15,55 @@
  * limitations under the License.
  */
 /**
- * 
+ *
  */
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
+import java.io.IOException;
+
+import com.gemstone.gemfire.cache.client.internal.ContainsKeyOp;
+import com.gemstone.gemfire.distributed.internal.DistributionStats;
 import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
 import com.gemstone.gemfire.internal.cache.tier.CachedRegionHelper;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.MessageType;
-import com.gemstone.gemfire.internal.cache.tier.sockets.*;
+import com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand;
+import com.gemstone.gemfire.internal.cache.tier.sockets.CacheServerStats;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Message;
+import com.gemstone.gemfire.internal.cache.tier.sockets.Part;
+import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 import com.gemstone.gemfire.security.NotAuthorizedException;
-import com.gemstone.gemfire.cache.client.internal.ContainsKeyOp;
-import com.gemstone.gemfire.distributed.internal.DistributionStats;
-import java.io.IOException;
 
 
 public class ContainsKey66 extends BaseCommand {
 
   private final static ContainsKey66 singleton = new ContainsKey66();
 
+  private ContainsKey66() {
+  }
+
   public static Command getCommand() {
     return singleton;
   }
 
-  private ContainsKey66() {
+  private static void writeContainsKeyResponse(boolean containsKey, Message origMsg, ServerConnection servConn)
+    throws IOException {
+    LogWriterI18n logger = servConn.getLogWriter();
+    Message responseMsg = servConn.getResponseMessage();
+    responseMsg.setMessageType(MessageType.RESPONSE);
+    responseMsg.setNumberOfParts(1);
+    responseMsg.setTransactionId(origMsg.getTransactionId());
+    responseMsg.addObjPart(containsKey ? Boolean.TRUE : Boolean.FALSE);
+    responseMsg.send(servConn);
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException {
     Part regionNamePart = null, keyPart = null;
     String regionName = null;
     Object key = null;
@@ -69,94 +85,82 @@ public class ContainsKey66 extends BaseCommand {
     regionName = regionNamePart.getString();
     try {
       key = keyPart.getStringOrObject();
-    }
-    catch (Exception e) {
+    } catch (Exception e) {
       writeException(msg, e, false, servConn);
       servConn.setAsTrue(RESPONDED);
       return;
     }
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Received containsKey request ({} bytes) from {} for region {} key {}", servConn.getName(), msg.getPayloadLength(), servConn.getSocketString(), regionName, key);
+      logger.debug("{}: Received containsKey request ({} bytes) from {} for region {} key {}", servConn.getName(), msg.getPayloadLength(), servConn
+        .getSocketString(), regionName, key);
     }
 
     // Process the containsKey request
     if (key == null || regionName == null) {
       String errMessage = "";
       if (key == null) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ContainsKey_0_THE_INPUT_KEY_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL, servConn.getName()));
+        logger.warn(LocalizedMessage.create(LocalizedStrings.ContainsKey_0_THE_INPUT_KEY_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL, servConn
+          .getName()));
         errMessage = LocalizedStrings.ContainsKey_THE_INPUT_KEY_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL.toLocalizedString();
       }
       if (regionName == null) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ContainsKey_0_THE_INPUT_REGION_NAME_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL, servConn.getName()));
+        logger.warn(LocalizedMessage.create(LocalizedStrings.ContainsKey_0_THE_INPUT_REGION_NAME_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL, servConn
+          .getName()));
         errMessage = LocalizedStrings.ContainsKey_THE_INPUT_REGION_NAME_FOR_THE_CONTAINSKEY_REQUEST_IS_NULL.toLocalizedString();
       }
-      writeErrorResponse(msg, MessageType.CONTAINS_KEY_DATA_ERROR, errMessage,
-          servConn);
+      writeErrorResponse(msg, MessageType.CONTAINS_KEY_DATA_ERROR, errMessage, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = LocalizedStrings.ContainsKey_WAS_NOT_FOUND_DURING_CONTAINSKEY_REQUEST.toLocalizedString();
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      }
-      else {
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        if (authzRequest != null) {
-          try {
-            authzRequest.containsKeyAuthorize(regionName, key);
-          }
-          catch (NotAuthorizedException ex) {
-            writeException(msg, ex, false, servConn);
-            servConn.setAsTrue(RESPONDED);
-            return;
-          }
-        }
-        // Execute the containsKey
-        boolean containsKey;
-        switch(mode) {
-        case KEY:
-          containsKey = region.containsKey(key);
-          break;
-        case VALUE:
-          containsKey = region.containsValue(key);
-          break;
-        case VALUE_FOR_KEY:
-          containsKey = region.containsValueForKey(key);
-          break;
-        default:
-          containsKey = false;
-          break;
-        }
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = LocalizedStrings.ContainsKey_WAS_NOT_FOUND_DURING_CONTAINSKEY_REQUEST.toLocalizedString();
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+
+    GeodeSecurityUtil.authorizeDataRead();
 
-        // Update the statistics and write the reply
-        {
-          long oldStart = start;
-          start = DistributionStats.getStatTime();
-          stats.incProcessContainsKeyTime(start - oldStart);
-        }
-        writeContainsKeyResponse(containsKey, msg, servConn);
+    AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+    if (authzRequest != null) {
+      try {
+        authzRequest.containsKeyAuthorize(regionName, key);
+      } catch (NotAuthorizedException ex) {
+        writeException(msg, ex, false, servConn);
         servConn.setAsTrue(RESPONDED);
-        if (logger.isDebugEnabled()) {
-          logger.debug("{}: Sent containsKey response for region {} key {}", servConn.getName(), regionName, key);
-        }
-        stats.incWriteContainsKeyResponseTime(DistributionStats.getStatTime()
-            - start);
+        return;
       }
     }
+    // Execute the containsKey
+    boolean containsKey;
+    switch (mode) {
+      case KEY:
+        containsKey = region.containsKey(key);
+        break;
+      case VALUE:
+        containsKey = region.containsValue(key);
+        break;
+      case VALUE_FOR_KEY:
+        containsKey = region.containsValueForKey(key);
+        break;
+      default:
+        containsKey = false;
+        break;
+    }
 
-  }
-
-  private static void writeContainsKeyResponse(boolean containsKey, Message origMsg,
-      ServerConnection servConn) throws IOException {
-    LogWriterI18n logger = servConn.getLogWriter();
-    Message responseMsg = servConn.getResponseMessage();
-    responseMsg.setMessageType(MessageType.RESPONSE);
-    responseMsg.setNumberOfParts(1);
-    responseMsg.setTransactionId(origMsg.getTransactionId());
-    responseMsg.addObjPart(containsKey ? Boolean.TRUE : Boolean.FALSE);
-    responseMsg.send(servConn);
+    // Update the statistics and write the reply
+    {
+      long oldStart = start;
+      start = DistributionStats.getStatTime();
+      stats.incProcessContainsKeyTime(start - oldStart);
+    }
+    writeContainsKeyResponse(containsKey, msg, servConn);
+    servConn.setAsTrue(RESPONDED);
+    if (logger.isDebugEnabled()) {
+      logger.debug("{}: Sent containsKey response for region {} key {}", servConn.getName(), regionName, key);
+    }
+    stats.incWriteContainsKeyResponseTime(DistributionStats.getStatTime() - start);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/baea1716/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
new file mode 100644
index 0000000..093d1ae
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
@@ -0,0 +1,53 @@
+/*
+ * 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.security;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+@Category(DistributedTest.class)
+public class IntegratedClientContainsKeyAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
+
+  @Test
+  public void testContainsKey() throws InterruptedException {
+
+    AsyncInvocation ai1 = client1.invokeAsync(() -> {
+      Cache cache = SecurityTestUtils.createCacheClient("dataReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      assertTrue(region.containsKeyOnServer("key1"));
+    });
+
+    AsyncInvocation ai2 = client2.invokeAsync(() -> {
+      Cache cache = SecurityTestUtils.createCacheClient("authRegionReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+
+      assertNotAuthorized(() -> region.containsKeyOnServer("key3"), "DATA:READ");
+    });
+    ai1.join();
+    ai2.join();
+    ai1.checkException();
+    ai2.checkException();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/baea1716/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
----------------------------------------------------------------------
diff --git a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
index e5313d2..c659709 100644
--- a/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
+++ b/geode-core/src/test/resources/com/gemstone/gemfire/management/internal/security/clientServer.json
@@ -129,6 +129,13 @@
       ]
     },
     {
+      "name": "dataReader",
+      "password": "1234567",
+      "roles": [
+        "data-read"
+      ]
+    },
+    {
       "name":"authRegionUser",
       "password": "1234567",
       "roles": [


[05/50] [abbrv] incubator-geode git commit: GEODE-1572: add a dunit test for client server query for lucene

Posted by ji...@apache.org.
GEODE-1572: add a dunit test for client server query for lucene


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

Branch: refs/heads/feature/GEODE-1571
Commit: de6105f98021095c110bbc91ba547f8a01904ae7
Parents: 9ccb4d5
Author: zhouxh <gz...@pivotal.io>
Authored: Tue Jun 21 10:15:21 2016 -0700
Committer: zhouxh <gz...@pivotal.io>
Committed: Tue Jun 21 10:15:59 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneQueriesBase.java | 10 +--
 .../lucene/LuceneQueriesClientDUnitTest.java    | 70 ++++++++++++++++++++
 2 files changed, 75 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/de6105f9/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
index 26cd4aa..2fdf531 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
@@ -47,7 +47,7 @@ public abstract class LuceneQueriesBase extends LuceneDUnitTest {
   protected VM accessor;
 
   @Override
-  public final void postSetUp() throws Exception {
+  public void postSetUp() throws Exception {
     super.postSetUp();
     accessor = Host.getHost(0).getVM(3);
   }
@@ -65,7 +65,7 @@ public abstract class LuceneQueriesBase extends LuceneDUnitTest {
     accessor.invoke(() -> initAccessor(createIndex));
 
     putDataInRegion(accessor);
-    assertTrue(waitForFlushBeforeExecuteTextSearch(accessor, 60000));
+    assertTrue(waitForFlushBeforeExecuteTextSearch(dataStore1, 60000));
     executeTextSearch(accessor);
   }
 
@@ -79,7 +79,7 @@ public abstract class LuceneQueriesBase extends LuceneDUnitTest {
     dataStore2.invoke(() -> initDataStore(createIndex));
     accessor.invoke(() -> initAccessor(createIndex));
     putDataInRegion(accessor);
-    assertTrue(waitForFlushBeforeExecuteTextSearch(accessor, 60000));
+    assertTrue(waitForFlushBeforeExecuteTextSearch(dataStore1, 60000));
     executeTextSearch(accessor, "world", "text", 3);
     executeTextSearch(accessor, "world", "noEntriesMapped", 0);
   }
@@ -98,10 +98,10 @@ public abstract class LuceneQueriesBase extends LuceneDUnitTest {
     dataStore1.invoke(() -> pauseSender(getCache()));
     dataStore2.invoke(() -> pauseSender(getCache()));
     putDataInRegion(accessor);
-    assertFalse(waitForFlushBeforeExecuteTextSearch(accessor, 500));
+    assertFalse(waitForFlushBeforeExecuteTextSearch(dataStore1, 500));
     dataStore1.invoke(() -> resumeSender(getCache()));
     dataStore2.invoke(() -> resumeSender(getCache()));
-    assertTrue(waitForFlushBeforeExecuteTextSearch(accessor, 60000));
+    assertTrue(waitForFlushBeforeExecuteTextSearch(dataStore1, 60000));
     executeTextSearch(accessor);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/de6105f9/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesClientDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesClientDUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesClientDUnitTest.java
new file mode 100755
index 0000000..1988baf
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesClientDUnitTest.java
@@ -0,0 +1,70 @@
+/*
+ * 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.cache.lucene;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
+
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.client.ClientCache;
+import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
+import com.gemstone.gemfire.cache.server.CacheServer;
+import com.gemstone.gemfire.test.dunit.SerializableCallableIF;
+import com.gemstone.gemfire.test.dunit.SerializableRunnableIF;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+@Category(DistributedTest.class)
+public class LuceneQueriesClientDUnitTest extends LuceneQueriesBase {
+
+  @Override public void postSetUp() throws Exception {
+    super.postSetUp();
+    SerializableCallableIF<Integer> launchServer = () ->
+    {
+      final Cache cache = getCache();
+      final CacheServer server = cache.addCacheServer();
+      server.setPort(0);
+      server.start();
+      return server.getPort();
+    };
+    final int port1 = dataStore1.invoke(launchServer);
+    final int port2 = dataStore2.invoke(launchServer);
+
+    accessor.invoke(() -> {
+      ClientCacheFactory clientCacheFactory = new ClientCacheFactory();
+      clientCacheFactory.addPoolServer("localhost", port1);
+      clientCacheFactory.addPoolServer("localhost", port2);
+      ClientCache clientCache = getClientCache(clientCacheFactory);
+      clientCache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create(REGION_NAME);
+    });
+  }
+
+  @Override
+  protected void initAccessor(SerializableRunnableIF createIndex) throws Exception {
+  }
+
+  @Override
+  protected void initDataStore(SerializableRunnableIF createIndex) throws Exception {
+    createIndex.run();
+    getCache().createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
+  }
+
+}


[09/50] [abbrv] incubator-geode git commit: Clear Region - fixed test

Posted by ji...@apache.org.
Clear Region - fixed test


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

Branch: refs/heads/feature/GEODE-1571
Commit: 2e1b3810d7fba94c03bc58b381df3144454daeed
Parents: dd1b04b
Author: Kevin J. Duling <kd...@pivotal.io>
Authored: Tue Jun 21 16:32:36 2016 -0700
Committer: Kevin J. Duling <kd...@pivotal.io>
Committed: Tue Jun 21 16:32:36 2016 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2e1b3810/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
index a408c81..121c916 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
@@ -268,11 +268,11 @@ public class IntegratedClientAuthDUnitTest extends JUnit4DistributedTestCase {
       public void run() {
         Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
         final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
-        assertNotAuthorized(() -> region.clear(), "DATA:WRITE:AuthRegion");
+        region.clear();
         cache.close();
       }
     };
-    client1.invoke(clearUnauthorized);
+    client2.invoke(clearAuthorized);
   }
 
 }


[14/50] [abbrv] incubator-geode git commit: GEODE-1209: Typo repair in javadoc comment (AsynEventQueue -> AsyncEventQueue)

Posted by ji...@apache.org.
GEODE-1209: Typo repair in javadoc comment (AsynEventQueue -> AsyncEventQueue)


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

Branch: refs/heads/feature/GEODE-1571
Commit: fc42db82e002fbe476fd9ffa01777b568ef58dba
Parents: 5882693
Author: Dave Barnes <db...@pivotal.io>
Authored: Wed Jun 22 11:33:51 2016 -0700
Committer: Dave Barnes <db...@pivotal.io>
Committed: Wed Jun 22 11:33:51 2016 -0700

----------------------------------------------------------------------
 .../gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/fc42db82/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
index 455fc14..3540ce3 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/cache/asyncqueue/AsyncEventQueueFactory.java
@@ -171,7 +171,7 @@ public interface AsyncEventQueueFactory {
       GatewayEventSubstitutionFilter filter);
 
   /**
-   * Forwards destroy operations from expiration action to AsynEventQueue.
+   * Forwards destroy operations from expiration action to AsyncEventQueue.
    * By default the expiration destroy events are not added to AEQ, by
    * setting this attribute to true adds destroy event to AEQ.
    *


[02/50] [abbrv] incubator-geode git commit: Merge branch 'develop' into feature/GEODE-1571

Posted by ji...@apache.org.
Merge branch 'develop' into feature/GEODE-1571

# Please enter a commit message to explain why this merge is necessary,
# especially if it merges an updated upstream into a topic branch.
#
# Lines starting with '#' will be ignored, and an empty message aborts
# the commit.


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

Branch: refs/heads/feature/GEODE-1571
Commit: 80765671cc4afa3610a21a27464bed85664b322a
Parents: 756d821 83c7c66
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Tue Jun 21 09:27:17 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Tue Jun 21 09:27:17 2016 -0700

----------------------------------------------------------------------
 .../CreateAlterDestroyRegionCommands.java       |  2 +-
 .../internal/cli/i18n/CliStrings.java           |  8 +-
 .../gemfire/distributed/LocatorDUnitTest.java   |  1 +
 .../commands/HelpCommandsIntegrationTest.java   |  4 +
 .../cli/commands/golden-help-offline.properties | 82 ++++++++++----------
 .../lucene/internal/LuceneEventListener.java    | 10 ++-
 .../LuceneIndexForPartitionedRegion.java        |  1 +
 .../LuceneIndexMaintenanceIntegrationTest.java  | 35 +++++++++
 .../internal/LuceneEventListenerJUnitTest.java  | 21 +++++
 9 files changed, 115 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/80765671/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/commands/CreateAlterDestroyRegionCommands.java
----------------------------------------------------------------------


[27/50] [abbrv] incubator-geode git commit: GEODE-11 Added the findKeys method to LuceneQuery

Posted by ji...@apache.org.
GEODE-11 Added the findKeys method to LuceneQuery

Added the findKeys method to LuceneQuery that returns the list of keys. Added
unit and intergration to test findKeys. Added an integration test for
pagination.


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

Branch: refs/heads/feature/GEODE-1571
Commit: 14437b72e1600233173586444e8a9d4f7ac88335
Parents: d0e61ec
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Fri Jun 17 16:17:19 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri Jun 24 14:41:41 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneQuery.java       |   8 ++
 .../cache/lucene/internal/LuceneQueryImpl.java  |  36 +++++-
 .../lucene/LuceneQueriesIntegrationTest.java    |  41 +++++++
 .../LuceneQueryImplIntegrationTest.java         | 120 -------------------
 .../internal/LuceneQueryImplJUnitTest.java      | 116 ++++++++++++++++++
 .../cache/lucene/test/LuceneTestUtilities.java  |   8 +-
 6 files changed, 199 insertions(+), 130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
index be47677..9915116 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
@@ -16,6 +16,8 @@
  */
 package com.gemstone.gemfire.cache.lucene;
 
+import java.util.Collection;
+
 import com.gemstone.gemfire.annotations.Experimental;
 
 /**
@@ -27,6 +29,11 @@ import com.gemstone.gemfire.annotations.Experimental;
 @Experimental
 public interface LuceneQuery<K, V> {
   /**
+   * Execute search and return keys.
+   */
+  public Collection<K> findKeys() throws LuceneQueryException;
+
+  /**
    * Execute the search and get results. 
    */
   public PageableLuceneQueryResults<K, V> findPages() throws LuceneQueryException;
@@ -45,4 +52,5 @@ public interface LuceneQuery<K, V> {
    * Get projected fields setting of current query. 
    */
   public String[] getProjectedFieldNames();
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
index 44fa446..cd49f72 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
@@ -19,7 +19,13 @@
 
 package com.gemstone.gemfire.cache.lucene.internal;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.execute.Execution;
 import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
@@ -28,6 +34,7 @@ import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntries;
@@ -57,20 +64,36 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
   }
 
   @Override
+  public Collection<K> findKeys() throws LuceneQueryException {
+    TopEntries entries = findTopEntries();
+    final List<EntryScore> hits = entries.getHits();
+
+    return hits.stream()
+      .map(hit -> (K) hit.getKey())
+      .collect(Collectors.toList());
+  }
+
+  @Override
   public PageableLuceneQueryResults<K, V> findPages() throws LuceneQueryException {
+    TopEntries entries = findTopEntries();
+
+    return new PageableLuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
+  }
+
+  private TopEntries findTopEntries() throws LuceneQueryException {
     TopEntriesCollectorManager manager = new TopEntriesCollectorManager(null, limit);
     LuceneFunctionContext<TopEntriesCollector> context = new LuceneFunctionContext<>(query, indexName, manager, limit);
     TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(context);
 
-    ResultCollector<TopEntriesCollector, TopEntries> rc = (ResultCollector<TopEntriesCollector, TopEntries>) FunctionService.onRegion(region)
+    ResultCollector<TopEntriesCollector, TopEntries> rc = (ResultCollector<TopEntriesCollector, TopEntries>) onRegion()
         .withArgs(context)
         .withCollector(collector)
         .execute(LuceneFunction.ID);
-    
+
     //TODO provide a timeout to the user?
+    TopEntries entries;
     try {
-      TopEntries entries = rc.getResult();
-      return new PageableLuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
+      entries = rc.getResult();
     } catch(FunctionException e) {
       if(e.getCause() instanceof LuceneQueryException) {
         throw new LuceneQueryException(e);
@@ -78,6 +101,11 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
         throw e;
       }
     }
+    return entries;
+  }
+
+  protected Execution onRegion() {
+    return FunctionService.onRegion(region);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index dbb92cf..cf1b5d2 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -17,8 +17,13 @@
 package com.gemstone.gemfire.cache.lucene;
 
 import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
@@ -110,6 +115,42 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
+  public void shouldPaginateResults() throws Exception {
+
+    luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1", "field2");
+    Region region = cache.createRegionFactory(RegionShortcut.PARTITION)
+      .create(REGION_NAME);
+    final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
+
+    //Put two values with some of the same tokens
+    String value1 = "one three";
+    String value2 = "one two three";
+    String value3 = "one@three";
+    region.put("A", new TestObject(value1, value1));
+    region.put("B", new TestObject(value2, value2));
+    region.put("C", new TestObject(value3, value3));
+
+    index.waitUntilFlushed(60000);
+    final LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+      .setPageSize(2)
+      .create(INDEX_NAME, REGION_NAME,
+      "one", "field1");
+
+    final PageableLuceneQueryResults<Object, Object> pages = query.findPages();
+    assertTrue(pages.hasNextPage());
+    assertEquals(3, pages.size());
+    final List<LuceneResultStruct<Object, Object>> page1 = pages.getNextPage();
+    final List<LuceneResultStruct<Object, Object>> page2 = pages.getNextPage();
+    List<LuceneResultStruct<Object, Object>> allEntries=new ArrayList<>();
+    allEntries.addAll(page1);
+    allEntries.addAll(page2);
+
+    assertEquals(region.keySet(), allEntries.stream().map(entry -> entry.getKey()).collect(Collectors.toSet()));
+    assertEquals(region.values(), allEntries.stream().map(entry -> entry.getValue()).collect(Collectors.toSet()));
+
+  }
+
+  @Test()
   public void shouldTokenizeUsingMyCharacterAnalyser() throws Exception {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     // not to specify field1's analyzer, it should use standard analyzer

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
deleted file mode 100644
index 62f4623..0000000
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
+++ /dev/null
@@ -1,120 +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.cache.lucene.internal;
-
-import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.DEFAULT_FIELD;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.util.List;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.execute.FunctionAdapter;
-import com.gemstone.gemfire.cache.execute.FunctionContext;
-import com.gemstone.gemfire.cache.execute.FunctionService;
-import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
-import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
-import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollectorManager;
-import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector;
-import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
-
-@Category(IntegrationTest.class)
-public class LuceneQueryImplIntegrationTest extends LuceneIntegrationTest {
-  private static int LIMIT = 123;
-  private Region<Object, Object> region;
-
-  @Before
-  public void createRegion() {
-    region = cache.createRegionFactory(RegionShortcut.PARTITION).create("region");
-  }
-
-  @After
-  public void removeFunction() {
-    FunctionService.unregisterFunction(LuceneFunction.ID);
-  }
-
-  @Test
-  public void shouldInvokeLuceneFunctionWithCorrectArguments() throws Exception {
-    // Register a fake function to observe the function invocation
-    FunctionService.unregisterFunction(LuceneFunction.ID);
-    TestLuceneFunction function = new TestLuceneFunction();
-    FunctionService.registerFunction(function);
-
-    StringQueryProvider provider = new StringQueryProvider(null, DEFAULT_FIELD);
-    LuceneQueryImpl<Object, Object> query = new LuceneQueryImpl<>("index", region, provider, null, LIMIT, 20);
-    PageableLuceneQueryResults<Object, Object> results = query.findPages();
-
-    assertTrue(function.wasInvoked);
-    assertEquals(2f * LIMIT, results.getMaxScore(), 0.01);
-    int resultCount = 0;
-    while (results.hasNextPage()) {
-      List<LuceneResultStruct<Object, Object>> nextPage = results.getNextPage();
-      resultCount += nextPage.size();
-      if (results.hasNextPage()) {
-        assertEquals(20, nextPage.size());
-      }
-    }
-    assertEquals(LIMIT, resultCount);
-
-    LuceneFunctionContext<? extends IndexResultCollector> funcArgs = function.args;
-    assertEquals(provider.getQueryString(), ((StringQueryProvider) funcArgs.getQueryProvider()).getQueryString());
-    assertEquals("index", funcArgs.getIndexName());
-    assertEquals(LIMIT, funcArgs.getLimit());
-  }
-
-  private static class TestLuceneFunction extends FunctionAdapter {
-    private static final long serialVersionUID = 1L;
-    private boolean wasInvoked;
-    private LuceneFunctionContext<? extends IndexResultCollector> args;
-
-    @Override
-    public void execute(FunctionContext context) {
-      this.args = (LuceneFunctionContext<?>) context.getArguments();
-      TopEntriesCollectorManager manager = (TopEntriesCollectorManager) args.getCollectorManager();
-
-      assertEquals(LIMIT, manager.getLimit());
-
-      wasInvoked = true;
-      TopEntriesCollector lastResult = new TopEntriesCollector(null, 2 * LIMIT);
-      // put more than LIMIT entries. The resultCollector should trim the results
-      for (int i = LIMIT * 2; i >= 0; i--) {
-        lastResult.collect(i, i * 1f);
-      }
-      assertEquals(LIMIT * 2, lastResult.getEntries().getHits().size());
-
-      context.getResultSender().lastResult(lastResult);
-    }
-
-    @Override
-    public String getId() {
-      return LuceneFunction.ID;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
new file mode 100644
index 0000000..1b36e37
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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.cache.lucene.internal;
+
+import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.DEFAULT_FIELD;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.*;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.ArgumentCaptor;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.execute.Execution;
+import com.gemstone.gemfire.cache.execute.FunctionAdapter;
+import com.gemstone.gemfire.cache.execute.FunctionContext;
+import com.gemstone.gemfire.cache.execute.FunctionService;
+import com.gemstone.gemfire.cache.execute.ResultCollector;
+import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
+import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntries;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollector;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntriesCollectorManager;
+import com.gemstone.gemfire.cache.lucene.internal.repository.IndexResultCollector;
+import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class LuceneQueryImplJUnitTest {
+  private static int LIMIT = 123;
+  private LuceneQueryImpl<Object, Object> query;
+  private Execution execution;
+  private LuceneQueryProvider provider;
+
+  @Before
+  public void createMocks() {
+    Region region = mock(Region.class);
+    execution = mock(Execution.class);
+    ResultCollector<TopEntriesCollector, TopEntries> collector = mock(ResultCollector.class);
+    provider = mock(LuceneQueryProvider.class);
+
+    when(execution.withArgs(any())).thenReturn(execution);
+    when(execution.withCollector(any())).thenReturn(execution);
+    when(execution.execute(anyString())).thenReturn((ResultCollector) collector);
+
+    TopEntries entries = new TopEntries();
+    entries.addHit(new EntryScore("hi", 5));
+    when(collector.getResult()).thenReturn(entries);
+
+
+    query = new LuceneQueryImpl<Object, Object>("index", region, provider, null, LIMIT, 20) {
+      @Override protected Execution onRegion() {
+        return execution;
+      }
+    };
+  }
+
+  @Test
+  public void shouldReturnKeysFromFindKeys() throws LuceneQueryException {
+    Collection<Object> results = query.findKeys();
+    assertEquals(Collections.singletonList("hi"), results);
+  }
+
+  @Test
+  public void shouldInvokeLuceneFunctionWithCorrectArguments() throws Exception {
+    PageableLuceneQueryResults<Object, Object> results = query.findPages();
+
+    verify(execution).execute(eq(LuceneFunction.ID));
+    ArgumentCaptor<LuceneFunctionContext> captor = ArgumentCaptor.forClass(LuceneFunctionContext.class);
+    verify(execution).withArgs(captor.capture());
+    LuceneFunctionContext context = captor.getValue();
+    assertEquals(LIMIT, context.getLimit());
+    assertEquals(provider, context.getQueryProvider());
+    assertEquals("index", context.getIndexName());
+
+    assertEquals(5, results.getMaxScore(), 0.01);
+    final List<LuceneResultStruct<Object, Object>> page = results.getNextPage();
+    assertEquals(1, page.size());
+    LuceneResultStruct element = page.iterator().next();
+    assertEquals("hi", element.getKey());
+    assertEquals(5, element.getScore(), 0.01);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/14437b72/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
index 2084e11..06c3c7d 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -21,6 +21,7 @@ package com.gemstone.gemfire.cache.lucene.test;
 import static org.junit.Assert.*;
 
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
@@ -89,12 +90,7 @@ public class LuceneTestUtilities {
    */
   public static <K> void  verifyQueryKeys(LuceneQuery<K,Object> query,K ... expectedKeys) throws LuceneQueryException {
     Set<K> expectedKeySet = new HashSet<>(Arrays.asList(expectedKeys));
-    Set<K> actualKeySet = new HashSet<>();
-    final PageableLuceneQueryResults<K, Object> results = query.findPages();
-    while(results.hasNextPage()) {
-      results.getNextPage().stream()
-        .forEach(struct -> actualKeySet.add(struct.getKey()));
-    }
+    Set<K> actualKeySet = new HashSet<>(query.findKeys());
     assertEquals(expectedKeySet, actualKeySet);
   }
 


[18/50] [abbrv] incubator-geode git commit: Extract tests to their own distributed test classes

Posted by ji...@apache.org.
Extract tests to their own distributed test classes


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

Branch: refs/heads/feature/GEODE-1571
Commit: 6a45ca0cc8cd9283a84cc43f52c54bc4c17e95b9
Parents: 46017f9 2b32829
Author: gmeilen <gr...@gmail.com>
Authored: Thu Jun 23 15:20:23 2016 -0700
Committer: gmeilen <gr...@gmail.com>
Committed: Thu Jun 23 15:20:23 2016 -0700

----------------------------------------------------------------------
 .../asyncqueue/AsyncEventQueueFactory.java      |   2 +-
 .../cache/tier/sockets/BaseCommand.java         |   7 -
 .../cache/tier/sockets/BaseCommandQuery.java    |  11 +-
 .../tier/sockets/command/CloseConnection.java   |  13 +-
 .../tier/sockets/command/CreateRegion.java      | 122 +++++++-------
 .../tier/sockets/command/DestroyRegion.java     | 167 +++++++++----------
 .../security/IntegratedClientAuthDUnitTest.java | 167 -------------------
 ...entDestroyInvalidateAuthDistributedTest.java |  78 +++++++++
 ...dClientDestroyRegionAuthDistributedTest.java |  50 ++++++
 ...tegratedClientGetPutAuthDistributedTest.java | 116 +++++++++++++
 ...ntegratedClientQueryAuthDistributedTest.java |  45 +++++
 ...tedClientRegionClearAuthDistributedTest.java |  54 ++++++
 .../internal/security/clientServer.json         |   7 +
 13 files changed, 507 insertions(+), 332 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6a45ca0c/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/BaseCommand.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6a45ca0c/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
index 506a171,4b7b573..10a8f73
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
@@@ -14,23 -14,22 +14,10 @@@
   * See the License for the specific language governing permissions and
   * limitations under the License.
   */
--
  package com.gemstone.gemfire.security;
  
 -import static com.gemstone.gemfire.security.SecurityTestUtils.*;
 -import static org.assertj.core.api.Assertions.*;
--import static org.junit.Assert.*;
--
--import java.util.ArrayList;
--import java.util.HashMap;
--import java.util.List;
--import java.util.Map;
--import java.util.Set;
- 
- import com.gemstone.gemfire.cache.Cache;
- import com.gemstone.gemfire.cache.Region;
- import com.gemstone.gemfire.test.dunit.AsyncInvocation;
- import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 +import com.gemstone.gemfire.test.junit.categories.DistributedTest;
  
 -import org.assertj.core.api.ThrowableAssert.ThrowingCallable;
 -import org.junit.After;
 -import org.junit.Before;
  import org.junit.Test;
  import org.junit.experimental.categories.Category;
  
@@@ -48,160 -90,193 +35,6 @@@ public class IntegratedClientAuthDUnitT
        SecurityTestUtils.createCacheClient("super-user", "wrong", port, SecurityTestUtils.AUTHFAIL_EXCEPTION);
      });
    }
- 
-   @Test
-   public void testGetPutAuthorization() throws InterruptedException {
-     Map<String, String> allValues = new HashMap<String, String>();
-     allValues.put("key1", "value1");
-     allValues.put("key2", "value2");
- 
-     List<String> keys = new ArrayList<>();
-     keys.add("key1");
-     keys.add("key2");
- 
-     // client1 connects to server as a user not authorized to do any operations
-     AsyncInvocation ai1 =  client1.invokeAsync(()->{
-       Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
- 
-       assertNotAuthorized(()->region.put("key3", "value3"), "DATA:WRITE:AuthRegion:key3");
-       assertNotAuthorized(()->region.get("key3"), "DATA:READ:AuthRegion:key3");
- 
-       //putall
-       assertNotAuthorized(()->region.putAll(allValues), "DATA:WRITE:AuthRegion");
- 
-       // not authorized for either keys, get no record back
-       Map keyValues =  region.getAll(keys);
-       assertEquals(0, keyValues.size());
- 
-       Set keySet = region.keySet();
-       assertEquals(0, keySet.size());
- 
- //      Query query = cache.getQueryService().newQuery("select * from /AuthRegion");
- //      Object result = query.execute();
- 
-       cache.close();
-     });
- 
- 
-     // client2 connects to user as a user authorized to use AuthRegion region
-     AsyncInvocation ai2 =  client2.invokeAsync(()->{
-       Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
- 
-       region.put("key3", "value3");
-       assertEquals("value3", region.get("key3"));
- 
-       // put all
-       region.putAll(allValues);
- 
-       // get all
-       Map keyValues =  region.getAll(keys);
-       assertEquals(2, keyValues.size());
- 
-       // keyset
-       Set keySet = region.keySet();
-       assertEquals(3, keySet.size());
-     });
- 
-     // client3 connects to user as a user authorized to use key1 in AuthRegion region
-     AsyncInvocation ai3 =  client3.invokeAsync(()->{
-       Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
- 
-       assertNotAuthorized(()->region.put("key2", "value1"), "DATA:WRITE:AuthRegion:key2");
-       assertNotAuthorized(()->region.get("key2"), "DATA:READ:AuthRegion:key2");
- 
-       assertNotAuthorized(()->region.putAll(allValues), "DATA:WRITE:AuthRegion");
- 
-       // only authorized for one recrod
-       Map keyValues =  region.getAll(keys);
-       assertEquals(1, keyValues.size());
- 
-       // keyset
-       Set keySet = region.keySet();
-       assertEquals(1, keySet.size());
-     });
- 
-     ai1.join();
-     ai2.join();
-     ai3.join();
- 
-     ai1.checkException();
-     ai2.checkException();
-     ai3.checkException();
-   }
- 
-   @Test
-   public void testDestroyInvalidate() throws InterruptedException {
- 
-     // Delete one key and invalidate another key with an authorized user.
-     AsyncInvocation ai1 = client1.invokeAsync(() -> {
-       Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
- 
-       assertEquals(region.get("key1"), "value1");
-       assertTrue(region.containsKey("key1")); // will only be true after we first get it, then it's cached locally
- 
-       // Destroy key1
-       region.destroy("key1");
-       assertFalse(region.containsKey("key1"));
- 
-       // Invalidate key2
-       assertNotNull("Value of key2 should not be null", region.get("key2"));
-       region.invalidate("key2");
-       assertNull("Value of key2 should have been null", region.get("key2"));
- 
-     });
- 
-     // Delete one key and invalidate another key with an unauthorized user.
-     AsyncInvocation ai2 = client2.invokeAsync(() -> {
-       Cache cache = SecurityTestUtils.createCacheClient("authRegionReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
- 
-       assertEquals(region.get("key3"), "value3");
-       assertTrue(region.containsKey("key3")); // will only be true after we first get it, then it's cached locally
- 
-       // Destroy key1
-       assertNotAuthorized(() -> region.destroy("key3"), "DATA:WRITE:AuthRegion");
-       assertTrue(region.containsKey("key3"));
- 
-       // Invalidate key2
-       assertNotNull("Value of key4 should not be null", region.get("key4"));
-       assertNotAuthorized(() -> region.invalidate("key4"), "DATA:WRITE:AuthRegion");
-       assertNotNull("Value of key4 should not be null", region.get("key4"));
-       cache.close();
-     });
-     ai1.join();
-     ai2.join();
-     ai1.checkException();
-     ai2.checkException();
-   }
- 
-   @Test
-   public void testRegionClear() throws InterruptedException {
-     // Verify that an unauthorized user can't clear the region
-     SerializableRunnable clearUnauthorized = new SerializableRunnable() {
-       @Override
-       public void run() {
-         Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-         final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
-         assertNotAuthorized(() -> region.clear(), "DATA:WRITE:AuthRegion");
-       }
-     };
-     client1.invoke(clearUnauthorized);
- 
-     // Verify that an authorized user can clear the region
-     SerializableRunnable clearAuthorized = new SerializableRunnable() {
-       @Override
-       public void run() {
-         Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-         final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
-         region.clear();
-       }
-     };
-     client2.invoke(clearAuthorized);
-   }
 +}
  
 -  @Test
 -  public void testGetPutAuthorization() throws InterruptedException {
 -    Map<String, String> allValues = new HashMap<String, String>();
 -    allValues.put("key1", "value1");
 -    allValues.put("key2", "value2");
 -
 -    List<String> keys = new ArrayList<>();
 -    keys.add("key1");
 -    keys.add("key2");
 -
 -    // client1 connects to server as a user not authorized to do any operations
 -    AsyncInvocation ai1 =  client1.invokeAsync(()->{
 -      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -
 -      assertNotAuthorized(()->region.put("key3", "value3"), "DATA:WRITE:AuthRegion:key3");
 -      assertNotAuthorized(()->region.get("key3"), "DATA:READ:AuthRegion:key3");
 -
 -      //putall
 -      assertNotAuthorized(()->region.putAll(allValues), "DATA:WRITE:AuthRegion");
 -
 -      // not authorized for either keys, get no record back
 -      Map keyValues =  region.getAll(keys);
 -      assertEquals(0, keyValues.size());
 -
 -      Set keySet = region.keySet();
 -      assertEquals(0, keySet.size());
 -    });
 -
 -
 -    // client2 connects to user as a user authorized to use AuthRegion region
 -    AsyncInvocation ai2 =  client2.invokeAsync(()->{
 -      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -
 -      region.put("key3", "value3");
 -      assertEquals("value3", region.get("key3"));
 -
 -      // put all
 -      region.putAll(allValues);
 -
 -      // get all
 -      Map keyValues =  region.getAll(keys);
 -      assertEquals(2, keyValues.size());
 -
 -      // keyset
 -      Set keySet = region.keySet();
 -      assertEquals(3, keySet.size());
 -    });
 -
 -    // client3 connects to user as a user authorized to use key1 in AuthRegion region
 -    AsyncInvocation ai3 =  client3.invokeAsync(()->{
 -      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -
 -      assertNotAuthorized(()->region.put("key2", "value1"), "DATA:WRITE:AuthRegion:key2");
 -      assertNotAuthorized(()->region.get("key2"), "DATA:READ:AuthRegion:key2");
 -
 -      assertNotAuthorized(()->region.putAll(allValues), "DATA:WRITE:AuthRegion");
 -
 -      // only authorized for one recrod
 -      Map keyValues =  region.getAll(keys);
 -      assertEquals(1, keyValues.size());
 -
 -      // keyset
 -      Set keySet = region.keySet();
 -      assertEquals(1, keySet.size());
 -    });
 -
 -    ai1.join();
 -    ai2.join();
 -    ai3.join();
 -
 -    ai1.checkException();
 -    ai2.checkException();
 -    ai3.checkException();
 -  }
 -
 -  @Test
 -  public void testQuery(){
 -    client1.invoke(()-> {
 -      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -
 -      String query = "select * from /AuthRegion";
 -      assertNotAuthorized(()->region.query(query), "DATA:READ:AuthRegion");
 -
 -      Pool pool = PoolManager.find(region);
 -      assertNotAuthorized(()->pool.getQueryService().newQuery(query).execute(), "DATA:READ:AuthRegion");
 -    });
 -  }
 -
 -  @Test
 -  public void testDestroyRegion() throws InterruptedException {
 -    client1.invoke(()-> {
 -      Cache cache = SecurityTestUtils.createCacheClient("dataWriter", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -      assertNotAuthorized(()->region.destroyRegion(), "DATA:MANAGE");
 -    });
 -
 -    client2.invoke(()-> {
 -      Cache cache = SecurityTestUtils.createCacheClient("authRegionManager", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -      assertNotAuthorized(()->region.destroyRegion(), "DATA:MANAGE");
 -    });
 -
 -    client3.invoke(()-> {
 -      Cache cache = SecurityTestUtils.createCacheClient("super-user", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -      region.destroyRegion();
 -    });
 -  }
 -
 -  @Test
 -  public void testDestroyInvalidate() throws InterruptedException {
 -
 -    // Delete one key and invalidate another key with an authorized user.
 -    AsyncInvocation ai1 = client1.invokeAsync(() -> {
 -      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -
 -      assertEquals(region.get("key1"), "value1");
 -      assertTrue(region.containsKey("key1")); // will only be true after we first get it, then it's cached locally
 -
 -      // Destroy key1
 -      region.destroy("key1");
 -      assertFalse(region.containsKey("key1"));
 -
 -      // Invalidate key2
 -      assertNotNull("Value of key2 should not be null", region.get("key2"));
 -      region.invalidate("key2");
 -      assertNull("Value of key2 should have been null", region.get("key2"));
 -
 -    });
 -
 -    // Delete one key and invalidate another key with an unauthorized user.
 -    AsyncInvocation ai2 = client2.invokeAsync(() -> {
 -      Cache cache = SecurityTestUtils.createCacheClient("authRegionReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -
 -      assertEquals(region.get("key3"), "value3");
 -      assertTrue(region.containsKey("key3")); // will only be true after we first get it, then it's cached locally
 -
 -      // Destroy key1
 -      assertNotAuthorized(() -> region.destroy("key3"), "DATA:WRITE:AuthRegion");
 -      assertTrue(region.containsKey("key3"));
 -
 -      // Invalidate key2
 -      assertNotNull("Value of key4 should not be null", region.get("key4"));
 -      assertNotAuthorized(() -> region.invalidate("key4"), "DATA:WRITE:AuthRegion");
 -      assertNotNull("Value of key4 should not be null", region.get("key4"));
 -      cache.close();
 -    });
 -    ai1.join();
 -    ai2.join();
 -    ai1.checkException();
 -    ai2.checkException();
 -  }
 -
 -  public static void assertNotAuthorized(ThrowingCallable shouldRaiseThrowable, String permString) {
 -    assertThatThrownBy(shouldRaiseThrowable).hasMessageContaining(permString);
 -  }
 -
 -  @Test
 -  public void testRegionClear() throws InterruptedException {
 -    // Verify that an unauthorized user can't clear the region
 -    SerializableRunnable clearUnauthorized = new SerializableRunnable() {
 -      @Override
 -      public void run() {
 -        Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -        final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -        assertNotAuthorized(() -> region.clear(), "DATA:WRITE:AuthRegion");
 -      }
 -    };
 -    client1.invoke(clearUnauthorized);
 -
 -    // Verify that an authorized user can clear the region
 -    SerializableRunnable clearAuthorized = new SerializableRunnable() {
 -      @Override
 -      public void run() {
 -        Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
 -        final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
 -        region.clear();
 -      }
 -    };
 -    client2.invoke(clearAuthorized);
 -  }
  
 -}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6a45ca0c/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyInvalidateAuthDistributedTest.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyInvalidateAuthDistributedTest.java
index 0000000,0000000..e78ed62
new file mode 100644
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyInvalidateAuthDistributedTest.java
@@@ -1,0 -1,0 +1,78 @@@
++/*
++ * 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.security;
++
++import static org.junit.Assert.*;
++
++import com.gemstone.gemfire.cache.Cache;
++import com.gemstone.gemfire.cache.Region;
++import com.gemstone.gemfire.test.dunit.AsyncInvocation;
++import com.gemstone.gemfire.test.junit.categories.DistributedTest;
++
++import org.junit.Test;
++import org.junit.experimental.categories.Category;
++
++@Category(DistributedTest.class)
++public class IntegratedClientDestroyInvalidateAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
++
++  @Test
++  public void testDestroyInvalidate() throws InterruptedException {
++
++    // Delete one key and invalidate another key with an authorized user.
++    AsyncInvocation ai1 = client1.invokeAsync(() -> {
++      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++
++      assertEquals(region.get("key1"), "value1");
++      assertTrue(region.containsKey("key1")); // will only be true after we first get it, then it's cached locally
++
++      // Destroy key1
++      region.destroy("key1");
++      assertFalse(region.containsKey("key1"));
++
++      // Invalidate key2
++      assertNotNull("Value of key2 should not be null", region.get("key2"));
++      region.invalidate("key2");
++      assertNull("Value of key2 should have been null", region.get("key2"));
++
++    });
++
++    // Delete one key and invalidate another key with an unauthorized user.
++    AsyncInvocation ai2 = client2.invokeAsync(() -> {
++      Cache cache = SecurityTestUtils.createCacheClient("authRegionReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++
++      assertEquals(region.get("key3"), "value3");
++      assertTrue(region.containsKey("key3")); // will only be true after we first get it, then it's cached locally
++
++      // Destroy key1
++      assertNotAuthorized(() -> region.destroy("key3"), "DATA:WRITE:AuthRegion");
++      assertTrue(region.containsKey("key3"));
++
++      // Invalidate key2
++      assertNotNull("Value of key4 should not be null", region.get("key4"));
++      assertNotAuthorized(() -> region.invalidate("key4"), "DATA:WRITE:AuthRegion");
++      assertNotNull("Value of key4 should not be null", region.get("key4"));
++      cache.close();
++    });
++    ai1.join();
++    ai2.join();
++    ai1.checkException();
++    ai2.checkException();
++  }
++
++}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6a45ca0c/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyRegionAuthDistributedTest.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyRegionAuthDistributedTest.java
index 0000000,0000000..a9cc354
new file mode 100644
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyRegionAuthDistributedTest.java
@@@ -1,0 -1,0 +1,50 @@@
++/*
++ * 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.security;
++
++import com.gemstone.gemfire.cache.Cache;
++import com.gemstone.gemfire.cache.Region;
++import com.gemstone.gemfire.test.junit.categories.DistributedTest;
++
++import org.junit.Test;
++import org.junit.experimental.categories.Category;
++
++@Category(DistributedTest.class)
++public class IntegratedClientDestroyRegionAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
++
++  @Test
++  public void testDestroyRegion() throws InterruptedException {
++    client1.invoke(()-> {
++      Cache cache = SecurityTestUtils.createCacheClient("dataWriter", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++      assertNotAuthorized(()->region.destroyRegion(), "DATA:MANAGE");
++    });
++
++    client2.invoke(()-> {
++      Cache cache = SecurityTestUtils.createCacheClient("authRegionManager", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++      assertNotAuthorized(()->region.destroyRegion(), "DATA:MANAGE");
++    });
++
++    client3.invoke(()-> {
++      Cache cache = SecurityTestUtils.createCacheClient("super-user", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++      region.destroyRegion();
++    });
++  }
++
++}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6a45ca0c/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
index 0000000,0000000..854e2f6
new file mode 100644
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
@@@ -1,0 -1,0 +1,116 @@@
++/*
++ * 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.security;
++
++import static org.junit.Assert.assertEquals;
++
++import java.util.ArrayList;
++import java.util.HashMap;
++import java.util.List;
++import java.util.Map;
++import java.util.Set;
++
++import com.gemstone.gemfire.cache.Cache;
++import com.gemstone.gemfire.cache.Region;
++import com.gemstone.gemfire.test.dunit.AsyncInvocation;
++import com.gemstone.gemfire.test.junit.categories.DistributedTest;
++
++import org.junit.Test;
++import org.junit.experimental.categories.Category;
++
++@Category(DistributedTest.class)
++public class IntegratedClientGetPutAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
++
++  @Test
++  public void testGetPutAuthorization() throws InterruptedException {
++    Map<String, String> allValues = new HashMap<String, String>();
++    allValues.put("key1", "value1");
++    allValues.put("key2", "value2");
++
++    List<String> keys = new ArrayList<>();
++    keys.add("key1");
++    keys.add("key2");
++
++    // client1 connects to server as a user not authorized to do any operations
++    AsyncInvocation ai1 =  client1.invokeAsync(()->{
++      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++
++      assertNotAuthorized(()->region.put("key3", "value3"), "DATA:WRITE:AuthRegion:key3");
++      assertNotAuthorized(()->region.get("key3"), "DATA:READ:AuthRegion:key3");
++
++      //putall
++      assertNotAuthorized(()->region.putAll(allValues), "DATA:WRITE:AuthRegion");
++
++      // not authorized for either keys, get no record back
++      Map keyValues =  region.getAll(keys);
++      assertEquals(0, keyValues.size());
++
++      Set keySet = region.keySet();
++      assertEquals(0, keySet.size());
++    });
++
++
++    // client2 connects to user as a user authorized to use AuthRegion region
++    AsyncInvocation ai2 =  client2.invokeAsync(()->{
++      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++
++      region.put("key3", "value3");
++      assertEquals("value3", region.get("key3"));
++
++      // put all
++      region.putAll(allValues);
++
++      // get all
++      Map keyValues =  region.getAll(keys);
++      assertEquals(2, keyValues.size());
++
++      // keyset
++      Set keySet = region.keySet();
++      assertEquals(3, keySet.size());
++    });
++
++    // client3 connects to user as a user authorized to use key1 in AuthRegion region
++    AsyncInvocation ai3 =  client3.invokeAsync(()->{
++      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++
++      assertNotAuthorized(()->region.put("key2", "value1"), "DATA:WRITE:AuthRegion:key2");
++      assertNotAuthorized(()->region.get("key2"), "DATA:READ:AuthRegion:key2");
++
++      assertNotAuthorized(()->region.putAll(allValues), "DATA:WRITE:AuthRegion");
++
++      // only authorized for one recrod
++      Map keyValues =  region.getAll(keys);
++      assertEquals(1, keyValues.size());
++
++      // keyset
++      Set keySet = region.keySet();
++      assertEquals(1, keySet.size());
++    });
++
++    ai1.join();
++    ai2.join();
++    ai3.join();
++
++    ai1.checkException();
++    ai2.checkException();
++    ai3.checkException();
++  }
++
++}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6a45ca0c/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
index 0000000,0000000..8651a2f
new file mode 100644
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
@@@ -1,0 -1,0 +1,45 @@@
++/*
++ * Licensed to the Apache Software Foundation (ASF) under one or more
++ * contributor license agreements.  See the NOTICE file distributed with
++ * this work for additional information regarding copyright ownership.
++ * The ASF licenses this file to You under the Apache License, Version 2.0
++ * (the "License"); you may not use this file except in compliance with
++ * the License.  You may obtain a copy of the License at
++ *
++ *      http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing, software
++ * distributed under the License is distributed on an "AS IS" BASIS,
++ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
++ * See the License for the specific language governing permissions and
++ * limitations under the License.
++ */
++package com.gemstone.gemfire.security;
++
++import com.gemstone.gemfire.cache.Cache;
++import com.gemstone.gemfire.cache.Region;
++import com.gemstone.gemfire.cache.client.Pool;
++import com.gemstone.gemfire.cache.client.PoolManager;
++import com.gemstone.gemfire.test.junit.categories.DistributedTest;
++
++import org.junit.Test;
++import org.junit.experimental.categories.Category;
++
++@Category(DistributedTest.class)
++public class IntegratedClientQueryAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
++
++  @Test
++  public void testQuery(){
++    client1.invoke(()-> {
++      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++
++      String query = "select * from /AuthRegion";
++      assertNotAuthorized(()->region.query(query), "DATA:READ:AuthRegion");
++
++      Pool pool = PoolManager.find(region);
++      assertNotAuthorized(()->pool.getQueryService().newQuery(query).execute(), "DATA:READ:AuthRegion");
++    });
++  }
++
++}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/6a45ca0c/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegionClearAuthDistributedTest.java
----------------------------------------------------------------------
diff --cc geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegionClearAuthDistributedTest.java
index 0000000,0000000..b344ce1
new file mode 100644
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegionClearAuthDistributedTest.java
@@@ -1,0 -1,0 +1,54 @@@
++/*
++ * 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.security;
++
++import com.gemstone.gemfire.cache.Cache;
++import com.gemstone.gemfire.cache.Region;
++import com.gemstone.gemfire.test.dunit.SerializableRunnable;
++import com.gemstone.gemfire.test.junit.categories.DistributedTest;
++
++import org.junit.Test;
++import org.junit.experimental.categories.Category;
++
++@Category(DistributedTest.class)
++public class IntegratedClientRegionClearAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
++
++  @Test
++  public void testRegionClear() throws InterruptedException {
++    // Verify that an unauthorized user can't clear the region
++    SerializableRunnable clearUnauthorized = new SerializableRunnable() {
++      @Override
++      public void run() {
++        Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++        final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++        assertNotAuthorized(() -> region.clear(), "DATA:WRITE:AuthRegion");
++      }
++    };
++    client1.invoke(clearUnauthorized);
++
++    // Verify that an authorized user can clear the region
++    SerializableRunnable clearAuthorized = new SerializableRunnable() {
++      @Override
++      public void run() {
++        Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
++        final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
++        region.clear();
++      }
++    };
++    client2.invoke(clearAuthorized);
++  }
++}


[29/50] [abbrv] incubator-geode git commit: GEODE-11: Added findValues() to LuceneQuery

Posted by ji...@apache.org.
GEODE-11: Added findValues() to LuceneQuery

Added findValues() to LuceneQuery and added unit and intergration 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/59d471c0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/59d471c0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/59d471c0

Branch: refs/heads/feature/GEODE-1571
Commit: 59d471c051932fb509214024117b8b7c6ccb5d28
Parents: 14437b7
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Mon Jun 20 11:31:07 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri Jun 24 14:41:42 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneQuery.java       |  5 ++
 .../cache/lucene/internal/LuceneQueryImpl.java  | 22 +++++++++
 .../lucene/LuceneQueriesIntegrationTest.java    | 41 ++++++++++------
 .../internal/LuceneQueryImplJUnitTest.java      | 50 +++++++++++++++++---
 4 files changed, 96 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/59d471c0/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
index 9915116..255bd79 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
@@ -34,6 +34,11 @@ public interface LuceneQuery<K, V> {
   public Collection<K> findKeys() throws LuceneQueryException;
 
   /**
+   * Execute search and return values.
+   */
+  public Collection<V> findValues() throws LuceneQueryException;
+
+  /**
    * Execute the search and get results. 
    */
   public PageableLuceneQueryResults<K, V> findPages() throws LuceneQueryException;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/59d471c0/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
index cd49f72..b910f19 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
@@ -21,6 +21,7 @@ package com.gemstone.gemfire.cache.lucene.internal;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.stream.Collectors;
 
@@ -33,6 +34,7 @@ import com.gemstone.gemfire.cache.lucene.LuceneQuery;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
+import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
 import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
@@ -74,12 +76,32 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
   }
 
   @Override
+  public Collection<V> findValues() throws LuceneQueryException {
+    PageableLuceneQueryResults<K, V> pages = findPages(0);
+    final List<LuceneResultStruct<K, V>> page = pages.getNextPage();
+    if(page == null) {
+      return Collections.emptyList();
+    }
+
+    return page.stream()
+      .map(entry -> entry.getValue())
+      .collect(Collectors.toList());
+  }
+
+  @Override
   public PageableLuceneQueryResults<K, V> findPages() throws LuceneQueryException {
+    return findPages(pageSize);
+  }
+
+  private PageableLuceneQueryResults<K, V> findPages(int pageSize) throws LuceneQueryException {
+
     TopEntries entries = findTopEntries();
 
     return new PageableLuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
   }
 
+
+
   private TopEntries findTopEntries() throws LuceneQueryException {
     TopEntriesCollectorManager manager = new TopEntriesCollectorManager(null, limit);
     LuceneFunctionContext<TopEntriesCollector> context = new LuceneFunctionContext<>(query, indexName, manager, limit);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/59d471c0/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index cf1b5d2..d563f4e 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -21,6 +21,7 @@ import static org.junit.Assert.*;
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
@@ -53,6 +54,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   public ExpectedException thrown = ExpectedException.none();
   private static final String INDEX_NAME = "index";
   protected static final String REGION_NAME = "index";
+  private Region region;
 
   @Test()
   public void shouldNotTokenizeWordsWithKeywordAnalyzer() throws Exception {
@@ -117,8 +119,30 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   @Test()
   public void shouldPaginateResults() throws Exception {
 
+    final LuceneQuery<Object, Object> query = addValuesAndCreateQuery();
+
+    final PageableLuceneQueryResults<Object, Object> pages = query.findPages();
+    assertTrue(pages.hasNextPage());
+    assertEquals(3, pages.size());
+    final List<LuceneResultStruct<Object, Object>> page1 = pages.getNextPage();
+    final List<LuceneResultStruct<Object, Object>> page2 = pages.getNextPage();
+    List<LuceneResultStruct<Object, Object>> allEntries=new ArrayList<>();
+    allEntries.addAll(page1);
+    allEntries.addAll(page2);
+
+    assertEquals(region.keySet(), allEntries.stream().map(entry -> entry.getKey()).collect(Collectors.toSet()));
+    assertEquals(region.values(), allEntries.stream().map(entry -> entry.getValue()).collect(Collectors.toSet()));
+
+  }
+  @Test
+  public void shouldReturnValuesFromFindValues() throws Exception {
+    final LuceneQuery<Object, Object> query = addValuesAndCreateQuery();
+    assertEquals(region.values(), new HashSet(query.findValues()));
+  }
+
+  private LuceneQuery<Object, Object> addValuesAndCreateQuery() {
     luceneService.createIndex(INDEX_NAME, REGION_NAME, "field1", "field2");
-    Region region = cache.createRegionFactory(RegionShortcut.PARTITION)
+    region = cache.createRegionFactory(RegionShortcut.PARTITION)
       .create(REGION_NAME);
     final LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
 
@@ -131,23 +155,10 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     region.put("C", new TestObject(value3, value3));
 
     index.waitUntilFlushed(60000);
-    final LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
+    return luceneService.createLuceneQueryFactory()
       .setPageSize(2)
       .create(INDEX_NAME, REGION_NAME,
       "one", "field1");
-
-    final PageableLuceneQueryResults<Object, Object> pages = query.findPages();
-    assertTrue(pages.hasNextPage());
-    assertEquals(3, pages.size());
-    final List<LuceneResultStruct<Object, Object>> page1 = pages.getNextPage();
-    final List<LuceneResultStruct<Object, Object>> page2 = pages.getNextPage();
-    List<LuceneResultStruct<Object, Object>> allEntries=new ArrayList<>();
-    allEntries.addAll(page1);
-    allEntries.addAll(page2);
-
-    assertEquals(region.keySet(), allEntries.stream().map(entry -> entry.getKey()).collect(Collectors.toSet()));
-    assertEquals(region.values(), allEntries.stream().map(entry -> entry.getValue()).collect(Collectors.toSet()));
-
   }
 
   @Test()

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/59d471c0/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
index 1b36e37..7b39e89 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplJUnitTest.java
@@ -28,7 +28,9 @@ import static org.mockito.Mockito.*;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.junit.After;
 import org.junit.Before;
@@ -64,23 +66,20 @@ public class LuceneQueryImplJUnitTest {
   private LuceneQueryImpl<Object, Object> query;
   private Execution execution;
   private LuceneQueryProvider provider;
+  private ResultCollector<TopEntriesCollector, TopEntries> collector;
+  private Region region;
 
   @Before
   public void createMocks() {
-    Region region = mock(Region.class);
+    region = mock(Region.class);
     execution = mock(Execution.class);
-    ResultCollector<TopEntriesCollector, TopEntries> collector = mock(ResultCollector.class);
+    collector = mock(ResultCollector.class);
     provider = mock(LuceneQueryProvider.class);
 
     when(execution.withArgs(any())).thenReturn(execution);
     when(execution.withCollector(any())).thenReturn(execution);
     when(execution.execute(anyString())).thenReturn((ResultCollector) collector);
 
-    TopEntries entries = new TopEntries();
-    entries.addHit(new EntryScore("hi", 5));
-    when(collector.getResult()).thenReturn(entries);
-
-
     query = new LuceneQueryImpl<Object, Object>("index", region, provider, null, LIMIT, 20) {
       @Override protected Execution onRegion() {
         return execution;
@@ -88,14 +87,50 @@ public class LuceneQueryImplJUnitTest {
     };
   }
 
+  private void addValueToResults() {
+    TopEntries entries = new TopEntries();
+    entries.addHit(new EntryScore("hi", 5));
+    when(collector.getResult()).thenReturn(entries);
+
+    Map<String, String> getAllResult = new HashMap<String, String>();
+    getAllResult.put("hi", "value");
+    when(region.getAll(eq(Collections.singletonList("hi"))))
+      .thenReturn(getAllResult);
+  }
+
   @Test
   public void shouldReturnKeysFromFindKeys() throws LuceneQueryException {
+    addValueToResults();
     Collection<Object> results = query.findKeys();
     assertEquals(Collections.singletonList("hi"), results);
   }
 
   @Test
+  public void shouldReturnEmptyListFromFindKeysWithNoResults() throws LuceneQueryException {
+    TopEntries entries = new TopEntries();
+    when(collector.getResult()).thenReturn(entries);
+    Collection<Object> results = query.findKeys();
+    assertEquals(Collections.emptyList(), results);
+  }
+
+  @Test
+  public void shouldReturnValuesFromFindValues() throws LuceneQueryException {
+    addValueToResults();
+    Collection<Object> results = query.findValues();
+    assertEquals(Collections.singletonList("value"), results);
+  }
+
+  @Test
+  public void shouldReturnEmptyListFromFindValuesWithNoResults() throws LuceneQueryException {
+    TopEntries entries = new TopEntries();
+    when(collector.getResult()).thenReturn(entries);
+    Collection<Object> results = query.findValues();
+    assertEquals(Collections.emptyList(), results);
+  }
+
+  @Test
   public void shouldInvokeLuceneFunctionWithCorrectArguments() throws Exception {
+    addValueToResults();
     PageableLuceneQueryResults<Object, Object> results = query.findPages();
 
     verify(execution).execute(eq(LuceneFunction.ID));
@@ -111,6 +146,7 @@ public class LuceneQueryImplJUnitTest {
     assertEquals(1, page.size());
     LuceneResultStruct element = page.iterator().next();
     assertEquals("hi", element.getKey());
+    assertEquals("value", element.getValue());
     assertEquals(5, element.getScore(), 0.01);
   }
 }


[26/50] [abbrv] incubator-geode git commit: GEODE-11 Renamed LuceneQueryResults and search

Posted by ji...@apache.org.
GEODE-11 Renamed LuceneQueryResults and search

Renamed LuceneQueryResults to PageableLuceneQueryResults and search to findPages.


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

Branch: refs/heads/feature/GEODE-1571
Commit: d0e61ecd59d33e628ebe340fd6835d6b4656997c
Parents: 87876a7
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Fri Jun 17 12:01:28 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri Jun 24 14:41:41 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneQuery.java       |   2 +-
 .../cache/lucene/LuceneQueryResults.java        |  57 ---------
 .../gemfire/cache/lucene/LuceneService.java     |   2 +-
 .../lucene/PageableLuceneQueryResults.java      |  57 +++++++++
 .../cache/lucene/internal/LuceneQueryImpl.java  |   6 +-
 .../lucene/internal/LuceneQueryResultsImpl.java | 120 ------------------
 .../PageableLuceneQueryResultsImpl.java         | 120 ++++++++++++++++++
 ...IndexCreationPersistenceIntegrationTest.java |  11 +-
 .../LuceneIndexMaintenanceIntegrationTest.java  |  12 +-
 .../gemfire/cache/lucene/LuceneQueriesBase.java |   4 +-
 .../lucene/LuceneQueriesIntegrationTest.java    |   2 +-
 ...LuceneQueriesPersistenceIntegrationTest.java |   2 +-
 .../LuceneQueryImplIntegrationTest.java         |   4 +-
 .../LuceneQueryResultsImplJUnitTest.java        | 122 -------------------
 ...PageableLuceneQueryResultsImplJUnitTest.java | 122 +++++++++++++++++++
 .../IndexRepositoryImplPerformanceTest.java     |   2 +-
 .../cache/lucene/test/LuceneTestUtilities.java  |  10 +-
 17 files changed, 324 insertions(+), 331 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
index 93426b9..be47677 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
@@ -29,7 +29,7 @@ public interface LuceneQuery<K, V> {
   /**
    * Execute the search and get results. 
    */
-  public LuceneQueryResults<K, V> search() throws LuceneQueryException;
+  public PageableLuceneQueryResults<K, V> findPages() throws LuceneQueryException;
   
   /**
    * Get page size setting of current query. 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryResults.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryResults.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryResults.java
deleted file mode 100644
index b49a159..0000000
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryResults.java
+++ /dev/null
@@ -1,57 +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.cache.lucene;
-
-import java.util.List;
-
-import com.gemstone.gemfire.annotations.Experimental;
-
-/**
- * <p>
- * Defines the interface for a container of lucene query result collected from function execution.<br>
- * 
- * 
- * @param <K> The type of the key
- * @param <V> The type of the value
- */
-@Experimental
-public interface LuceneQueryResults<K, V> {
-  /**
-   * @return total number of hits for this query
-   */
-  public int size();
-
-  /**
-   * Returns the maximum score value encountered. Note that in case scores are not tracked, this returns {@link Float#NaN}.
-   */
-  public float getMaxScore();
-
-  /**
-   * Get the next page of results.
-   * 
-   * @return a page of results, or null if there are no more pages
-   */
-  public List<LuceneResultStruct<K, V>> getNextPage();
-
-  /**
-   *  True if there another page of results. 
-   */
-  public boolean hasNextPage();
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
index 974cf96..8a94959 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneService.java
@@ -52,7 +52,7 @@ import com.gemstone.gemfire.cache.GemFireCache;
  * 
  * The querystring is using lucene's queryparser syntax, such as "field1:zhou* AND field2:gzhou@pivotal.io"
  *  
- * LuceneQueryResults results = query.search();
+ * PageableLuceneQueryResults results = query.search();
  * 
  * If pagination is not specified:
  * List list = results.getNextPage(); // return all results in one getNextPage() call

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/PageableLuceneQueryResults.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/PageableLuceneQueryResults.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/PageableLuceneQueryResults.java
new file mode 100644
index 0000000..ef229da
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/PageableLuceneQueryResults.java
@@ -0,0 +1,57 @@
+/*
+ * 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.cache.lucene;
+
+import java.util.List;
+
+import com.gemstone.gemfire.annotations.Experimental;
+
+/**
+ * <p>
+ * Defines the interface for a container of lucene query result collected from function execution.<br>
+ * 
+ * 
+ * @param <K> The type of the key
+ * @param <V> The type of the value
+ */
+@Experimental
+public interface PageableLuceneQueryResults<K, V> {
+  /**
+   * @return total number of hits for this query
+   */
+  public int size();
+
+  /**
+   * Returns the maximum score value encountered. Note that in case scores are not tracked, this returns {@link Float#NaN}.
+   */
+  public float getMaxScore();
+
+  /**
+   * Get the next page of results.
+   * 
+   * @return a page of results, or null if there are no more pages
+   */
+  public List<LuceneResultStruct<K, V>> getNextPage();
+
+  /**
+   *  True if there another page of results. 
+   */
+  public boolean hasNextPage();
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
index 92f1957..44fa446 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
@@ -27,7 +27,7 @@ import com.gemstone.gemfire.cache.lucene.LuceneQuery;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
-import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.TopEntries;
@@ -57,7 +57,7 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
   }
 
   @Override
-  public LuceneQueryResults<K, V> search() throws LuceneQueryException {
+  public PageableLuceneQueryResults<K, V> findPages() throws LuceneQueryException {
     TopEntriesCollectorManager manager = new TopEntriesCollectorManager(null, limit);
     LuceneFunctionContext<TopEntriesCollector> context = new LuceneFunctionContext<>(query, indexName, manager, limit);
     TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(context);
@@ -70,7 +70,7 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
     //TODO provide a timeout to the user?
     try {
       TopEntries entries = rc.getResult();
-      return new LuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
+      return new PageableLuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
     } catch(FunctionException e) {
       if(e.getCause() instanceof LuceneQueryException) {
         throw new LuceneQueryException(e);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImpl.java
deleted file mode 100644
index f0e98c8..0000000
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImpl.java
+++ /dev/null
@@ -1,120 +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.cache.lucene.internal;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
-import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
-
-/**
- * Implementation of LuceneQueryResults that fetchs a page at a time
- * from the server, given a set of EntryScores (key and score).
- *
- * @param <K> The type of the key
- * @param <V> The type of the value
- */
-public class LuceneQueryResultsImpl<K,V> implements LuceneQueryResults<K,V> {
-
-  /**
-   *  list of docs matching search query
-   */
-  private final List<EntryScore> hits;
-  
-  /**
-   * The maximum score. Lazily evaluated
-   */
-  private float maxScore = Float.MIN_VALUE;
-  
-  /**
-   * The user region where values are stored.
-   */
-  private final Region<K, V> userRegion;
-  
-  /**
-   * The start of the next page of results we want to fetch 
-   */
-  private int currentHit = 0;
-  
-  /**
-   * The page size the user wants.
-   */
-  private int pageSize;
-  
-  public LuceneQueryResultsImpl(List<EntryScore> hits, Region<K,V> userRegion, int pageSize) {
-    this.hits = hits;
-    this.userRegion = userRegion;
-    this.pageSize = pageSize == 0 ? Integer.MAX_VALUE : pageSize;
-  }
-
-  @Override
-  public List<LuceneResultStruct<K,V>> getNextPage() {
-    if(!hasNextPage()) {
-      return null;
-    }
-    
-    int end = currentHit + pageSize;
-    end = end > hits.size() ? hits.size() : end;
-    List<EntryScore> scores = hits.subList(currentHit, end);
-    
-    ArrayList<K> keys = new ArrayList<K>(hits.size());
-    for(EntryScore score : scores) {
-      keys.add((K) score.getKey());
-    }
-    
-    Map<K,V> values = userRegion.getAll(keys);
-    
-    ArrayList<LuceneResultStruct<K,V>> results = new ArrayList<LuceneResultStruct<K,V>>(hits.size());
-    for(EntryScore score : scores) {
-      V value = values.get(score.getKey());
-      results.add(new LuceneResultStructImpl(score.getKey(), value, score.getScore()));
-    }
-    
-
-    currentHit = end;
-    
-    return results;
-  }
-
-  @Override
-  public boolean hasNextPage() {
-    return hits.size() > currentHit;
-  }
-
-  @Override
-  public int size() {
-    return hits.size();
-  }
-
-  @Override
-  public float getMaxScore() {
-    if(maxScore == Float.MIN_VALUE) {
-      for(EntryScore score : hits) {
-        maxScore = Math.max(maxScore, score.getScore());
-      }
-    }
-    
-    return maxScore;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
new file mode 100644
index 0000000..dc2d340
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImpl.java
@@ -0,0 +1,120 @@
+/*
+ * 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.cache.lucene.internal;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
+
+/**
+ * Implementation of PageableLuceneQueryResults that fetchs a page at a time
+ * from the server, given a set of EntryScores (key and score).
+ *
+ * @param <K> The type of the key
+ * @param <V> The type of the value
+ */
+public class PageableLuceneQueryResultsImpl<K,V> implements PageableLuceneQueryResults<K,V> {
+
+  /**
+   *  list of docs matching search query
+   */
+  private final List<EntryScore> hits;
+  
+  /**
+   * The maximum score. Lazily evaluated
+   */
+  private float maxScore = Float.MIN_VALUE;
+  
+  /**
+   * The user region where values are stored.
+   */
+  private final Region<K, V> userRegion;
+  
+  /**
+   * The start of the next page of results we want to fetch 
+   */
+  private int currentHit = 0;
+  
+  /**
+   * The page size the user wants.
+   */
+  private int pageSize;
+  
+  public PageableLuceneQueryResultsImpl(List<EntryScore> hits, Region<K,V> userRegion, int pageSize) {
+    this.hits = hits;
+    this.userRegion = userRegion;
+    this.pageSize = pageSize == 0 ? Integer.MAX_VALUE : pageSize;
+  }
+
+  @Override
+  public List<LuceneResultStruct<K,V>> getNextPage() {
+    if(!hasNextPage()) {
+      return null;
+    }
+    
+    int end = currentHit + pageSize;
+    end = end > hits.size() ? hits.size() : end;
+    List<EntryScore> scores = hits.subList(currentHit, end);
+    
+    ArrayList<K> keys = new ArrayList<K>(hits.size());
+    for(EntryScore score : scores) {
+      keys.add((K) score.getKey());
+    }
+    
+    Map<K,V> values = userRegion.getAll(keys);
+    
+    ArrayList<LuceneResultStruct<K,V>> results = new ArrayList<LuceneResultStruct<K,V>>(hits.size());
+    for(EntryScore score : scores) {
+      V value = values.get(score.getKey());
+      results.add(new LuceneResultStructImpl(score.getKey(), value, score.getScore()));
+    }
+    
+
+    currentHit = end;
+    
+    return results;
+  }
+
+  @Override
+  public boolean hasNextPage() {
+    return hits.size() > currentHit;
+  }
+
+  @Override
+  public int size() {
+    return hits.size();
+  }
+
+  @Override
+  public float getMaxScore() {
+    if(maxScore == Float.MIN_VALUE) {
+      for(EntryScore score : hits) {
+        maxScore = Math.max(maxScore, score.getScore());
+      }
+    }
+    
+    return maxScore;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
index ca9f4f1..f2fd67c 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
@@ -24,16 +24,13 @@ import static junitparams.JUnitParamsRunner.*;
 import static org.junit.Assert.*;
 
 import java.io.File;
-import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 
-import junit.framework.AssertionFailedError;
-
 import com.jayway.awaitility.Awaitility;
 import junitparams.JUnitParamsRunner;
 import junitparams.Parameters;
-import org.apache.lucene.queryparser.classic.ParseException;
+
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -112,7 +109,7 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
     LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
       .create(INDEX_NAME, REGION_NAME,
         "field1:world", DEFAULT_FIELD);
-    assertEquals(1, query.search().size());
+    assertEquals(1, query.findPages().size());
   }
 
   @Test
@@ -130,7 +127,7 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
     LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory()
       .create(INDEX_NAME, REGION_NAME,
       "field1:world", DEFAULT_FIELD);
-    assertEquals(1, query.search().size());
+    assertEquals(1, query.findPages().size());
   }
 
   @Test
@@ -166,7 +163,7 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
     LuceneQuery query = luceneService.createLuceneQueryFactory().create(indexName, regionName, queryString, defaultField);
     Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
       try {
-        assertEquals(size, query.search().size());
+        assertEquals(size, query.findPages().size());
       } catch(LuceneQueryException e) {
         throw new RuntimeException(e);
       }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
index 352c6b9..d0c91e5 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexMaintenanceIntegrationTest.java
@@ -56,14 +56,14 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
     LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
     index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
     LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "description:\"hello world\"", DEFAULT_FIELD);
-    LuceneQueryResults<Integer, TestObject> results = query.search();
+    PageableLuceneQueryResults<Integer, TestObject> results = query.findPages();
     assertEquals(3, results.size());
 
     //begin transaction
     cache.getCacheTransactionManager().begin();
     region.put("object-1", new TestObject("title 1", "updated"));
     index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
-    assertEquals(3, query.search().size());
+    assertEquals(3, query.findPages().size());
   }
 
   @Test
@@ -79,7 +79,7 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
     LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
     index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
     LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "description:\"hello world\"", DEFAULT_FIELD);
-    LuceneQueryResults<Integer, TestObject> results = query.search();
+    PageableLuceneQueryResults<Integer, TestObject> results = query.findPages();
     assertEquals(3, results.size());
 
     cache.getCacheTransactionManager().begin();
@@ -87,7 +87,7 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
     cache.getCacheTransactionManager().commit();
     index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
 
-    assertEquals(2, query.search().size());
+    assertEquals(2, query.findPages().size());
   }
 
   @Test
@@ -103,7 +103,7 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
     LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME);
     index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
     LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "description:\"hello world\"", DEFAULT_FIELD);
-    LuceneQueryResults<Integer, TestObject> results = query.search();
+    PageableLuceneQueryResults<Integer, TestObject> results = query.findPages();
     assertEquals(3, results.size());
 
     cache.getCacheTransactionManager().begin();
@@ -111,7 +111,7 @@ public class LuceneIndexMaintenanceIntegrationTest extends LuceneIntegrationTest
     cache.getCacheTransactionManager().rollback();
     index.waitUntilFlushed(WAIT_FOR_FLUSH_TIME);
 
-    assertEquals(3, query.search().size());
+    assertEquals(3, query.findPages().size());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
index f077bad..77ce0bb 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
@@ -150,7 +150,7 @@ public abstract class LuceneQueriesBase extends LuceneDUnitTest {
       LuceneService service = LuceneServiceProvider.get(cache);
       LuceneQuery<Integer, TestObject> query;
       query = service.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "text:world", DEFAULT_FIELD);
-      LuceneQueryResults<Integer, TestObject> results = query.search();
+      PageableLuceneQueryResults<Integer, TestObject> results = query.findPages();
       assertEquals(3, results.size());
       List<LuceneResultStruct<Integer, TestObject>> page = results.getNextPage();
 
@@ -171,7 +171,7 @@ public abstract class LuceneQueriesBase extends LuceneDUnitTest {
       LuceneService service = LuceneServiceProvider.get(cache);
       LuceneQuery<Integer, TestObject> query;
       query = service.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, queryString, defaultField);
-      LuceneQueryResults<Integer, TestObject> results = query.search();
+      PageableLuceneQueryResults<Integer, TestObject> results = query.findPages();
       assertEquals(results.size(), expectedResultsSize);
     });
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index 7aa4cf5..dbb92cf 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -206,7 +206,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
 
 
     thrown.expect(LuceneQueryException.class);
-    query.search();
+    query.findPages();
   }
   
   private PdxInstance insertAJson(Region region, String key) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
index 6eb222d..fcd748a 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesPersistenceIntegrationTest.java
@@ -96,7 +96,7 @@ public class LuceneQueriesPersistenceIntegrationTest extends LuceneIntegrationTe
     Assert.assertTrue(0 < userRegion.getDiskRegionStats().getNumOverflowOnDisk());
 
     LuceneQuery<Integer, Type1> query = service.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "s:world", DEFAULT_FIELD);
-    LuceneQueryResults<Integer, Type1> results = query.search();
+    PageableLuceneQueryResults<Integer, Type1> results = query.findPages();
     Assert.assertEquals(3, results.size());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
index f748b79..62f4623 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
@@ -36,7 +36,7 @@ import com.gemstone.gemfire.cache.execute.FunctionAdapter;
 import com.gemstone.gemfire.cache.execute.FunctionContext;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.lucene.LuceneIntegrationTest;
-import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
+import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunction;
 import com.gemstone.gemfire.cache.lucene.internal.distributed.LuceneFunctionContext;
@@ -69,7 +69,7 @@ public class LuceneQueryImplIntegrationTest extends LuceneIntegrationTest {
 
     StringQueryProvider provider = new StringQueryProvider(null, DEFAULT_FIELD);
     LuceneQueryImpl<Object, Object> query = new LuceneQueryImpl<>("index", region, provider, null, LIMIT, 20);
-    LuceneQueryResults<Object, Object> results = query.search();
+    PageableLuceneQueryResults<Object, Object> results = query.findPages();
 
     assertTrue(function.wasInvoked);
     assertEquals(2f * LIMIT, results.getMaxScore(), 0.01);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImplJUnitTest.java
deleted file mode 100644
index 9c3a65e..0000000
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryResultsImplJUnitTest.java
+++ /dev/null
@@ -1,122 +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.cache.lucene.internal;
-
-import static org.junit.Assert.*;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-import com.gemstone.gemfire.cache.Region;
-import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
-import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class LuceneQueryResultsImplJUnitTest {
-
-  private List<EntryScore> hits;
-  private List<LuceneResultStruct> expected = new ArrayList<LuceneResultStruct>();
-  private Region<String, String> userRegion;
-  
-  @Before
-  public void setUp() {
-    hits = new ArrayList<EntryScore>();
-    
-    for(int i =0; i < 23; i++) {
-      hits.add(new EntryScore("key_" + i, i));
-      expected.add(new LuceneResultStructImpl<String, String>("key_" + i, "value_" + i, i));
-    }
-    
-    userRegion = Mockito.mock(Region.class);
-    
-    Mockito.when(userRegion.getAll(Mockito.anyCollection())).thenAnswer(new Answer() {
-
-      @Override
-      public Map answer(InvocationOnMock invocation) throws Throwable {
-        Collection<String> keys = invocation.getArgumentAt(0, Collection.class);
-        Map<String, String> results = new HashMap<String, String>();
-        for(String key : keys) {
-          results.put(key, key.replace("key_", "value_"));
-        }
-        
-        return results;
-      }
-    });
-  }
-  
-  @Test
-  public void testMaxStore() {
-    hits.set(5, new EntryScore("key_5", 502));
-    
-    LuceneQueryResultsImpl<String, String> results = new LuceneQueryResultsImpl<String, String>(hits, null, 5);
-    
-    assertEquals(502, results.getMaxScore(), 0.1f);
-  }
-  
-  @Test
-  public void testPagination() {
-    LuceneQueryResultsImpl<String, String> results = new LuceneQueryResultsImpl<String, String>(hits, userRegion, 10);
-    
-    assertEquals(23, results.size());
-    
-    assertTrue(results.hasNextPage());
-    
-    List<LuceneResultStruct<String, String>> next  = results.getNextPage();
-    assertEquals(expected.subList(0, 10), next);
-    
-    assertTrue(results.hasNextPage());
-    next  = results.getNextPage();
-    assertEquals(expected.subList(10, 20), next);
-    
-    assertTrue(results.hasNextPage());
-    next  = results.getNextPage();
-    assertEquals(expected.subList(20, 23), next);
-    
-    
-    assertFalse(results.hasNextPage());
-    assertNull(results.getNextPage());
-  }
-  
-  @Test
-  public void testNoPagination() {
-    LuceneQueryResultsImpl<String, String> results = new LuceneQueryResultsImpl<String, String>(hits, userRegion, 0);
-    
-    assertEquals(23, results.size());
-    
-    assertTrue(results.hasNextPage());
-    
-    List<LuceneResultStruct<String, String>> next  = results.getNextPage();
-    assertEquals(expected, next);
-    
-    assertFalse(results.hasNextPage());
-    assertNull(results.getNextPage());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
new file mode 100644
index 0000000..5047d0d
--- /dev/null
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/PageableLuceneQueryResultsImplJUnitTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.cache.lucene.internal;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
+import com.gemstone.gemfire.cache.lucene.internal.distributed.EntryScore;
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class PageableLuceneQueryResultsImplJUnitTest {
+
+  private List<EntryScore> hits;
+  private List<LuceneResultStruct> expected = new ArrayList<LuceneResultStruct>();
+  private Region<String, String> userRegion;
+  
+  @Before
+  public void setUp() {
+    hits = new ArrayList<EntryScore>();
+    
+    for(int i =0; i < 23; i++) {
+      hits.add(new EntryScore("key_" + i, i));
+      expected.add(new LuceneResultStructImpl<String, String>("key_" + i, "value_" + i, i));
+    }
+    
+    userRegion = Mockito.mock(Region.class);
+    
+    Mockito.when(userRegion.getAll(Mockito.anyCollection())).thenAnswer(new Answer() {
+
+      @Override
+      public Map answer(InvocationOnMock invocation) throws Throwable {
+        Collection<String> keys = invocation.getArgumentAt(0, Collection.class);
+        Map<String, String> results = new HashMap<String, String>();
+        for(String key : keys) {
+          results.put(key, key.replace("key_", "value_"));
+        }
+        
+        return results;
+      }
+    });
+  }
+  
+  @Test
+  public void testMaxStore() {
+    hits.set(5, new EntryScore("key_5", 502));
+    
+    PageableLuceneQueryResultsImpl<String, String> results = new PageableLuceneQueryResultsImpl<String, String>(hits, null, 5);
+    
+    assertEquals(502, results.getMaxScore(), 0.1f);
+  }
+  
+  @Test
+  public void testPagination() {
+    PageableLuceneQueryResultsImpl<String, String> results = new PageableLuceneQueryResultsImpl<String, String>(hits, userRegion, 10);
+    
+    assertEquals(23, results.size());
+    
+    assertTrue(results.hasNextPage());
+    
+    List<LuceneResultStruct<String, String>> next  = results.getNextPage();
+    assertEquals(expected.subList(0, 10), next);
+    
+    assertTrue(results.hasNextPage());
+    next  = results.getNextPage();
+    assertEquals(expected.subList(10, 20), next);
+    
+    assertTrue(results.hasNextPage());
+    next  = results.getNextPage();
+    assertEquals(expected.subList(20, 23), next);
+    
+    
+    assertFalse(results.hasNextPage());
+    assertNull(results.getNextPage());
+  }
+  
+  @Test
+  public void testNoPagination() {
+    PageableLuceneQueryResultsImpl<String, String> results = new PageableLuceneQueryResultsImpl<String, String>(hits, userRegion, 0);
+    
+    assertEquals(23, results.size());
+    
+    assertTrue(results.hasNextPage());
+    
+    List<LuceneResultStruct<String, String>> next  = results.getNextPage();
+    assertEquals(expected, next);
+    
+    assertFalse(results.hasNextPage());
+    assertNull(results.getNextPage());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
index 61f0ec4..3155aaf 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
@@ -196,7 +196,7 @@ public class IndexRepositoryImplPerformanceTest {
           }
         });
         
-        LuceneQueryResults<Object, Object> results = luceneQuery.search();
+        PageableLuceneQueryResults<Object, Object> results = luceneQuery.findPages();
         return results.size();
       }
     });

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/d0e61ecd/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
index c1f540d..2084e11 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -25,8 +25,6 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.function.Consumer;
-import java.util.stream.Collectors;
-import org.apache.logging.log4j.Logger;
 
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
@@ -34,14 +32,12 @@ import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
-import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
-import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
+import com.gemstone.gemfire.cache.lucene.PageableLuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
 import com.gemstone.gemfire.cache.lucene.internal.LuceneIndexForPartitionedRegion;
 import com.gemstone.gemfire.cache.lucene.internal.LuceneServiceImpl;
 import com.gemstone.gemfire.internal.cache.LocalRegion;
-import com.gemstone.gemfire.internal.logging.LogService;
 import com.gemstone.gemfire.pdx.JSONFormatter;
 import com.gemstone.gemfire.pdx.PdxInstance;
 
@@ -94,7 +90,7 @@ public class LuceneTestUtilities {
   public static <K> void  verifyQueryKeys(LuceneQuery<K,Object> query,K ... expectedKeys) throws LuceneQueryException {
     Set<K> expectedKeySet = new HashSet<>(Arrays.asList(expectedKeys));
     Set<K> actualKeySet = new HashSet<>();
-    final LuceneQueryResults<K, Object> results = query.search();
+    final PageableLuceneQueryResults<K, Object> results = query.findPages();
     while(results.hasNextPage()) {
       results.getNextPage().stream()
         .forEach(struct -> actualKeySet.add(struct.getKey()));
@@ -107,7 +103,7 @@ public class LuceneTestUtilities {
    */
   public static <K> void verifyQueryKeyAndValues(LuceneQuery<K,Object> query, HashMap expectedResults) throws LuceneQueryException {
     HashMap actualResults = new HashMap<>();
-    final LuceneQueryResults<K, Object> results = query.search();
+    final PageableLuceneQueryResults<K, Object> results = query.findPages();
     while(results.hasNextPage()) {
       results.getNextPage().stream()
         .forEach(struct -> {



[06/50] [abbrv] incubator-geode git commit: GEODE-1565: add quotes around any value starting with hyphen

Posted by ji...@apache.org.
GEODE-1565: add quotes around any value starting with hyphen

* refactor OptionJFormatter into HyphenFormatter to format all values starting with hypen


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

Branch: refs/heads/feature/GEODE-1571
Commit: 170919bcc33dcb48d931b5c4ad707df7eec4a2f0
Parents: de6105f
Author: Kirk Lund <kl...@pivotal.io>
Authored: Tue Jun 21 10:22:29 2016 -0700
Committer: Kirk Lund <kl...@pivotal.io>
Committed: Tue Jun 21 10:24:09 2016 -0700

----------------------------------------------------------------------
 .../management/internal/cli/GfshParser.java     |   4 +-
 .../cli/parser/jopt/JoptOptionParser.java       |   4 +-
 .../internal/cli/remote/CommandProcessor.java   |   1 -
 .../internal/cli/util/HyphenFormatter.java      |  92 +++++++++
 .../internal/cli/util/OptionJFormatter.java     | 115 -----------
 .../internal/cli/GfshParserIntegrationTest.java |  38 ++++
 .../commands/GemfireDataCommandsDUnitTest.java  |  42 +++-
 .../internal/cli/util/HyphenFormatterTest.java  | 201 +++++++++++++++++++
 .../internal/cli/util/OptionJFormatterTest.java | 189 -----------------
 .../cli/commands/CommandOverHttpDUnitTest.java  |   2 +-
 10 files changed, 367 insertions(+), 321 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/170919bc/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
index 0bf2f66..9023ebe 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
@@ -893,7 +893,7 @@ public class GfshParser implements Parser {
           preConfigureConverters(commandTarget);
 
           try {
-            // TODO: next call invokes OptionJFormatter
+            // TODO: next call invokes HyphenFormatter
             parse = commandTarget.getOptionParser().parse(gfshMethodTarget.getRemainingBuffer());
           } catch (CliException ce) {
             if (ce instanceof CliCommandOptionException) {
@@ -925,7 +925,7 @@ public class GfshParser implements Parser {
           } else {
             if (coe != null) {
               logWrapper.fine("Handling exception: " + coe.getMessage());
-              ExceptionHandler.handleException(coe);
+              ExceptionHandler.handleException(coe); // TODO: this eats exception that would make it easier to debug GemfireDataCommandsDUnitTest
               // ExceptionHandler.handleException() only logs it on console.
               // When on member, we need to handle this.
               if (!CliUtil.isGfshVM()) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/170919bc/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/parser/jopt/JoptOptionParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/parser/jopt/JoptOptionParser.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/parser/jopt/JoptOptionParser.java
index 04590ed..de47ce6 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/parser/jopt/JoptOptionParser.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/parser/jopt/JoptOptionParser.java
@@ -39,7 +39,7 @@ import com.gemstone.gemfire.management.internal.cli.parser.SyntaxConstants;
 import com.gemstone.gemfire.management.internal.cli.parser.preprocessor.Preprocessor;
 import com.gemstone.gemfire.management.internal.cli.parser.preprocessor.PreprocessorUtils;
 import com.gemstone.gemfire.management.internal.cli.parser.preprocessor.TrimmedInput;
-import com.gemstone.gemfire.management.internal.cli.util.OptionJFormatter;
+import com.gemstone.gemfire.management.internal.cli.util.HyphenFormatter;
 
 /**
  * Implementation of {@link GfshOptionParser} which internally makes use of
@@ -115,7 +115,7 @@ public class JoptOptionParser implements GfshOptionParser {
     optionSet.setUserInput(userInput!=null?userInput.trim():"");
     if (userInput != null) {
       TrimmedInput input = PreprocessorUtils.trim(userInput);
-      String[] preProcessedInput = preProcess(new OptionJFormatter().formatCommand(input.getString()));
+      String[] preProcessedInput = preProcess(new HyphenFormatter().formatCommand(input.getString()));
       joptsimple.OptionSet joptOptionSet = null;
       CliCommandOptionException ce = null;
       // int factor = 0;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/170919bc/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
index b62f922..790dd6c 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/remote/CommandProcessor.java
@@ -30,7 +30,6 @@ import com.gemstone.gemfire.management.internal.cli.GfshParser;
 import com.gemstone.gemfire.management.internal.cli.LogWrapper;
 import com.gemstone.gemfire.management.internal.cli.result.ResultBuilder;
 import com.gemstone.gemfire.management.internal.cli.util.CommentSkipHelper;
-import com.gemstone.gemfire.management.internal.cli.util.OptionJFormatter;
 import com.gemstone.gemfire.management.internal.security.ResourceOperation;
 import com.gemstone.gemfire.security.NotAuthorizedException;
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/170919bc/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/HyphenFormatter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/HyphenFormatter.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/HyphenFormatter.java
new file mode 100644
index 0000000..0fb4fcd
--- /dev/null
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/HyphenFormatter.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.management.internal.cli.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * Inserts quotes around the values of any option values that begin with hyphen.
+ */
+public class HyphenFormatter {
+
+  private static final String OPTION_PATTERN = "\\-\\-[a-zA-Z]+\\-?[a-zA-Z]*\\=";
+
+  private static final String QUOTE = "\"";
+  private static final String EQUAL_HYPHEN = "=-";
+  private static final String EQUAL = "=";
+  private static final String SPACE = " ";
+
+  private StringBuilder formatted;
+
+  /**
+   * Returns command with quotes around the values of any option values that begin with hyphen.
+   */
+  public String formatCommand(String command){
+    if (!containsOption(command)) {
+      return command;
+    }
+    this.formatted = new StringBuilder();
+
+    List<String> strings = split(command);
+    for (String string : strings) {
+      if (string.contains(EQUAL_HYPHEN)) {
+        int indexOfEquals = string.indexOf(EQUAL);
+        formatted.append(string.substring(0, indexOfEquals + 1));
+        formatted.append(QUOTE);
+        formatted.append(string.substring(indexOfEquals + 1));
+        formatted.append(QUOTE);
+      } else {
+        formatted.append(string);
+      }
+      formatted.append(SPACE);
+    }
+    return formatted.toString().trim();
+  }
+
+  /**
+   * Returns true if command contains any options.
+   */
+  boolean containsOption(String cmd) {
+    return Pattern.compile(OPTION_PATTERN).matcher(cmd).find();
+  }
+
+  private List<String> split(String cmd) {
+    List<String> strings = new ArrayList<>();
+
+    Matcher matcher = Pattern.compile(OPTION_PATTERN).matcher(cmd);
+
+    int index = 0; // first index of --option=
+
+    while (matcher.find()) {
+      if (matcher.start() - index > 0) {
+        String option = cmd.substring(index, matcher.start()).trim();
+        strings.add(option);
+      }
+      index = matcher.start();
+    }
+
+    String lastOne = cmd.substring(index);
+    if (lastOne != null) {
+      strings.add(lastOne);
+    }
+
+    return strings;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/170919bc/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatter.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatter.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatter.java
deleted file mode 100644
index 0e3265a..0000000
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatter.java
+++ /dev/null
@@ -1,115 +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.management.internal.cli.util;
-
-/**
- * Inserts quotes around the values of any --J options.
- */
-public class OptionJFormatter {
-
-  private static final String J_OPTION = "--J=";
-  private static final char QUOTE = '\"';
-  private static final char SPACE = ' ';
-
-  private boolean quotesOpened;
-  private int previousSpace;
-  private String command;
-  private StringBuilder formatted;
-
-  /**
-   * Returns command with quotes inserted around the values of any --J options.
-   */
-  public String formatCommand(String command){
-    if (!containsJopt(command)) {
-      return command;
-    }
-    this.command = command;
-    this.formatted = new StringBuilder();
-    quotesOpened = false;
-
-    int nextJ = this.command.indexOf(J_OPTION);
-
-    while (nextJ > -1) {
-      String stringBeforeJ = this.command.substring(0, nextJ+4);
-      if (quotesOpened && stringBeforeJ.contains("--")){
-        previousSpace = stringBeforeJ.indexOf("--") - 1;
-        while (stringBeforeJ.charAt(previousSpace) == SPACE){
-          previousSpace--;
-        }
-        stringBeforeJ = stringBeforeJ.substring(0,previousSpace + 1) + QUOTE + stringBeforeJ.substring(previousSpace + 1);
-        quotesOpened = false;
-      }
-
-      this.command = this.command.substring(nextJ+4);
-
-      this.formatted.append(stringBeforeJ);
-      if (!this.command.startsWith(""+QUOTE)){
-        this.formatted.append(QUOTE);
-        quotesOpened = true;
-      }
-      quotesOpened = true;
-
-      int nextSpace = this.command.indexOf(SPACE);
-      String stringAfterJ = null;
-      if (nextSpace > -1) {
-        stringAfterJ = this.command.substring(0, nextSpace);
-        this.command = this.command.substring(nextSpace);
-      } else {
-        stringAfterJ = this.command.substring(0);
-        this.command = "";
-      }
-
-      this.formatted.append(stringAfterJ);
-      if (stringAfterJ.endsWith("\"")){
-        quotesOpened = false;
-      }
-
-      nextSpace = this.command.indexOf(SPACE);
-
-      if (nextSpace == -1) {
-        if (!stringAfterJ.endsWith("" + QUOTE)) {
-          this.formatted.append(QUOTE);
-          quotesOpened = false;
-        }
-      } else if (!this.formatted.toString().endsWith(""+QUOTE)) {
-        if(this.command.startsWith(" --")){
-          this.formatted.append(QUOTE);
-          quotesOpened = false;
-        }
-      }
-
-      if (!containsJopt(this.command)){
-        this.formatted.append(this.command);
-      }
-
-      nextJ = this.command.indexOf(J_OPTION);
-    }
-
-    return formatted.toString();
-  }
-
-  /**
-   * Returns true if command contains any --J options.
-   */
-  boolean containsJopt(String cmd){
-    if (cmd.contains("--J")){
-      return true;
-    }
-    return false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/170919bc/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserIntegrationTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserIntegrationTest.java
index 17e78a5..b8f0d10 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserIntegrationTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/GfshParserIntegrationTest.java
@@ -63,6 +63,44 @@ public class GfshParserIntegrationTest {
   }
 
   @Test
+  public void optionStartsWithHyphenWithoutQuotes() throws Exception {
+    String input = "rebalance --exclude-region=/GemfireDataCommandsDUnitTestRegion2 --simulate=true --time-out=-1";
+    Map<String, String> params = params(input, "rebalance", "rebalance");
+
+    assertThat(params.get("exclude-region")).isEqualTo("/GemfireDataCommandsDUnitTestRegion2");
+    assertThat(params.get("simulate")).isEqualTo("true");
+    assertThat(params.get("time-out")).isEqualTo("\"-1\"");
+  }
+
+  @Test
+  public void optionStartsWithHyphenWithQuotes() throws Exception {
+    String input = "rebalance --exclude-region=/GemfireDataCommandsDUnitTestRegion2 --simulate=true --time-out=\"-1\"";
+    Map<String, String> params = params(input, "rebalance", "rebalance");
+
+    assertThat(params.get("exclude-region")).isEqualTo("/GemfireDataCommandsDUnitTestRegion2");
+    assertThat(params.get("simulate")).isEqualTo("true");
+    assertThat(params.get("time-out")).isEqualTo("\"-1\"");
+  }
+
+  @Test
+  public void optionContainingHyphen() throws Exception {
+    String input = "rebalance --exclude-region=/The-Region --simulate=true";
+    Map<String, String> params = params(input, "rebalance", "rebalance");
+
+    assertThat(params.get("exclude-region")).isEqualTo("/The-Region");
+    assertThat(params.get("simulate")).isEqualTo("true");
+  }
+
+  @Test
+  public void optionContainingUnderscore() throws Exception {
+    String input = "rebalance --exclude-region=/The_region --simulate=true";
+    Map<String, String> params = params(input, "rebalance", "rebalance");
+
+    assertThat(params.get("exclude-region")).isEqualTo("/The_region");
+    assertThat(params.get("simulate")).isEqualTo("true");
+  }
+
+  @Test
   public void oneJOptionWithQuotes() throws Exception {
     String input = "start locator  --J=\"-Dgemfire.http-service-port=8080\" --name=loc1";
     Map<String, String> params = params(input, "start locator", "startLocator");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/170919bc/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
index 68fe0ae..b0272b2 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
@@ -34,6 +34,10 @@ import java.util.Properties;
 import java.util.Random;
 import java.util.Set;
 
+import org.junit.Ignore;
+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.cache.DataPolicy;
@@ -80,9 +84,6 @@ import com.gemstone.gemfire.test.dunit.WaitCriterion;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 import com.gemstone.gemfire.test.junit.categories.FlakyTest;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 /**
  * Dunit class for testing gemfire data commands : get, put, remove, select, rebalance
  */
@@ -1765,7 +1766,6 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
-  @Category(FlakyTest.class) // GEODE-1484
   @Test
   public void testRebalanceCommandForSimulate() {
     setupTestRebalanceForEntireDS();
@@ -1787,7 +1787,6 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
-  @Category(FlakyTest.class)
   @Test
   public void testRebalanceCommandForSimulateWithNoMember() {
     setupTestRebalanceForEntireDS();
@@ -1812,7 +1811,6 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
-  @Category(FlakyTest.class) // GEODE-1483
   @Test
   public void testRebalanceForIncludeRegionFunction() {
     // setup();
@@ -1834,7 +1832,6 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
-  @Category(FlakyTest.class) // GEODE-1551: org.eclipse.jetty.io.EofException
   @Test
   public void testSimulateForEntireDS() {
     setupTestRebalanceForEntireDS();
@@ -1842,7 +1839,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     final VM manager = Host.getHost(0).getVM(0);
     manager.invoke(checkRegionMBeans);
 
-    getLogWriter().info("testSimulateForEntireDS verified Mbean and executin command");
+    getLogWriter().info("testSimulateForEntireDS verified MBean and executing command");
 
     String command = "rebalance --simulate=true";
 
@@ -1859,7 +1856,31 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
-  @Category(FlakyTest.class) // GEODE-1487
+  @Ignore("TODO: enable test after GEODE-1574 is fixed")
+  @Test
+  public void testSimulateForEntireDSWithTimeout() {
+    setupTestRebalanceForEntireDS();
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+
+    getLogWriter().info("testSimulateForEntireDS verified MBean and executing command");
+
+    String command = "rebalance --simulate=true --time-out=-1";
+
+    CommandResult cmdResult = executeCommand(command);
+
+    getLogWriter().info("testSimulateForEntireDS just after executing " + cmdResult);
+
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testSimulateForEntireDS stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
+    }
+  }
+
   @Test
   public void testRebalanceForEntireDS() {
     setupTestRebalanceForEntireDS();
@@ -1950,7 +1971,6 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
-  @Category(FlakyTest.class)
   @Test
   public void testRebalanceForExcludeRegionFunction() {
     setupWith2Regions();
@@ -1968,7 +1988,7 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     if (cmdResult != null) {
       String stringResult = commandResultToString(cmdResult);
       getLogWriter().info("testRebalanceForExcludeRegionFunction stringResult : " + stringResult);
-      assertEquals(Result.Status.OK, cmdResult.getStatus());
+      assertEquals("CommandResult=" + cmdResult, Result.Status.OK, cmdResult.getStatus());
     } else {
       fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/170919bc/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/util/HyphenFormatterTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/util/HyphenFormatterTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/util/HyphenFormatterTest.java
new file mode 100644
index 0000000..fedc597
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/util/HyphenFormatterTest.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.management.internal.cli.util;
+
+import static org.assertj.core.api.Assertions.*;
+import static org.junit.Assert.*;
+
+import java.util.Arrays;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+@Category(UnitTest.class)
+public class HyphenFormatterTest {
+
+  private HyphenFormatter formatter;
+  
+  @Before
+  public void setUp() {
+    this.formatter = new HyphenFormatter();
+  }
+
+  @Test
+  public void containsOptionWithOneOptionReturnsTrue() {
+    String cmd = "start locator --name=loc1";
+    assertTrue(this.formatter.containsOption(cmd));
+  }
+
+  @Test
+  public void containsOptionWithNoOptionReturnsFalse() {
+    String cmd = "start locator";
+    assertFalse(this.formatter.containsOption(cmd));
+  }
+
+  @Test
+  public void containsOptionWithMultipleOptionsReturnsTrue() {
+    String cmd = "start locator --name=loc1 --J=-Dfoo=bar --J=-Dbar=foo";
+    assertTrue(this.formatter.containsOption(cmd));
+  }
+
+  @Test
+  public void valueWithoutQuotesReturnsWithQuotes() {
+    String cmd = "start locator --name=loc1 --J=-Dfoo=bar";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\"";
+    assertThat(formattedCmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void valueWithoutQuotesReturnsWithQuotes_2() {
+    String cmd = "start locator --J=-Dfoo=bar --name=loc1";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+
+    String expected = "start locator --J=\"-Dfoo=bar\" --name=loc1";
+    assertThat(formattedCmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void valueWithHyphenWithoutQuotesFails() {
+    String cmd = "rebalance --exclude-region=/GemfireDataCommandsDUnitTestRegion2 --simulate=true --time-out=-1";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+
+    String expected = "rebalance --exclude-region=/GemfireDataCommandsDUnitTestRegion2 --simulate=true --time-out=\"-1\"";
+    assertThat(formattedCmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void valueWithHyphenWithoutQuotes() {
+    String cmd = "rebalance --exclude-region=/GemfireDataCommandsDUnitTestRegion2 --simulate=true --time-out=-1";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+
+    String expected = "rebalance --exclude-region=/GemfireDataCommandsDUnitTestRegion2 --simulate=true --time-out=\"-1\"";
+    assertThat(formattedCmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void nullShouldThrowNullPointerException() {
+    assertThatThrownBy(() -> this.formatter.formatCommand(null)).isExactlyInstanceOf(NullPointerException.class);
+  }
+
+  @Test
+  public void emptyShouldThrowNullPointerException() {
+    assertThat(this.formatter.formatCommand("")).isEqualTo("");
+  }
+
+  @Test
+  public void multipleJOptions() {
+    String cmd = "start locator --name=loc1 --J=-Dfoo=bar --J=-Dbar=foo";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=\"-Dbar=foo\"";
+    assertThat(formattedCmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void multipleJOptionsWithSomethingAfter() {
+    String cmd = "start locator --name=loc1 --J=-Dfoo=bar --J=-Dbar=foo --group=locators";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=\"-Dbar=foo\" --group=locators";
+    assertThat(formattedCmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void multipleJOptionsWithSomethingBetween() {
+    String cmd = "start locator --name=loc1 --J=-Dfoo=bar --group=locators --J=-Dbar=foo";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\" --group=locators --J=\"-Dbar=foo\"";
+    assertThat(formattedCmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void valueWithQuotes() {
+    String cmd = "start locator --name=loc1 --J=\"-Dfoo=bar\"";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+    assertThat(formattedCmd).isEqualTo(cmd);
+  }
+
+  @Test
+  public void oneValueWithQuotesOneWithout() {
+    String cmd = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=-Dfoo=bar";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=\"-Dfoo=bar\"";
+    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void oneValueWithoutQuotesOneWith() {
+    String cmd = "start locator --name=loc1 --J=-Dfoo=bar --J=\"-Dfoo=bar\"";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=\"-Dfoo=bar\"";
+    assertThat(formattedCmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void twoValuesWithQuotes() {
+    String cmd = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=\"-Dfoo=bar\"";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+    assertThat(formattedCmd).as(cmd).isEqualTo(cmd);
+  }
+
+  @Test
+  public void valueContainingQuotes() {
+    String cmd = "start locator --name=loc1 --J=\"-Dfoo=region\"";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=region\"";
+    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void valueContainingQuotesAndSpace() {
+    String cmd = "start locator --name=loc1 --J=\"-Dfoo=my phrase\"";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=my phrase\"";
+    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void valueContainingQuotesAndMultipleSpaces() {
+    String cmd = "start locator --name=loc1 --J=\"-Dfoo=this is a phrase\"";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=this is a phrase\"";
+    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void valueContainingMultipleJWithSpaces() {
+    String cmd = "start locator --name=loc1 --J=-Dfoo=this is a phrase             --J=\"-Dfoo=a short sentence\"";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=this is a phrase\" --J=\"-Dfoo=a short sentence\"";
+    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
+  }
+
+  @Test
+  public void valueContainingMultipleJWithSpaces2() {
+    String cmd = "start locator --name=loc1 --J=\"-Dfoo=this is a phrase            \" --J=\"-Dfoo=a short sentence\"";
+    String formattedCmd = this.formatter.formatCommand(cmd);
+    String expected = "start locator --name=loc1 --J=\"-Dfoo=this is a phrase            \" --J=\"-Dfoo=a short sentence\"";
+    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/170919bc/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatterTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatterTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatterTest.java
deleted file mode 100644
index 12dcd5e..0000000
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/util/OptionJFormatterTest.java
+++ /dev/null
@@ -1,189 +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.management.internal.cli.util;
-
-import static org.assertj.core.api.Assertions.*;
-import static org.junit.Assert.*;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.test.junit.categories.UnitTest;
-
-@Category(UnitTest.class)
-public class OptionJFormatterTest {
-
-  private OptionJFormatter formatter;
-  
-  @Before
-  public void setUp() {
-    this.formatter = new OptionJFormatter();
-  }
-
-  @Test
-  public void containsJoptShouldReturnTrueIfCmdHasJ() {
-    String cmd = "start locator --name=loc1 --J=-Dfoo=bar";
-    assertTrue(this.formatter.containsJopt(cmd));
-  }
-
-  @Test
-  public void containsJoptShouldReturnFalseIfCmdDoesntHaveJ() {
-    String cmd = "start locator --name=loc1 ";
-    assertFalse(this.formatter.containsJopt(cmd));
-  }
-
-  @Test
-  public void containsJoptShouldReturnTrueIfCmdHasMultipleJ() {
-    String cmd = "start locator --name=loc1 --J=-Dfoo=bar --J=-Dbar=foo";
-    assertTrue(this.formatter.containsJopt(cmd));
-  }
-
-  @Test
-  public void valueWithoutQuotesReturnsWithQuotes() {
-    String cmd = "start locator --name=loc1 --J=-Dfoo=bar";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\"";
-    assertThat(formattedCmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void valueWithoutQuotesReturnsWithQuotes_2() {
-    String cmd = "start locator --J=-Dfoo=bar --name=loc1";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-
-    String expected = "start locator --J=\"-Dfoo=bar\" --name=loc1";
-    assertThat(formattedCmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void nullShouldThrowNullPointerException() {
-    assertThatThrownBy(() -> this.formatter.formatCommand(null)).isExactlyInstanceOf(NullPointerException.class);
-  }
-
-  @Test
-  public void emptyShouldThrowNullPointerException() {
-    assertThat(this.formatter.formatCommand("")).isEqualTo("");
-  }
-
-  @Test
-  public void multipleJOptions() {
-    String cmd = "start locator --name=loc1 --J=-Dfoo=bar --J=-Dbar=foo";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=\"-Dbar=foo\"";
-    assertThat(formattedCmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void multipleJOptionsWithSomethingAfter() {
-    String cmd = "start locator --name=loc1 --J=-Dfoo=bar --J=-Dbar=foo --group=locators";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=\"-Dbar=foo\" --group=locators";
-    assertThat(formattedCmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void multipleJOptionsWithSomethingBetween() {
-    String cmd = "start locator --name=loc1 --J=-Dfoo=bar --group=locators --J=-Dbar=foo";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\" --group=locators --J=\"-Dbar=foo\"";
-    assertThat(formattedCmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void valueWithQuotes() {
-    String cmd = "start locator --name=loc1 --J=\"-Dfoo=bar\"";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-    assertThat(formattedCmd).isEqualTo(cmd);
-  }
-
-  @Test
-  public void valueWithMissingEndQuote() {
-    String cmd = "start locator --J=\"-Dfoo=bar --name=loc1";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-    String expected = "start locator --J=\"-Dfoo=bar\" --name=loc1";
-    assertThat(formattedCmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void valueWithMissingStartQuote() {
-    String cmd = "start locator --name=loc1 --J=-Dfoo=bar\"";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\"";
-    assertThat(formattedCmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void oneValueWithQuotesOneWithout() {
-    String cmd = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=-Dfoo=bar";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=\"-Dfoo=bar\"";
-    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void oneValueWithoutQuotesOneWith() {
-    String cmd = "start locator --name=loc1 --J=-Dfoo=bar --J=\"-Dfoo=bar\"";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=\"-Dfoo=bar\"";
-    assertThat(formattedCmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void twoValuesWithQuotes() {
-    String cmd = "start locator --name=loc1 --J=\"-Dfoo=bar\" --J=\"-Dfoo=bar\"";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-    assertThat(formattedCmd).as(cmd).isEqualTo(cmd);
-  }
-
-  @Test
-  public void valueContainingQuotes() {
-    String cmd = "start locator --name=loc1 --J=\"-Dfoo=region\"";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=region\"";
-    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void valueContainingQuotesAndSpace() {
-    String cmd = "start locator --name=loc1 --J=\"-Dfoo=my phrase\"";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=my phrase\"";
-    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void valueContainingQuotesAndMultipleSpaces() {
-    String cmd = "start locator --name=loc1 --J=\"-Dfoo=this is a phrase\"";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=this is a phrase\"";
-    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
-  }
-
-  @Test
-  public void valueContainingMultipleJWithSpaces() {
-    String cmd = "start locator --name=loc1 --J=-Dfoo=this is a phrase             --J=\"-Dfoo=a short sentence\"";
-    String formattedCmd = this.formatter.formatCommand(cmd);
-    String expected = "start locator --name=loc1 --J=\"-Dfoo=this is a phrase\"             --J=\"-Dfoo=a short sentence\"";
-    assertThat(formattedCmd).as(cmd).isEqualTo(expected);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/170919bc/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
index cad3f31..14a48c7 100644
--- a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CommandOverHttpDUnitTest.java
@@ -42,7 +42,7 @@ import org.junit.runners.Suite;
   DeployCommandsDUnitTest.class,
   DiskStoreCommandsDUnitTest.class,
   FunctionCommandsDUnitTest.class,
-  //GemfireDataCommandsDUnitTest.class, restore when GEODE-1565 is fixed
+  GemfireDataCommandsDUnitTest.class,
   GetCommandOnRegionWithCacheLoaderDuringCacheMissDUnitTest.class,
   IndexCommandsDUnitTest.class,
   ListAndDescribeDiskStoreCommandsDUnitTest.class,



[43/50] [abbrv] incubator-geode git commit: GEODE-1751: putting security checks in all applicable client-server commands.

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java
index 001b912..e11787c 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction65.java
@@ -18,7 +18,6 @@
 package com.gemstone.gemfire.internal.cache.tier.sockets.command;
 
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.HashSet;
 import java.util.Set;
 
@@ -28,7 +27,6 @@ import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.FunctionInvocationTargetException;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.operations.ExecuteFunctionOperationContext;
-import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.cache.DistributedRegion;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
 import com.gemstone.gemfire.internal.cache.execute.AbstractExecution;
@@ -50,10 +48,10 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 /**
- * 
- *  @since GemFire 6.5
+ * @since GemFire 6.5
  */
 public class ExecuteRegionFunction65 extends BaseCommand {
 
@@ -67,8 +65,7 @@ public class ExecuteRegionFunction65 extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException {
     String regionName = null;
     Object function = null;
     Object args = null;
@@ -83,10 +80,9 @@ public class ExecuteRegionFunction65 extends BaseCommand {
     byte functionState = 0;
     try {
       functionState = msg.getPart(0).getSerializedForm()[0];
-      if(functionState != 1) {
+      if (functionState != 1) {
         hasResult = (byte) ((functionState & 2) - 1);
-      }
-      else  {
+      } else {
         hasResult = functionState;
       }
       if (hasResult == 1) {
@@ -100,7 +96,7 @@ public class ExecuteRegionFunction65 extends BaseCommand {
       if (part != null) {
         Object obj = part.getObject();
         if (obj instanceof MemberMappedArgument) {
-          memberMappedArg = (MemberMappedArgument)obj;
+          memberMappedArg = (MemberMappedArgument) obj;
         }
       }
       isReExecute = msg.getPart(5).getSerializedForm()[0];
@@ -112,21 +108,20 @@ public class ExecuteRegionFunction65 extends BaseCommand {
           filter.add(msg.getPart(partNumber + i).getStringOrObject());
         }
       }
-      
+
       partNumber = 7 + filterSize;
       removedNodesSize = msg.getPart(partNumber).getInt();
-      
-      if(removedNodesSize != 0){
+
+      if (removedNodesSize != 0) {
         removedNodesSet = new HashSet<Object>();
         partNumber = partNumber + 1;
-        
+
         for (int i = 0; i < removedNodesSize; i++) {
           removedNodesSet.add(msg.getPart(partNumber + i).getStringOrObject());
         }
       }
-      
-    }
-    catch (ClassNotFoundException exception) {
+
+    } catch (ClassNotFoundException exception) {
       logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), exception);
       if (hasResult == 1) {
         writeChunkedException(msg, exception, false, servConn);
@@ -146,214 +141,192 @@ public class ExecuteRegionFunction65 extends BaseCommand {
       sendError(hasResult, msg, message, servConn);
       return;
     }
-    else {
-      Region region = crHelper.getRegion(regionName);
-      if (region == null) {
-        String message = 
-          LocalizedStrings.ExecuteRegionFunction_THE_REGION_NAMED_0_WAS_NOT_FOUND_DURING_EXECUTE_FUNCTION_REQUEST
-          .toLocalizedString(regionName);
-        logger.warn("{}: {}", servConn.getName(), message);
-        sendError(hasResult, msg, message, servConn);
-        return;
-      }
-      HandShake handShake = (HandShake)servConn.getHandshake();
-      int earlierClientReadTimeout = handShake.getClientReadTimeout();
-      handShake.setClientReadTimeout(0);
-      ServerToClientFunctionResultSender resultSender = null;
-      Function functionObject = null;
-      try { 
-        if (function instanceof String) {
-          functionObject = FunctionService.getFunction((String)function);
-          if (functionObject == null) {
-            String message = LocalizedStrings.
-              ExecuteRegionFunction_THE_FUNCTION_0_HAS_NOT_BEEN_REGISTERED
-                .toLocalizedString(function);
+
+    Region region = crHelper.getRegion(regionName);
+    if (region == null) {
+      String message = LocalizedStrings.ExecuteRegionFunction_THE_REGION_NAMED_0_WAS_NOT_FOUND_DURING_EXECUTE_FUNCTION_REQUEST
+        .toLocalizedString(regionName);
+      logger.warn("{}: {}", servConn.getName(), message);
+      sendError(hasResult, msg, message, servConn);
+      return;
+    }
+
+    HandShake handShake = (HandShake) servConn.getHandshake();
+    int earlierClientReadTimeout = handShake.getClientReadTimeout();
+    handShake.setClientReadTimeout(0);
+    ServerToClientFunctionResultSender resultSender = null;
+    Function functionObject = null;
+    try {
+      if (function instanceof String) {
+        functionObject = FunctionService.getFunction((String) function);
+        if (functionObject == null) {
+          String message = LocalizedStrings.
+            ExecuteRegionFunction_THE_FUNCTION_0_HAS_NOT_BEEN_REGISTERED.toLocalizedString(function);
+          logger.warn("{}: {}", servConn.getName(), message);
+          sendError(hasResult, msg, message, servConn);
+          return;
+        } else {
+          byte functionStateOnServerSide = AbstractExecution.getFunctionState(functionObject.isHA(), functionObject.hasResult(), functionObject
+            .optimizeForWrite());
+          if (logger.isDebugEnabled()) {
+            logger.debug("Function State on server side: {} on client: {}", functionStateOnServerSide, functionState);
+          }
+          if (functionStateOnServerSide != functionState) {
+            String message = LocalizedStrings.FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER.toLocalizedString(function);
             logger.warn("{}: {}", servConn.getName(), message);
             sendError(hasResult, msg, message, servConn);
             return;
           }
-          else {
-            byte functionStateOnServerSide = AbstractExecution.getFunctionState(
-                functionObject.isHA(), functionObject.hasResult(),
-                functionObject.optimizeForWrite());
-            if (logger.isDebugEnabled()) {
-              logger.debug("Function State on server side: {} on client: {}", functionStateOnServerSide, functionState);
-            }
-            if (functionStateOnServerSide != functionState) {
-              String message = LocalizedStrings.FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER
-                  .toLocalizedString(function);
-              logger.warn("{}: {}", servConn.getName(), message);
-              sendError(hasResult, msg, message, servConn);
-              return;
-            }
-          }
-        }
-        else {
-          functionObject = (Function)function;
-        }
-        // check if the caller is authorized to do this operation on server
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        final String functionName = functionObject.getId();
-        final String regionPath = region.getFullPath();
-        ExecuteFunctionOperationContext executeContext = null;
-        if (authzRequest != null) {
-          executeContext = authzRequest.executeFunctionAuthorize(functionName,
-              regionPath, filter, args, functionObject.optimizeForWrite());
-        }
-        
-        //Construct execution 
-        AbstractExecution execution = (AbstractExecution)FunctionService.onRegion(region);
-        ChunkedMessage m = servConn.getFunctionResponseMessage();
-        m.setTransactionId(msg.getTransactionId());        
-        resultSender = new ServerToClientFunctionResultSender65(m,
-            MessageType.EXECUTE_REGION_FUNCTION_RESULT, servConn,functionObject,executeContext);
-        
-        
-        if (execution instanceof PartitionedRegionFunctionExecutor) {
-          execution = new PartitionedRegionFunctionExecutor(
-              (PartitionedRegion)region, filter, args, memberMappedArg,
-              resultSender, removedNodesSet, false);
-        }
-        else {
-          execution = new DistributedRegionFunctionExecutor(
-              (DistributedRegion)region, filter, args, memberMappedArg,
-              resultSender);          
-        }
-        if (isReExecute == 1) {
-          execution = execution.setIsReExecute();
-        }
-        if (logger.isDebugEnabled()) {
-          logger.debug("Executing Function: {} on Server: {} with Execution: {} functionState={} reexecute={} hasResult={}", functionObject.getId(), servConn, execution, functionState, isReExecute, hasResult);
-        }
-        if (hasResult == 1) {
-          if (function instanceof String) {
-            switch (functionState) {
-              case AbstractExecution.NO_HA_HASRESULT_NO_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, false, false).getResult();
-                break;
-              case AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, true, false).getResult();
-                break;
-              case AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE:
-                execution.execute((String)function,true, true, true).getResult();
-                break;
-              case AbstractExecution.NO_HA_HASRESULT_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, false, true).getResult();
-                break;
-            }
-          }
-          else {
-            execution.execute(functionObject).getResult();
-          }
-        }else {
-          if (function instanceof String) {
-            switch (functionState) {
-              case AbstractExecution.NO_HA_NO_HASRESULT_NO_OPTIMIZEFORWRITE:
-                execution.execute((String)function, false, false, false);
-                break;
-              case AbstractExecution.NO_HA_NO_HASRESULT_OPTIMIZEFORWRITE:
-                execution.execute((String)function, false, false, true);
-                break;
-            }
-          }
-          else {
-            execution.execute(functionObject);
-          }
         }
+      } else {
+        functionObject = (Function) function;
       }
-      catch (IOException ioe) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioe);
-        final String message = LocalizedStrings.
-          ExecuteRegionFunction_SERVER_COULD_NOT_SEND_THE_REPLY
-            .toLocalizedString();
-        sendException(hasResult, msg, message, servConn,ioe);
+
+      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+
+      // check if the caller is authorized to do this operation on server
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      final String functionName = functionObject.getId();
+      final String regionPath = region.getFullPath();
+      ExecuteFunctionOperationContext executeContext = null;
+      if (authzRequest != null) {
+        executeContext = authzRequest.executeFunctionAuthorize(functionName, regionPath, filter, args, functionObject.optimizeForWrite());
       }
-      catch (FunctionException fe) {
-        String message = fe.getMessage();
-        
-        if (fe.getCause() instanceof FunctionInvocationTargetException) {
-          if (fe.getCause() instanceof InternalFunctionInvocationTargetException) {
-            // Fix for #44709: User should not be aware of
-            // InternalFunctionInvocationTargetException. No instance of
-            // InternalFunctionInvocationTargetException is giving useful
-            // information to user to take any corrective action hence logging
-            // this at fine level logging
-            // 1> When bucket is moved
-            // 2> Incase of HA FucntionInvocationTargetException thrown. Since
-            // it is HA, fucntion will be reexecuted on right node
-            // 3> Multiple target nodes found for single hop operation
-            // 4> in case of HA member departed
-            if (logger.isDebugEnabled()) {
-              logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] { function }), fe);
-            }
-          }
-          else if (functionObject.isHA()) {
-            logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function + " :" + message));
+
+      //Construct execution
+      AbstractExecution execution = (AbstractExecution) FunctionService.onRegion(region);
+      ChunkedMessage m = servConn.getFunctionResponseMessage();
+      m.setTransactionId(msg.getTransactionId());
+      resultSender = new ServerToClientFunctionResultSender65(m, MessageType.EXECUTE_REGION_FUNCTION_RESULT, servConn, functionObject, executeContext);
+
+
+      if (execution instanceof PartitionedRegionFunctionExecutor) {
+        execution = new PartitionedRegionFunctionExecutor((PartitionedRegion) region, filter, args, memberMappedArg, resultSender, removedNodesSet, false);
+      } else {
+        execution = new DistributedRegionFunctionExecutor((DistributedRegion) region, filter, args, memberMappedArg, resultSender);
+      }
+      if (isReExecute == 1) {
+        execution = execution.setIsReExecute();
+      }
+      if (logger.isDebugEnabled()) {
+        logger.debug("Executing Function: {} on Server: {} with Execution: {} functionState={} reexecute={} hasResult={}", functionObject
+          .getId(), servConn, execution, functionState, isReExecute, hasResult);
+      }
+      if (hasResult == 1) {
+        if (function instanceof String) {
+          switch (functionState) {
+            case AbstractExecution.NO_HA_HASRESULT_NO_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, false, false).getResult();
+              break;
+            case AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, true, false).getResult();
+              break;
+            case AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, true, true).getResult();
+              break;
+            case AbstractExecution.NO_HA_HASRESULT_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, false, true).getResult();
+              break;
           }
-          else {
-            logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), fe);
+        } else {
+          execution.execute(functionObject).getResult();
+        }
+      } else {
+        if (function instanceof String) {
+          switch (functionState) {
+            case AbstractExecution.NO_HA_NO_HASRESULT_NO_OPTIMIZEFORWRITE:
+              execution.execute((String) function, false, false, false);
+              break;
+            case AbstractExecution.NO_HA_NO_HASRESULT_OPTIMIZEFORWRITE:
+              execution.execute((String) function, false, false, true);
+              break;
           }
-
-          resultSender.setException(fe);
+        } else {
+          execution.execute(functionObject);
         }
-        else {
+      }
+    } catch (IOException ioe) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioe);
+      final String message = LocalizedStrings.
+        ExecuteRegionFunction_SERVER_COULD_NOT_SEND_THE_REPLY.toLocalizedString();
+      sendException(hasResult, msg, message, servConn, ioe);
+    } catch (FunctionException fe) {
+      String message = fe.getMessage();
+
+      if (fe.getCause() instanceof FunctionInvocationTargetException) {
+        if (fe.getCause() instanceof InternalFunctionInvocationTargetException) {
+          // Fix for #44709: User should not be aware of
+          // InternalFunctionInvocationTargetException. No instance of
+          // InternalFunctionInvocationTargetException is giving useful
+          // information to user to take any corrective action hence logging
+          // this at fine level logging
+          // 1> When bucket is moved
+          // 2> Incase of HA FucntionInvocationTargetException thrown. Since
+          // it is HA, fucntion will be reexecuted on right node
+          // 3> Multiple target nodes found for single hop operation
+          // 4> in case of HA member departed
+          if (logger.isDebugEnabled()) {
+            logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] {
+              function
+            }), fe);
+          }
+        } else if (functionObject.isHA()) {
+          logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function + " :" + message));
+        } else {
           logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), fe);
-          sendException(hasResult, msg, message, servConn, fe);
         }
 
-      }
-      catch (Exception e) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
-        String message = e.getMessage();
-        sendException(hasResult, msg, message, servConn,e);
+        resultSender.setException(fe);
+      } else {
+        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), fe);
+        sendException(hasResult, msg, message, servConn, fe);
       }
 
-      finally{
-        handShake.setClientReadTimeout(earlierClientReadTimeout);
-      }
+    } catch (Exception e) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
+      String message = e.getMessage();
+      sendException(hasResult, msg, message, servConn, e);
+    } finally {
+      handShake.setClientReadTimeout(earlierClientReadTimeout);
     }
   }
 
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection servConn, Throwable e) throws IOException {
+  private void sendException(byte hasResult, Message msg, String message, ServerConnection servConn, Throwable e)
+    throws IOException {
     synchronized (msg) {
       if (hasResult == 1) {
-        writeFunctionResponseException(msg, MessageType.EXCEPTION, message,
-            servConn, e);
+        writeFunctionResponseException(msg, MessageType.EXCEPTION, message, servConn, e);
         servConn.setAsTrue(RESPONDED);
       }
     }
   }
-  
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection servConn) throws IOException {
+
+  private void sendError(byte hasResult, Message msg, String message, ServerConnection servConn) throws IOException {
     synchronized (msg) {
       if (hasResult == 1) {
-        writeFunctionResponseError(msg,
-            MessageType.EXECUTE_REGION_FUNCTION_ERROR, message, servConn);
+        writeFunctionResponseError(msg, MessageType.EXECUTE_REGION_FUNCTION_ERROR, message, servConn);
         servConn.setAsTrue(RESPONDED);
       }
     }
   }
-  
+
   protected static void writeFunctionResponseException(Message origMsg,
-      int messageType, String message, ServerConnection servConn, Throwable e)
-      throws IOException {
+                                                       int messageType,
+                                                       String message,
+                                                       ServerConnection servConn,
+                                                       Throwable e) throws IOException {
     ChunkedMessage functionResponseMsg = servConn.getFunctionResponseMessage();
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     int numParts = 0;
     if (functionResponseMsg.headerHasBeenSent()) {
-      if (e instanceof FunctionException
-          && e.getCause() instanceof InternalFunctionInvocationTargetException) {
+      if (e instanceof FunctionException && e.getCause() instanceof InternalFunctionInvocationTargetException) {
         functionResponseMsg.setNumberOfParts(3);
         functionResponseMsg.addObjPart(e);
         functionResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
-        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException)e
-            .getCause();
+        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException) e.getCause();
         functionResponseMsg.addObjPart(fe.getFailedNodeSet());
         numParts = 3;
-      }
-      else {
+      } else {
         functionResponseMsg.setNumberOfParts(2);
         functionResponseMsg.addObjPart(e);
         functionResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
@@ -366,22 +339,18 @@ public class ExecuteRegionFunction65 extends BaseCommand {
       functionResponseMsg.setLastChunkAndNumParts(true, numParts);
       //functionResponseMsg.setLastChunk(true);
       functionResponseMsg.sendChunk(servConn);
-    }
-    else {
+    } else {
       chunkedResponseMsg.setMessageType(messageType);
       chunkedResponseMsg.setTransactionId(origMsg.getTransactionId());
       chunkedResponseMsg.sendHeader();
-      if (e instanceof FunctionException
-          && e.getCause() instanceof InternalFunctionInvocationTargetException) {
+      if (e instanceof FunctionException && e.getCause() instanceof InternalFunctionInvocationTargetException) {
         chunkedResponseMsg.setNumberOfParts(3);
         chunkedResponseMsg.addObjPart(e);
         chunkedResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
-        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException)e
-            .getCause();
+        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException) e.getCause();
         chunkedResponseMsg.addObjPart(fe.getFailedNodeSet());
         numParts = 3;
-      }
-      else {
+      } else {
         chunkedResponseMsg.setNumberOfParts(2);
         chunkedResponseMsg.addObjPart(e);
         chunkedResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
@@ -391,7 +360,7 @@ public class ExecuteRegionFunction65 extends BaseCommand {
         logger.debug("{}: Sending exception chunk: ", servConn.getName(), e);
       }
       chunkedResponseMsg.setServerConnection(servConn);
-      chunkedResponseMsg.setLastChunkAndNumParts(true,numParts);
+      chunkedResponseMsg.setLastChunkAndNumParts(true, numParts);
       chunkedResponseMsg.sendChunk(servConn);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction66.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction66.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction66.java
index 6e42316..36285bf 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction66.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunction66.java
@@ -52,13 +52,13 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 /**
- * 
- *  @since GemFire 6.6
+ * @since GemFire 6.6
  */
 public class ExecuteRegionFunction66 extends BaseCommand {
-  
+
   private final static ExecuteRegionFunction66 singleton = new ExecuteRegionFunction66();
 
   public static Command getCommand() {
@@ -69,14 +69,13 @@ public class ExecuteRegionFunction66 extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException {
     String regionName = null;
     Object function = null;
     Object args = null;
     MemberMappedArgument memberMappedArg = null;
-    final boolean isBucketsAsFilter ;
-    final byte isReExecute ;
+    final boolean isBucketsAsFilter;
+    final byte isReExecute;
     Set<Object> filter = null;
     byte hasResult = 0;
     int removedNodesSize = 0;
@@ -92,9 +91,8 @@ public class ExecuteRegionFunction66 extends BaseCommand {
         functionTimeout = Part.decodeInt(bytes, 1);
       }
       if (functionState != 1) {
-        hasResult = (byte)((functionState & 2) - 1);
-      }
-      else {
+        hasResult = (byte) ((functionState & 2) - 1);
+      } else {
         hasResult = functionState;
       }
       if (hasResult == 1) {
@@ -108,14 +106,14 @@ public class ExecuteRegionFunction66 extends BaseCommand {
       if (part != null) {
         Object obj = part.getObject();
         if (obj instanceof MemberMappedArgument) {
-          memberMappedArg = (MemberMappedArgument)obj;
+          memberMappedArg = (MemberMappedArgument) obj;
         }
       }
-      byte[] flags =msg.getPart(5).getSerializedForm();
-      if(servConn.getClientVersion().ordinal() > Version.GFE_81.ordinal()) {
-        isBucketsAsFilter = (flags[0] & ExecuteFunctionHelper.BUCKETS_AS_FILTER_MASK) != 0 ; 
-        isReExecute = (flags[0] & ExecuteFunctionHelper.IS_REXECUTE_MASK) != 0 ? (byte)1 : 0;
-      }else {
+      byte[] flags = msg.getPart(5).getSerializedForm();
+      if (servConn.getClientVersion().ordinal() > Version.GFE_81.ordinal()) {
+        isBucketsAsFilter = (flags[0] & ExecuteFunctionHelper.BUCKETS_AS_FILTER_MASK) != 0;
+        isReExecute = (flags[0] & ExecuteFunctionHelper.IS_REXECUTE_MASK) != 0 ? (byte) 1 : 0;
+      } else {
         isReExecute = flags[0];
         isBucketsAsFilter = false;
       }
@@ -140,13 +138,11 @@ public class ExecuteRegionFunction66 extends BaseCommand {
         }
       }
 
-    }
-    catch (ClassNotFoundException exception) {
+    } catch (ClassNotFoundException exception) {
       logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), exception);
       if (hasResult == 1) {
         writeChunkedException(msg, exception, false, servConn);
-      }
-      else {
+      } else {
         writeException(msg, exception, false, servConn);
       }
       servConn.setAsTrue(RESPONDED);
@@ -155,240 +151,206 @@ public class ExecuteRegionFunction66 extends BaseCommand {
     if (function == null || regionName == null) {
       String message = null;
       if (function == null) {
-        message = LocalizedStrings.ExecuteRegionFunction_THE_INPUT_0_FOR_THE_EXECUTE_FUNCTION_REQUEST_IS_NULL
-            .toLocalizedString("function");
+        message = LocalizedStrings.ExecuteRegionFunction_THE_INPUT_0_FOR_THE_EXECUTE_FUNCTION_REQUEST_IS_NULL.toLocalizedString("function");
       }
       if (regionName == null) {
-        message = LocalizedStrings.ExecuteRegionFunction_THE_INPUT_0_FOR_THE_EXECUTE_FUNCTION_REQUEST_IS_NULL
-            .toLocalizedString("region");
+        message = LocalizedStrings.ExecuteRegionFunction_THE_INPUT_0_FOR_THE_EXECUTE_FUNCTION_REQUEST_IS_NULL.toLocalizedString("region");
       }
       logger.warn("{}: {}", servConn.getName(), message);
       sendError(hasResult, msg, message, servConn);
       return;
     }
-    else {
-      Region region = crHelper.getRegion(regionName);
-      if (region == null) {
-        String message = LocalizedStrings.ExecuteRegionFunction_THE_REGION_NAMED_0_WAS_NOT_FOUND_DURING_EXECUTE_FUNCTION_REQUEST
-            .toLocalizedString(regionName);
-        logger.warn("{}: {}", servConn.getName(), message);
-        sendError(hasResult, msg, message, servConn);
-        return;
-      }
-      HandShake handShake = (HandShake)servConn.getHandshake();
-      int earlierClientReadTimeout = handShake.getClientReadTimeout();
-      handShake.setClientReadTimeout(functionTimeout);
-      ServerToClientFunctionResultSender resultSender = null;
-      Function functionObject = null;
-      try {
-        if (function instanceof String) {
-          functionObject = FunctionService.getFunction((String)function);
-          if (functionObject == null) {
-            String message = LocalizedStrings.ExecuteRegionFunction_THE_FUNCTION_0_HAS_NOT_BEEN_REGISTERED
-                .toLocalizedString(function);
+
+    Region region = crHelper.getRegion(regionName);
+    if (region == null) {
+      String message = LocalizedStrings.ExecuteRegionFunction_THE_REGION_NAMED_0_WAS_NOT_FOUND_DURING_EXECUTE_FUNCTION_REQUEST
+        .toLocalizedString(regionName);
+      logger.warn("{}: {}", servConn.getName(), message);
+      sendError(hasResult, msg, message, servConn);
+      return;
+    }
+    HandShake handShake = (HandShake) servConn.getHandshake();
+    int earlierClientReadTimeout = handShake.getClientReadTimeout();
+    handShake.setClientReadTimeout(functionTimeout);
+    ServerToClientFunctionResultSender resultSender = null;
+    Function functionObject = null;
+    try {
+      if (function instanceof String) {
+        functionObject = FunctionService.getFunction((String) function);
+        if (functionObject == null) {
+          String message = LocalizedStrings.ExecuteRegionFunction_THE_FUNCTION_0_HAS_NOT_BEEN_REGISTERED.toLocalizedString(function);
+          logger.warn("{}: {}", servConn.getName(), message);
+          sendError(hasResult, msg, message, servConn);
+          return;
+        } else {
+          byte functionStateOnServerSide = AbstractExecution.getFunctionState(functionObject.isHA(), functionObject.hasResult(), functionObject
+            .optimizeForWrite());
+          if (logger.isDebugEnabled()) {
+            logger.debug("Function State on server side: {} on client: {}", functionStateOnServerSide, functionState);
+          }
+          if (functionStateOnServerSide != functionState) {
+            String message = LocalizedStrings.FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER.toLocalizedString(function);
             logger.warn("{}: {}", servConn.getName(), message);
             sendError(hasResult, msg, message, servConn);
             return;
           }
-          else {
-            byte functionStateOnServerSide = AbstractExecution
-                .getFunctionState(functionObject.isHA(), functionObject
-                    .hasResult(), functionObject.optimizeForWrite());
-            if (logger.isDebugEnabled()) {
-              logger.debug("Function State on server side: {} on client: {}", functionStateOnServerSide, functionState);
-            }
-            if (functionStateOnServerSide != functionState) {
-              String message = LocalizedStrings.FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER
-                  .toLocalizedString(function);
-              logger.warn("{}: {}", servConn.getName(), message);
-              sendError(hasResult, msg, message, servConn);
-              return;
-            }
-          }
-        }
-        else {
-          functionObject = (Function)function;
-        }
-        // check if the caller is authorized to do this operation on server
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        final String functionName = functionObject.getId();
-        final String regionPath = region.getFullPath();
-        ExecuteFunctionOperationContext executeContext = null;
-        if (authzRequest != null) {
-          executeContext = authzRequest.executeFunctionAuthorize(functionName,
-              regionPath, filter, args, functionObject.optimizeForWrite());
         }
+      } else {
+        functionObject = (Function) function;
+      }
 
-        // Construct execution
-        AbstractExecution execution = (AbstractExecution)FunctionService
-            .onRegion(region);
-        ChunkedMessage m = servConn.getFunctionResponseMessage();
-        m.setTransactionId(msg.getTransactionId());
-        resultSender = new ServerToClientFunctionResultSender65(m,
-            MessageType.EXECUTE_REGION_FUNCTION_RESULT, servConn,
-            functionObject, executeContext);
+      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
 
-        if (execution instanceof PartitionedRegionFunctionExecutor) {
-          if((hasResult == 1) && filter!= null &&filter.size() == 1) {
-            ServerConnection.executeFunctionOnLocalNodeOnly((byte)1);
-          }
-          execution = new PartitionedRegionFunctionExecutor(
-              (PartitionedRegion)region, filter, args, memberMappedArg,
-              resultSender, removedNodesSet, isBucketsAsFilter);
-        }
-        else {
-          execution = new DistributedRegionFunctionExecutor(
-              (DistributedRegion)region, filter, args, memberMappedArg,
-              resultSender);
-        }
-        if (isReExecute == 1) {
-          execution = execution.setIsReExecute();
-        }
-        if (logger.isDebugEnabled()) {
-          logger.debug("Executing Function: {} on Server: {} with Execution: {} functionState={} reExecute={} hasResult={}", functionObject.getId(), servConn, execution, functionState, isReExecute, hasResult);
+      // check if the caller is authorized to do this operation on server
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      final String functionName = functionObject.getId();
+      final String regionPath = region.getFullPath();
+      ExecuteFunctionOperationContext executeContext = null;
+      if (authzRequest != null) {
+        executeContext = authzRequest.executeFunctionAuthorize(functionName, regionPath, filter, args, functionObject.optimizeForWrite());
+      }
+
+      // Construct execution
+      AbstractExecution execution = (AbstractExecution) FunctionService.onRegion(region);
+      ChunkedMessage m = servConn.getFunctionResponseMessage();
+      m.setTransactionId(msg.getTransactionId());
+      resultSender = new ServerToClientFunctionResultSender65(m, MessageType.EXECUTE_REGION_FUNCTION_RESULT, servConn, functionObject, executeContext);
+
+      if (execution instanceof PartitionedRegionFunctionExecutor) {
+        if ((hasResult == 1) && filter != null && filter.size() == 1) {
+          ServerConnection.executeFunctionOnLocalNodeOnly((byte) 1);
         }
-        if (hasResult == 1) {
-          if (function instanceof String) {
-            switch (functionState) {
-              case AbstractExecution.NO_HA_HASRESULT_NO_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, false, false)
-                    .getResult();
-                break;
-              case AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, true, false)
-                    .getResult();
-                break;
-              case AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, true, true)
-                    .getResult();
-                break;
-              case AbstractExecution.NO_HA_HASRESULT_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, false, true)
-                    .getResult();
-                break;
-            }
-          }
-          else {
-            execution.execute(functionObject).getResult();
+        execution = new PartitionedRegionFunctionExecutor((PartitionedRegion) region, filter, args, memberMappedArg, resultSender, removedNodesSet, isBucketsAsFilter);
+      } else {
+        execution = new DistributedRegionFunctionExecutor((DistributedRegion) region, filter, args, memberMappedArg, resultSender);
+      }
+      if (isReExecute == 1) {
+        execution = execution.setIsReExecute();
+      }
+      if (logger.isDebugEnabled()) {
+        logger.debug("Executing Function: {} on Server: {} with Execution: {} functionState={} reExecute={} hasResult={}", functionObject
+          .getId(), servConn, execution, functionState, isReExecute, hasResult);
+      }
+      if (hasResult == 1) {
+        if (function instanceof String) {
+          switch (functionState) {
+            case AbstractExecution.NO_HA_HASRESULT_NO_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, false, false).getResult();
+              break;
+            case AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, true, false).getResult();
+              break;
+            case AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, true, true).getResult();
+              break;
+            case AbstractExecution.NO_HA_HASRESULT_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, false, true).getResult();
+              break;
           }
+        } else {
+          execution.execute(functionObject).getResult();
         }
-        else {
-          if (function instanceof String) {
-            switch (functionState) {
-              case AbstractExecution.NO_HA_NO_HASRESULT_NO_OPTIMIZEFORWRITE:
-                execution.execute((String)function, false, false, false);
-                break;
-              case AbstractExecution.NO_HA_NO_HASRESULT_OPTIMIZEFORWRITE:
-                execution.execute((String)function, false, false, true);
-                break;
-            }
-          }
-          else {
-            execution.execute(functionObject);
+      } else {
+        if (function instanceof String) {
+          switch (functionState) {
+            case AbstractExecution.NO_HA_NO_HASRESULT_NO_OPTIMIZEFORWRITE:
+              execution.execute((String) function, false, false, false);
+              break;
+            case AbstractExecution.NO_HA_NO_HASRESULT_OPTIMIZEFORWRITE:
+              execution.execute((String) function, false, false, true);
+              break;
           }
-          writeReply(msg, servConn);
+        } else {
+          execution.execute(functionObject);
         }
+        writeReply(msg, servConn);
       }
-      catch (IOException ioe) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioe);
-        final String message = LocalizedStrings.ExecuteRegionFunction_SERVER_COULD_NOT_SEND_THE_REPLY
-            .toLocalizedString();
-        sendException(hasResult, msg, message, servConn, ioe);
-      }
-      catch (FunctionException fe) {
-        String message = fe.getMessage();
-        Object cause = fe.getCause();
-        if (cause instanceof FunctionInvocationTargetException || cause instanceof QueryInvocationTargetException) {
-          if (cause instanceof InternalFunctionInvocationTargetException) {
-            // Fix for #44709: User should not be aware of
-            // InternalFunctionInvocationTargetException. No instance of
-            // InternalFunctionInvocationTargetException is giving useful
-            // information to user to take any corrective action hence logging
-            // this at fine level logging
-            // 1> When bucket is moved
-            // 2> Incase of HA FucntionInvocationTargetException thrown. Since
-            // it is HA, fucntion will be reexecuted on right node
-            // 3> Multiple target nodes found for single hop operation
-            // 4> in case of HA member departed
-            if (logger.isDebugEnabled()) {
-              logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] { function }), fe);
-            }
+    } catch (IOException ioe) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioe);
+      final String message = LocalizedStrings.ExecuteRegionFunction_SERVER_COULD_NOT_SEND_THE_REPLY.toLocalizedString();
+      sendException(hasResult, msg, message, servConn, ioe);
+    } catch (FunctionException fe) {
+      String message = fe.getMessage();
+      Object cause = fe.getCause();
+      if (cause instanceof FunctionInvocationTargetException || cause instanceof QueryInvocationTargetException) {
+        if (cause instanceof InternalFunctionInvocationTargetException) {
+          // Fix for #44709: User should not be aware of
+          // InternalFunctionInvocationTargetException. No instance of
+          // InternalFunctionInvocationTargetException is giving useful
+          // information to user to take any corrective action hence logging
+          // this at fine level logging
+          // 1> When bucket is moved
+          // 2> Incase of HA FucntionInvocationTargetException thrown. Since
+          // it is HA, fucntion will be reexecuted on right node
+          // 3> Multiple target nodes found for single hop operation
+          // 4> in case of HA member departed
+          if (logger.isDebugEnabled()) {
+            logger.debug(LocalizedMessage.create(LocalizedStrings.ExecuteFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, new Object[] {
+              function
+            }), fe);
           }
-          else if (functionObject.isHA()) {
-            logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function + " :" + message));
-          }
-          else {
-            logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), fe);
-          }
-
-          resultSender.setException(fe);
-        }
-        else {
+        } else if (functionObject.isHA()) {
+          logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function + " :" + message));
+        } else {
           logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), fe);
-          sendException(hasResult, msg, message, servConn, fe);
         }
 
-      }
-      catch (Exception e) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
-        String message = e.getMessage();
-        sendException(hasResult, msg, message, servConn, e);
+        resultSender.setException(fe);
+      } else {
+        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), fe);
+        sendException(hasResult, msg, message, servConn, fe);
       }
 
-      finally {
-        handShake.setClientReadTimeout(earlierClientReadTimeout);
-        ServerConnection.executeFunctionOnLocalNodeOnly((byte)0);
-      }
+    } catch (Exception e) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
+      String message = e.getMessage();
+      sendException(hasResult, msg, message, servConn, e);
+    } finally {
+      handShake.setClientReadTimeout(earlierClientReadTimeout);
+      ServerConnection.executeFunctionOnLocalNodeOnly((byte) 0);
     }
   }
 
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection servConn, Throwable e) throws IOException {
+  private void sendException(byte hasResult, Message msg, String message, ServerConnection servConn, Throwable e)
+    throws IOException {
     synchronized (msg) {
       if (hasResult == 1) {
-        writeFunctionResponseException(msg, MessageType.EXCEPTION, message,
-            servConn, e);
-      }
-      else {
+        writeFunctionResponseException(msg, MessageType.EXCEPTION, message, servConn, e);
+      } else {
         writeException(msg, e, false, servConn);
       }
       servConn.setAsTrue(RESPONDED);
     }
   }
 
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection servConn) throws IOException {
+  private void sendError(byte hasResult, Message msg, String message, ServerConnection servConn) throws IOException {
     synchronized (msg) {
       if (hasResult == 1) {
-        writeFunctionResponseError(msg,
-            MessageType.EXECUTE_REGION_FUNCTION_ERROR, message, servConn);
-      }
-      else {
-        writeErrorResponse(msg, MessageType.EXECUTE_REGION_FUNCTION_ERROR,
-            message, servConn);
+        writeFunctionResponseError(msg, MessageType.EXECUTE_REGION_FUNCTION_ERROR, message, servConn);
+      } else {
+        writeErrorResponse(msg, MessageType.EXECUTE_REGION_FUNCTION_ERROR, message, servConn);
       }
       servConn.setAsTrue(RESPONDED);
     }
   }
 
   protected static void writeFunctionResponseException(Message origMsg,
-      int messageType, String message, ServerConnection servConn, Throwable e)
-      throws IOException {
+                                                       int messageType,
+                                                       String message,
+                                                       ServerConnection servConn,
+                                                       Throwable e) throws IOException {
     ChunkedMessage functionResponseMsg = servConn.getFunctionResponseMessage();
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     int numParts = 0;
     if (functionResponseMsg.headerHasBeenSent()) {
-      if (e instanceof FunctionException
-          && e.getCause() instanceof InternalFunctionInvocationTargetException) {
+      if (e instanceof FunctionException && e.getCause() instanceof InternalFunctionInvocationTargetException) {
         functionResponseMsg.setNumberOfParts(3);
         functionResponseMsg.addObjPart(e);
         functionResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
-        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException)e
-            .getCause();
+        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException) e.getCause();
         functionResponseMsg.addObjPart(fe.getFailedNodeSet());
         numParts = 3;
-      }
-      else {
+      } else {
         functionResponseMsg.setNumberOfParts(2);
         functionResponseMsg.addObjPart(e);
         functionResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
@@ -401,22 +363,18 @@ public class ExecuteRegionFunction66 extends BaseCommand {
       functionResponseMsg.setLastChunkAndNumParts(true, numParts);
       // functionResponseMsg.setLastChunk(true);
       functionResponseMsg.sendChunk(servConn);
-    }
-    else {
+    } else {
       chunkedResponseMsg.setMessageType(messageType);
       chunkedResponseMsg.setTransactionId(origMsg.getTransactionId());
       chunkedResponseMsg.sendHeader();
-      if (e instanceof FunctionException
-          && e.getCause() instanceof InternalFunctionInvocationTargetException) {
+      if (e instanceof FunctionException && e.getCause() instanceof InternalFunctionInvocationTargetException) {
         chunkedResponseMsg.setNumberOfParts(3);
         chunkedResponseMsg.addObjPart(e);
         chunkedResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
-        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException)e
-            .getCause();
+        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException) e.getCause();
         chunkedResponseMsg.addObjPart(fe.getFailedNodeSet());
         numParts = 3;
-      }
-      else {
+      } else {
         chunkedResponseMsg.setNumberOfParts(2);
         chunkedResponseMsg.addObjPart(e);
         chunkedResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunctionSingleHop.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunctionSingleHop.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunctionSingleHop.java
index f5d8e51..1bfe7de 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunctionSingleHop.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ExecuteRegionFunctionSingleHop.java
@@ -28,7 +28,6 @@ import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.FunctionInvocationTargetException;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.operations.ExecuteFunctionOperationContext;
-import com.gemstone.gemfire.i18n.LogWriterI18n;
 import com.gemstone.gemfire.internal.Version;
 import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
 import com.gemstone.gemfire.internal.cache.PartitionedRegion;
@@ -50,10 +49,10 @@ import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.i18n.LocalizedStrings;
 import com.gemstone.gemfire.internal.logging.log4j.LocalizedMessage;
 import com.gemstone.gemfire.internal.security.AuthorizeRequest;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 /**
- * 
- *  @since GemFire 6.5
+ * @since GemFire 6.5
  */
 public class ExecuteRegionFunctionSingleHop extends BaseCommand {
 
@@ -67,9 +66,8 @@ public class ExecuteRegionFunctionSingleHop extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException {
-    
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException {
+
     String regionName = null;
     Object function = null;
     Object args = null;
@@ -81,7 +79,7 @@ public class ExecuteRegionFunctionSingleHop extends BaseCommand {
     byte functionState = 0;
     int removedNodesSize = 0;
     Set<Object> removedNodesSet = null;
-    int filterSize = 0, bucketIdsSize = 0,  partNumber = 0;
+    int filterSize = 0, bucketIdsSize = 0, partNumber = 0;
     CachedRegionHelper crHelper = servConn.getCachedRegionHelper();
     int functionTimeout = GemFireCacheImpl.DEFAULT_CLIENT_FUNCTION_TIMEOUT;
     try {
@@ -90,10 +88,9 @@ public class ExecuteRegionFunctionSingleHop extends BaseCommand {
       if (bytes.length >= 5 && servConn.getClientVersion().ordinal() >= Version.GFE_8009.ordinal()) {
         functionTimeout = Part.decodeInt(bytes, 1);
       }
-      if(functionState != 1) {
+      if (functionState != 1) {
         hasResult = (byte) ((functionState & 2) - 1);
-      }
-      else {
+      } else {
         hasResult = functionState;
       }
       if (hasResult == 1) {
@@ -107,13 +104,13 @@ public class ExecuteRegionFunctionSingleHop extends BaseCommand {
       if (part != null) {
         Object obj = part.getObject();
         if (obj instanceof MemberMappedArgument) {
-          memberMappedArg = (MemberMappedArgument)obj;
+          memberMappedArg = (MemberMappedArgument) obj;
         }
       }
       isExecuteOnAllBuckets = msg.getPart(5).getSerializedForm()[0];
-      if(isExecuteOnAllBuckets  == 1) {
+      if (isExecuteOnAllBuckets == 1) {
         filter = new HashSet();
-        bucketIdsSize  = msg.getPart(6).getInt();
+        bucketIdsSize = msg.getPart(6).getInt();
         if (bucketIdsSize != 0) {
           buckets = new HashSet<Integer>();
           partNumber = 7;
@@ -122,8 +119,7 @@ public class ExecuteRegionFunctionSingleHop extends BaseCommand {
           }
         }
         partNumber = 7 + bucketIdsSize;
-      }
-      else {
+      } else {
         filterSize = msg.getPart(6).getInt();
         if (filterSize != 0) {
           filter = new HashSet<Object>();
@@ -131,24 +127,23 @@ public class ExecuteRegionFunctionSingleHop extends BaseCommand {
           for (int i = 0; i < filterSize; i++) {
             filter.add(msg.getPart(partNumber + i).getStringOrObject());
           }
-        } 
+        }
         partNumber = 7 + filterSize;
       }
-      
-      
+
+
       removedNodesSize = msg.getPart(partNumber).getInt();
-      
-      if(removedNodesSize != 0){
+
+      if (removedNodesSize != 0) {
         removedNodesSet = new HashSet<Object>();
         partNumber = partNumber + 1;
-        
+
         for (int i = 0; i < removedNodesSize; i++) {
           removedNodesSet.add(msg.getPart(partNumber + i).getStringOrObject());
         }
       }
-      
-    }
-    catch (ClassNotFoundException exception) {
+
+    } catch (ClassNotFoundException exception) {
       logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), exception);
       if (hasResult == 1) {
         writeChunkedException(msg, exception, false, servConn);
@@ -168,213 +163,182 @@ public class ExecuteRegionFunctionSingleHop extends BaseCommand {
       sendError(hasResult, msg, message, servConn);
       return;
     }
-    else {
-      Region region = crHelper.getRegion(regionName);
-      if (region == null) {
-        String message = 
-          LocalizedStrings.ExecuteRegionFunction_THE_REGION_NAMED_0_WAS_NOT_FOUND_DURING_EXECUTE_FUNCTION_REQUEST
-          .toLocalizedString(regionName);
-        logger.warn("{}: {}", servConn.getName(), message);
-        sendError(hasResult, msg, message, servConn);
-        return;
-      }
-      HandShake handShake = (HandShake)servConn.getHandshake();
-      int earlierClientReadTimeout = handShake.getClientReadTimeout();
-      handShake.setClientReadTimeout(functionTimeout);
-      ServerToClientFunctionResultSender resultSender = null;
-      Function functionObject = null;
-      try { 
-        if (function instanceof String) {
-          functionObject = FunctionService.getFunction((String)function);
-          if (functionObject == null) {
-            String message = LocalizedStrings.
-              ExecuteRegionFunction_THE_FUNCTION_0_HAS_NOT_BEEN_REGISTERED
-                .toLocalizedString(function);
+
+    Region region = crHelper.getRegion(regionName);
+    if (region == null) {
+      String message = LocalizedStrings.ExecuteRegionFunction_THE_REGION_NAMED_0_WAS_NOT_FOUND_DURING_EXECUTE_FUNCTION_REQUEST
+        .toLocalizedString(regionName);
+      logger.warn("{}: {}", servConn.getName(), message);
+      sendError(hasResult, msg, message, servConn);
+      return;
+    }
+    HandShake handShake = (HandShake) servConn.getHandshake();
+    int earlierClientReadTimeout = handShake.getClientReadTimeout();
+    handShake.setClientReadTimeout(functionTimeout);
+    ServerToClientFunctionResultSender resultSender = null;
+    Function functionObject = null;
+    try {
+      if (function instanceof String) {
+        functionObject = FunctionService.getFunction((String) function);
+        if (functionObject == null) {
+          String message = LocalizedStrings.
+            ExecuteRegionFunction_THE_FUNCTION_0_HAS_NOT_BEEN_REGISTERED.toLocalizedString(function);
+          logger.warn("{}: {}", servConn.getName(), message);
+          sendError(hasResult, msg, message, servConn);
+          return;
+        } else {
+          byte functionStateOnServer = AbstractExecution.getFunctionState(functionObject.isHA(), functionObject.hasResult(), functionObject
+            .optimizeForWrite());
+          if (functionStateOnServer != functionState) {
+            String message = LocalizedStrings.FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER.toLocalizedString(function);
             logger.warn("{}: {}", servConn.getName(), message);
             sendError(hasResult, msg, message, servConn);
             return;
           }
-          else {
-            byte functionStateOnServer = AbstractExecution.getFunctionState(
-                functionObject.isHA(), functionObject.hasResult(),
-                functionObject.optimizeForWrite());
-            if (functionStateOnServer != functionState) {
-              String message = LocalizedStrings.FunctionService_FUNCTION_ATTRIBUTE_MISMATCH_CLIENT_SERVER
-                  .toLocalizedString(function);
-              logger.warn("{}: {}", servConn.getName(), message);
-              sendError(hasResult, msg, message, servConn);
-              return;
-            }
-          }
-        }
-        else {
-          functionObject = (Function)function;
-        }
-        // check if the caller is authorized to do this operation on server
-        AuthorizeRequest authzRequest = servConn.getAuthzRequest();
-        final String functionName = functionObject.getId();
-        final String regionPath = region.getFullPath();
-        ExecuteFunctionOperationContext executeContext = null;
-        if (authzRequest != null) {
-          executeContext = authzRequest.executeFunctionAuthorize(functionName,
-              regionPath, filter, args, functionObject.optimizeForWrite());
-        }
-        
-        //Construct execution 
-        AbstractExecution execution = (AbstractExecution)FunctionService.onRegion(region);
-        ChunkedMessage m = servConn.getFunctionResponseMessage();
-        m.setTransactionId(msg.getTransactionId());        
-        resultSender = new ServerToClientFunctionResultSender65(m,
-            MessageType.EXECUTE_REGION_FUNCTION_RESULT, servConn,functionObject,executeContext);
-        
-        if (isExecuteOnAllBuckets == 1) {
-          PartitionedRegion pr = (PartitionedRegion)region;
-          Set<Integer> actualBucketSet = pr.getRegionAdvisor().getBucketSet();
-          try {
-            buckets.retainAll(actualBucketSet);
-          }
-          catch (NoSuchElementException done) {
-          }
-          if (buckets.isEmpty()) {
-            throw new FunctionException("Buckets are null");
-          }
-          execution = new PartitionedRegionFunctionExecutor(
-              (PartitionedRegion)region, buckets, args, memberMappedArg,
-              resultSender, removedNodesSet, true, true);
-        }
-        else {
-          execution = new PartitionedRegionFunctionExecutor(
-              (PartitionedRegion)region, filter, args, memberMappedArg,
-              resultSender, removedNodesSet, false, true);
-        }
-        
-        if((hasResult == 1) && filter!= null &&filter.size() == 1) {
-          ServerConnection.executeFunctionOnLocalNodeOnly((byte)1);
         }
-        
-        if (logger.isDebugEnabled()) {
-          logger.debug("Executing Function: {} on Server: {} with Execution: {}", functionObject.getId(), servConn, execution);
-        }
-        if (hasResult == 1) {
-          if (function instanceof String) {
-            switch (functionState) {
-              case AbstractExecution.NO_HA_HASRESULT_NO_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, false, false)
-                    .getResult();
-                break;
-              case AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, true, false)
-                    .getResult();
-                break;
-              case AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, true, true)
-                    .getResult();
-                break;
-              case AbstractExecution.NO_HA_HASRESULT_OPTIMIZEFORWRITE:
-                execution.execute((String)function, true, false, true)
-                    .getResult();
-                break;
-            }
-          }
-          else {
-            execution.execute(functionObject).getResult();
-          }
+      } else {
+        functionObject = (Function) function;
+      }
+
+      GeodeSecurityUtil.authorizeFunctionExec(functionObject.getId());
+
+      // check if the caller is authorized to do this operation on server
+      AuthorizeRequest authzRequest = servConn.getAuthzRequest();
+      final String functionName = functionObject.getId();
+      final String regionPath = region.getFullPath();
+      ExecuteFunctionOperationContext executeContext = null;
+      if (authzRequest != null) {
+        executeContext = authzRequest.executeFunctionAuthorize(functionName, regionPath, filter, args, functionObject.optimizeForWrite());
+      }
+
+      //Construct execution
+      AbstractExecution execution = (AbstractExecution) FunctionService.onRegion(region);
+      ChunkedMessage m = servConn.getFunctionResponseMessage();
+      m.setTransactionId(msg.getTransactionId());
+      resultSender = new ServerToClientFunctionResultSender65(m, MessageType.EXECUTE_REGION_FUNCTION_RESULT, servConn, functionObject, executeContext);
+
+      if (isExecuteOnAllBuckets == 1) {
+        PartitionedRegion pr = (PartitionedRegion) region;
+        Set<Integer> actualBucketSet = pr.getRegionAdvisor().getBucketSet();
+        try {
+          buckets.retainAll(actualBucketSet);
+        } catch (NoSuchElementException done) {
         }
-        else {
-          if (function instanceof String) {
-            switch (functionState) {
-              case AbstractExecution.NO_HA_NO_HASRESULT_NO_OPTIMIZEFORWRITE:
-                execution.execute((String)function, false, false, false);
-                break;
-              case AbstractExecution.NO_HA_NO_HASRESULT_OPTIMIZEFORWRITE:
-                execution.execute((String)function, false, false, true);
-                break;
-            }
-          }
-          else {
-            execution.execute(functionObject);
-          }
+        if (buckets.isEmpty()) {
+          throw new FunctionException("Buckets are null");
         }
+        execution = new PartitionedRegionFunctionExecutor((PartitionedRegion) region, buckets, args, memberMappedArg, resultSender, removedNodesSet, true, true);
+      } else {
+        execution = new PartitionedRegionFunctionExecutor((PartitionedRegion) region, filter, args, memberMappedArg, resultSender, removedNodesSet, false, true);
       }
-      catch (IOException ioe) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioe);
-        final String message = LocalizedStrings.
-          ExecuteRegionFunction_SERVER_COULD_NOT_SEND_THE_REPLY
-            .toLocalizedString();
-        sendException(hasResult, msg, message, servConn,ioe);
+
+      if ((hasResult == 1) && filter != null && filter.size() == 1) {
+        ServerConnection.executeFunctionOnLocalNodeOnly((byte) 1);
       }
-      catch (FunctionException fe) {
-        String message = fe.getMessage();
 
-        if (fe.getCause() instanceof FunctionInvocationTargetException) {
-          if (functionObject.isHA() && logger.isDebugEnabled()) {
-            logger.debug("Exception on server while executing function: {}: {}", function, message);
-          }
-          else if (logger.isDebugEnabled()) {
-            logger.debug("Exception on server while executing function: {}: {}", function, message, fe);
-          }
-          synchronized (msg) {
-            resultSender.setException(fe);
+      if (logger.isDebugEnabled()) {
+        logger.debug("Executing Function: {} on Server: {} with Execution: {}", functionObject.getId(), servConn, execution);
+      }
+      if (hasResult == 1) {
+        if (function instanceof String) {
+          switch (functionState) {
+            case AbstractExecution.NO_HA_HASRESULT_NO_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, false, false).getResult();
+              break;
+            case AbstractExecution.HA_HASRESULT_NO_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, true, false).getResult();
+              break;
+            case AbstractExecution.HA_HASRESULT_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, true, true).getResult();
+              break;
+            case AbstractExecution.NO_HA_HASRESULT_OPTIMIZEFORWRITE:
+              execution.execute((String) function, true, false, true).getResult();
+              break;
           }
+        } else {
+          execution.execute(functionObject).getResult();
         }
-        else {
-          logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), fe);
-          sendException(hasResult, msg, message, servConn, fe);
+      } else {
+        if (function instanceof String) {
+          switch (functionState) {
+            case AbstractExecution.NO_HA_NO_HASRESULT_NO_OPTIMIZEFORWRITE:
+              execution.execute((String) function, false, false, false);
+              break;
+            case AbstractExecution.NO_HA_NO_HASRESULT_OPTIMIZEFORWRITE:
+              execution.execute((String) function, false, false, true);
+              break;
+          }
+        } else {
+          execution.execute(functionObject);
         }
       }
-      catch (Exception e) {
-        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
-        String message = e.getMessage();
-        sendException(hasResult, msg, message, servConn,e);
-      }
+    } catch (IOException ioe) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), ioe);
+      final String message = LocalizedStrings.
+        ExecuteRegionFunction_SERVER_COULD_NOT_SEND_THE_REPLY.toLocalizedString();
+      sendException(hasResult, msg, message, servConn, ioe);
+    } catch (FunctionException fe) {
+      String message = fe.getMessage();
 
-      finally{
-        handShake.setClientReadTimeout(earlierClientReadTimeout);
-        ServerConnection.executeFunctionOnLocalNodeOnly((byte)0);
+      if (fe.getCause() instanceof FunctionInvocationTargetException) {
+        if (functionObject.isHA() && logger.isDebugEnabled()) {
+          logger.debug("Exception on server while executing function: {}: {}", function, message);
+        } else if (logger.isDebugEnabled()) {
+          logger.debug("Exception on server while executing function: {}: {}", function, message, fe);
+        }
+        synchronized (msg) {
+          resultSender.setException(fe);
+        }
+      } else {
+        logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), fe);
+        sendException(hasResult, msg, message, servConn, fe);
       }
+    } catch (Exception e) {
+      logger.warn(LocalizedMessage.create(LocalizedStrings.ExecuteRegionFunction_EXCEPTION_ON_SERVER_WHILE_EXECUTIONG_FUNCTION_0, function), e);
+      String message = e.getMessage();
+      sendException(hasResult, msg, message, servConn, e);
+    } finally {
+      handShake.setClientReadTimeout(earlierClientReadTimeout);
+      ServerConnection.executeFunctionOnLocalNodeOnly((byte) 0);
     }
   }
 
-  private void sendException(byte hasResult, Message msg, String message,
-      ServerConnection servConn, Throwable e) throws IOException {
+  private void sendException(byte hasResult, Message msg, String message, ServerConnection servConn, Throwable e)
+    throws IOException {
     synchronized (msg) {
       if (hasResult == 1) {
-        writeFunctionResponseException(msg, MessageType.EXCEPTION, message,
-            servConn, e);
+        writeFunctionResponseException(msg, MessageType.EXCEPTION, message, servConn, e);
         servConn.setAsTrue(RESPONDED);
       }
     }
   }
-  
-  private void sendError(byte hasResult, Message msg, String message,
-      ServerConnection servConn) throws IOException {
+
+  private void sendError(byte hasResult, Message msg, String message, ServerConnection servConn) throws IOException {
     synchronized (msg) {
       if (hasResult == 1) {
-        writeFunctionResponseError(msg,
-            MessageType.EXECUTE_REGION_FUNCTION_ERROR, message, servConn);
+        writeFunctionResponseError(msg, MessageType.EXECUTE_REGION_FUNCTION_ERROR, message, servConn);
         servConn.setAsTrue(RESPONDED);
       }
     }
   }
-  
+
   protected static void writeFunctionResponseException(Message origMsg,
-      int messageType, String message, ServerConnection servConn, Throwable e)
-      throws IOException {
+                                                       int messageType,
+                                                       String message,
+                                                       ServerConnection servConn,
+                                                       Throwable e) throws IOException {
     ChunkedMessage functionResponseMsg = servConn.getFunctionResponseMessage();
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     int numParts = 0;
     if (functionResponseMsg.headerHasBeenSent()) {
-      if (e instanceof FunctionException
-          && e.getCause() instanceof InternalFunctionInvocationTargetException) {
+      if (e instanceof FunctionException && e.getCause() instanceof InternalFunctionInvocationTargetException) {
         functionResponseMsg.setNumberOfParts(3);
         functionResponseMsg.addObjPart(e);
         functionResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
-        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException)e
-            .getCause();
+        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException) e.getCause();
         functionResponseMsg.addObjPart(fe.getFailedNodeSet());
         numParts = 3;
-      }
-      else {
+      } else {
         functionResponseMsg.setNumberOfParts(2);
         functionResponseMsg.addObjPart(e);
         functionResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
@@ -386,22 +350,18 @@ public class ExecuteRegionFunctionSingleHop extends BaseCommand {
       functionResponseMsg.setServerConnection(servConn);
       functionResponseMsg.setLastChunkAndNumParts(true, numParts);
       functionResponseMsg.sendChunk(servConn);
-    }
-    else {
+    } else {
       chunkedResponseMsg.setMessageType(messageType);
       chunkedResponseMsg.setTransactionId(origMsg.getTransactionId());
       chunkedResponseMsg.sendHeader();
-      if (e instanceof FunctionException
-          && e.getCause() instanceof InternalFunctionInvocationTargetException) {
+      if (e instanceof FunctionException && e.getCause() instanceof InternalFunctionInvocationTargetException) {
         chunkedResponseMsg.setNumberOfParts(3);
         chunkedResponseMsg.addObjPart(e);
         chunkedResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));
-        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException)e
-            .getCause();
+        InternalFunctionInvocationTargetException fe = (InternalFunctionInvocationTargetException) e.getCause();
         chunkedResponseMsg.addObjPart(fe.getFailedNodeSet());
         numParts = 3;
-      }
-      else {
+      } else {
         chunkedResponseMsg.setNumberOfParts(2);
         chunkedResponseMsg.addObjPart(e);
         chunkedResponseMsg.addStringPart(BaseCommand.getExceptionTrace(e));

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/536c13bd/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
index a807440..5a49241 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetAll.java
@@ -52,8 +52,7 @@ public class GetAll extends BaseCommand {
   }
 
   @Override
-  public void cmdExecute(Message msg, ServerConnection servConn, long start)
-      throws IOException, InterruptedException {
+  public void cmdExecute(Message msg, ServerConnection servConn, long start) throws IOException, InterruptedException {
     Part regionNamePart = null, keysPart = null;
     String regionName = null;
     Object[] keys = null;
@@ -69,8 +68,7 @@ public class GetAll extends BaseCommand {
     keysPart = msg.getPart(1);
     try {
       keys = (Object[]) keysPart.getObject();
-    }
-    catch (Exception e) {
+    } catch (Exception e) {
       writeChunkedException(msg, e, false, servConn);
       servConn.setAsTrue(RESPONDED);
       return;
@@ -78,21 +76,19 @@ public class GetAll extends BaseCommand {
 
     if (logger.isDebugEnabled()) {
       StringBuffer buffer = new StringBuffer();
-      buffer
-        .append(servConn.getName())
-        .append(": Received getAll request (")
-        .append(msg.getPayloadLength())
-        .append(" bytes) from ")
-        .append(servConn.getSocketString())
-        .append(" for region ")
-        .append(regionName)
-        .append(" keys ");
+      buffer.append(servConn.getName())
+            .append(": Received getAll request (")
+            .append(msg.getPayloadLength())
+            .append(" bytes) from ")
+            .append(servConn.getSocketString())
+            .append(" for region ")
+            .append(regionName)
+            .append(" keys ");
       if (keys != null) {
-        for (int i=0; i<keys.length; i++) {
+        for (int i = 0; i < keys.length; i++) {
           buffer.append(keys[i]).append(" ");
         }
-      }
-      else {
+      } else {
         buffer.append("NULL");
       }
       if (logger.isDebugEnabled()) {
@@ -103,50 +99,48 @@ public class GetAll extends BaseCommand {
     // Process the getAll request
     if (regionName == null) {
       String message = null;
-//      if (regionName == null) (can only be null) 
+      //      if (regionName == null) (can only be null)
       {
         message = LocalizedStrings.GetAll_THE_INPUT_REGION_NAME_FOR_THE_GETALL_REQUEST_IS_NULL.toLocalizedString();
       }
       logger.warn("{}: {}", servConn.getName(), message);
-      writeChunkedErrorResponse(msg, MessageType.GET_ALL_DATA_ERROR, message,
-          servConn);
+      writeChunkedErrorResponse(msg, MessageType.GET_ALL_DATA_ERROR, message, servConn);
       servConn.setAsTrue(RESPONDED);
+      return;
     }
-    else {
-      LocalRegion region = (LocalRegion)crHelper.getRegion(regionName);
-      if (region == null) {
-        String reason = " was not found during getAll request";
-        writeRegionDestroyedEx(msg, regionName, reason, servConn);
-        servConn.setAsTrue(RESPONDED);
-      }
-      else {
-        // Send header
-        ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
-        chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
-        chunkedResponseMsg.setTransactionId(msg.getTransactionId());
-        chunkedResponseMsg.sendHeader();
 
-        // Send chunk response
-        try {
-          fillAndSendGetAllResponseChunks(region, regionName, keys, servConn);
-          servConn.setAsTrue(RESPONDED);
-        }
-        catch (Exception e) {
-          // If an interrupted exception is thrown , rethrow it
-          checkForInterrupt(servConn, e);
+    LocalRegion region = (LocalRegion) crHelper.getRegion(regionName);
+    if (region == null) {
+      String reason = " was not found during getAll request";
+      writeRegionDestroyedEx(msg, regionName, reason, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
+    }
+    // Send header
+    ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
+    chunkedResponseMsg.setMessageType(MessageType.RESPONSE);
+    chunkedResponseMsg.setTransactionId(msg.getTransactionId());
+    chunkedResponseMsg.sendHeader();
 
-          // Otherwise, write an exception message and continue
-          writeChunkedException(msg, e, false, servConn);
-          servConn.setAsTrue(RESPONDED);
-          return;
-        }
-      }
+    // Send chunk response
+    try {
+      fillAndSendGetAllResponseChunks(region, regionName, keys, servConn);
+      servConn.setAsTrue(RESPONDED);
+    } catch (Exception e) {
+      // If an interrupted exception is thrown , rethrow it
+      checkForInterrupt(servConn, e);
+
+      // Otherwise, write an exception message and continue
+      writeChunkedException(msg, e, false, servConn);
+      servConn.setAsTrue(RESPONDED);
+      return;
     }
   }
 
   private void fillAndSendGetAllResponseChunks(Region region,
-      String regionName, Object[] keys, ServerConnection servConn)
-      throws IOException {
+                                               String regionName,
+                                               Object[] keys,
+                                               ServerConnection servConn) throws IOException {
 
     // Interpret null keys object as a request to get all key,value entry pairs
     // of the region; otherwise iterate each key and perform the get behavior.
@@ -155,8 +149,7 @@ public class GetAll extends BaseCommand {
     if (keys != null) {
       allKeysIter = null;
       numKeys = keys.length;
-    }
-    else {
+    } else {
       Set allKeys = region.keySet();
       allKeysIter = allKeys.iterator();
       numKeys = allKeys.size();
@@ -167,7 +160,7 @@ public class GetAll extends BaseCommand {
     AuthorizeRequestPP postAuthzRequest = servConn.getPostAuthzRequest();
     Request request = (Request) Request.getCommand();
     Object[] valueAndIsObject = new Object[3];
-    for (int i=0; i<numKeys; i++) {
+    for (int i = 0; i < numKeys; i++) {
       // Send the intermediate chunk if necessary
       if (values.size() == maximumChunkSize) {
         // Send the chunk and clear the list
@@ -178,8 +171,7 @@ public class GetAll extends BaseCommand {
       Object key;
       if (keys != null) {
         key = keys[i];
-      }
-      else {
+      } else {
         key = allKeysIter.next();
       }
       if (logger.isDebugEnabled()) {
@@ -193,22 +185,32 @@ public class GetAll extends BaseCommand {
           if (logger.isDebugEnabled()) {
             logger.debug("{}: Passed GET pre-authorization for key={}", servConn.getName(), key);
           }
-        }
-        catch (NotAuthorizedException ex) {
-          logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {servConn.getName(), key}), ex);
+        } catch (NotAuthorizedException ex) {
+          logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {
+            servConn.getName(),
+            key
+          }), ex);
           values.addExceptionPart(key, ex);
           continue;
         }
       }
 
-      GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+      try {
+        GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
+      } catch (NotAuthorizedException ex) {
+        logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {
+          servConn.getName(),
+          key
+        }), ex);
+        values.addExceptionPart(key, ex);
+        continue;
+      }
 
       // Get the value and update the statistics. Do not deserialize
       // the value if it is a byte[].
       // Getting a value in serialized form is pretty nasty. I split this out
       // so the logic can be re-used by the CacheClientProxy.
-      request.getValueAndIsObject(region, key,
-          null, servConn, valueAndIsObject);
+      request.getValueAndIsObject(region, key, null, servConn, valueAndIsObject);
       Object value = valueAndIsObject[0];
       boolean isObject = ((Boolean) valueAndIsObject[1]).booleanValue();
       if (logger.isDebugEnabled()) {
@@ -217,22 +219,22 @@ public class GetAll extends BaseCommand {
 
       if (postAuthzRequest != null) {
         try {
-          getContext = postAuthzRequest.getAuthorize(regionName, key, value,
-              isObject, getContext);
+          getContext = postAuthzRequest.getAuthorize(regionName, key, value, isObject, getContext);
           byte[] serializedValue = getContext.getSerializedValue();
           if (serializedValue == null) {
             value = getContext.getObject();
-          }
-          else {
+          } else {
             value = serializedValue;
           }
           isObject = getContext.isObject();
           if (logger.isDebugEnabled()) {
             logger.debug("{}: Passed GET post-authorization for key={}: {}", servConn.getName(), key, value);
           }
-        }
-        catch (NotAuthorizedException ex) {
-          logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {servConn.getName(), key}), ex);
+        } catch (NotAuthorizedException ex) {
+          logger.warn(LocalizedMessage.create(LocalizedStrings.GetAll_0_CAUGHT_THE_FOLLOWING_EXCEPTION_ATTEMPTING_TO_GET_VALUE_FOR_KEY_1, new Object[] {
+            servConn.getName(),
+            key
+          }), ex);
           values.addExceptionPart(key, ex);
           continue;
         }
@@ -249,20 +251,23 @@ public class GetAll extends BaseCommand {
       values.addObjectPart(key, value, isObject, null);
     }
 
-  // Send the last chunk even if the list is of zero size.
+    // Send the last chunk even if the list is of zero size.
     sendGetAllResponseChunk(region, values, true, servConn);
     servConn.setAsTrue(RESPONDED);
   }
 
-  private static void sendGetAllResponseChunk(Region region, ObjectPartList list,
-      boolean lastChunk, ServerConnection servConn) throws IOException {
+  private static void sendGetAllResponseChunk(Region region,
+                                              ObjectPartList list,
+                                              boolean lastChunk,
+                                              ServerConnection servConn) throws IOException {
     ChunkedMessage chunkedResponseMsg = servConn.getChunkedResponseMessage();
     chunkedResponseMsg.setNumberOfParts(1);
     chunkedResponseMsg.setLastChunk(lastChunk);
     chunkedResponseMsg.addObjPart(list, zipValues);
 
     if (logger.isDebugEnabled()) {
-      logger.debug("{}: Sending {} getAll response chunk for region={} values={} chunk=<{}>", servConn.getName(), (lastChunk ? " last " : " "), region.getFullPath(), list, chunkedResponseMsg);
+      logger.debug("{}: Sending {} getAll response chunk for region={} values={} chunk=<{}>", servConn.getName(), (lastChunk ? " last " : " "), region
+        .getFullPath(), list, chunkedResponseMsg);
     }
 
     chunkedResponseMsg.sendChunk(servConn);


[24/50] [abbrv] incubator-geode git commit: GEODE-1574: Remove quotes around negative numbers in command option values

Posted by ji...@apache.org.
GEODE-1574: Remove quotes around negative numbers in command option values

* Test for CommandRequest
* This closes #172


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

Branch: refs/heads/feature/GEODE-1571
Commit: f38d6964ba5840ea192db12be2e2ed67d3379dd2
Parents: fc42db8
Author: gmeilen <gr...@gmail.com>
Authored: Tue Jun 21 17:01:28 2016 -0700
Committer: Kirk Lund <kl...@apache.org>
Committed: Fri Jun 24 11:21:58 2016 -0700

----------------------------------------------------------------------
 .../management/internal/cli/CommandRequest.java |  22 ++-
 .../management/internal/cli/GfshParser.java     |   8 +-
 .../internal/cli/CommandRequestTest.java        | 161 +++++++++++++++++++
 .../cli/commands/CliCommandTestBase.java        |   2 +-
 .../commands/GemfireDataCommandsDUnitTest.java  |   1 -
 .../DataCommandsOverHttpDistributedTest.java    | 150 +++++++++++++++++
 6 files changed, 339 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandRequest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandRequest.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandRequest.java
index ffbad51..405925f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandRequest.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/CommandRequest.java
@@ -113,7 +113,19 @@ public class CommandRequest {
   }
 
   public Map<String, String> getParameters() {
-    final Map<String, String> parameters = new HashMap<String, String>(getUserParameters());
+    final Map<String, String> parameters = new HashMap<>();
+    for (Map.Entry<String, String> mapEntry : getUserParameters().entrySet()) {
+      String key = mapEntry.getKey();
+      String value = mapEntry.getValue();
+
+      if (hasQuotesAroundNegativeNumber(value)){
+        String trimmed = value.substring(1, value.length() - 1);
+        parameters.put(key, trimmed);
+      }
+      else {
+        parameters.put(key, value);
+      }
+    }
     parameters.putAll(getCustomParameters());
     return Collections.unmodifiableMap(parameters);
   }
@@ -130,4 +142,12 @@ public class CommandRequest {
     return getParseResult().getParamValueStrings();
   }
 
+  private boolean hasQuotesAroundNegativeNumber(String value) {
+    if (value == null) {
+      return false;
+    } else {
+      return value.startsWith("\"") && value.endsWith("\"") && value.matches("\"-[0-9]+\"");
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
index 9023ebe..4d2564f 100755
--- a/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/management/internal/cli/GfshParser.java
@@ -893,7 +893,6 @@ public class GfshParser implements Parser {
           preConfigureConverters(commandTarget);
 
           try {
-            // TODO: next call invokes HyphenFormatter
             parse = commandTarget.getOptionParser().parse(gfshMethodTarget.getRemainingBuffer());
           } catch (CliException ce) {
             if (ce instanceof CliCommandOptionException) {
@@ -1090,8 +1089,13 @@ public class GfshParser implements Parser {
       }
 
       // Remove outer single or double quotes if found
-      if (string != null && ((string.endsWith("\"") && string.endsWith("\"")) || (string.startsWith("\'") && string.endsWith("\'")))) {
+      boolean hasDoubleQuotes = string.startsWith("\"") && string.endsWith("\"");
+      boolean hasSingleQuotes = string.startsWith("\'") && string.endsWith("\'");
+
+      while (string != null && (hasDoubleQuotes || hasSingleQuotes)) {
         string = string.substring(1, string.length() - 1);
+        hasDoubleQuotes = string.startsWith("\"") && string.endsWith("\"");
+        hasSingleQuotes = string.startsWith("\'") && string.endsWith("\'");
       }
 
       if (converter != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandRequestTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandRequestTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandRequestTest.java
new file mode 100644
index 0000000..8e5d27e
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/CommandRequestTest.java
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.management.internal.cli;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.*;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.gemstone.gemfire.test.junit.categories.UnitTest;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Categories.ExcludeCategory;
+import org.junit.experimental.categories.Category;
+
+@Category(UnitTest.class)
+public class CommandRequestTest {
+
+  private Map<String, String> paramValues;
+  private GfshParseResult mockParseResult;
+  private Map<String, String> mockEnvironment;
+  private CommandRequest commandRequest;
+
+  @Before
+  public void setUp() {
+    this.paramValues = new HashMap<>();
+
+    this.mockParseResult = mock(GfshParseResult.class);
+    when(this.mockParseResult.getUserInput()).thenReturn("rebalance --simulate=true --time-out=-1");
+    when(this.mockParseResult.getParamValueStrings()).thenReturn(this.paramValues);
+
+    this.mockEnvironment = new HashMap<>();
+    this.commandRequest = new CommandRequest(this.mockParseResult, this.mockEnvironment, null);
+  }
+
+  @Test
+  public void getParametersRemovesQuotesAroundNegativeNumbers() {
+    String key = "time-out";
+    String value = "\"-1\"";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value.substring(1, value.length()-1));
+  }
+
+  @Test
+  public void getParametersWithNullValue() {
+    String key = "key-with-null-value";
+    String value = null;
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isNull();
+  }
+
+  @Test
+  public void getParametersWithEmptyValue() {
+    String key = "key-with-empty-value";
+    String value = "";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEmpty();
+  }
+
+  @Test
+  public void getParametersWithEmptyQuotesValue() {
+    String key = "key-with-empty-quotes-value";
+    String value = "\"\"";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithNumberValue() {
+    String key = "key-with-number-value";
+    String value = "1";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithNegativeNumberValue() {
+    String key = "key-with-negative-number-value";
+    String value = "-1";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithHyphenAlphaValue() {
+    String key = "key-with-hyphen-alpha-value";
+    String value = "-A";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithHyphenHyphenNumberValue() {
+    String key = "key-with-hyphen-alpha-value";
+    String value = "--1";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithQuotesAndMoreValue() {
+    String key = "key-with-hyphen-alpha-value";
+    String value = "\"-1 this is giberish\"";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+
+  @Test
+  public void getParametersWithLotsaQuotesValue() {
+    String key = "key-with-hyphen-alpha-value";
+    String value = "\"\"-1\"\"";
+    this.paramValues.put(key, value);
+
+    Map<String, String> parameters = this.commandRequest.getParameters();
+    assertThat(parameters).containsKey(key);
+    assertThat(parameters.get(key)).isEqualTo(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
index 1b9d770..ab2ce62 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/CliCommandTestBase.java
@@ -356,7 +356,7 @@ public abstract class CliCommandTestBase extends JUnit4CacheTestCase {
 
     CommandResult result = null;
     try {
-      result = (CommandResult) shell.getResult();
+      result = (CommandResult) shell.getResult(); // TODO: this can result in ClassCastException if command resulted in error
     } catch (InterruptedException ex) {
       error("shell received InterruptedException");
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
index b0272b2..902c3b5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/GemfireDataCommandsDUnitTest.java
@@ -1856,7 +1856,6 @@ public class GemfireDataCommandsDUnitTest extends CliCommandTestBase {
     }
   }
 
-  @Ignore("TODO: enable test after GEODE-1574 is fixed")
   @Test
   public void testSimulateForEntireDSWithTimeout() {
     setupTestRebalanceForEntireDS();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f38d6964/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommandsOverHttpDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommandsOverHttpDistributedTest.java b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommandsOverHttpDistributedTest.java
new file mode 100644
index 0000000..3c673f5
--- /dev/null
+++ b/geode-web/src/test/java/com/gemstone/gemfire/management/internal/cli/commands/DataCommandsOverHttpDistributedTest.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package com.gemstone.gemfire.management.internal.cli.commands;
+
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.*;
+import static com.gemstone.gemfire.test.dunit.Wait.*;
+import static org.junit.Assert.*;
+
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionFactory;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.management.DistributedRegionMXBean;
+import com.gemstone.gemfire.management.ManagementService;
+import com.gemstone.gemfire.management.cli.Result;
+import com.gemstone.gemfire.management.internal.cli.result.CommandResult;
+import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.SerializableRunnable;
+import com.gemstone.gemfire.test.dunit.VM;
+import com.gemstone.gemfire.test.dunit.WaitCriterion;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.contrib.java.lang.system.ProvideSystemProperty;
+import org.junit.experimental.categories.Category;
+
+@Category(DistributedTest.class)
+public class DataCommandsOverHttpDistributedTest extends CliCommandTestBase {
+
+  private static final String REBALANCE_REGION_NAME = DataCommandsOverHttpDistributedTest.class.getSimpleName() + "Region";
+
+  @ClassRule
+  public static ProvideSystemProperty provideSystemProperty = new ProvideSystemProperty(CliCommandTestBase.USE_HTTP_SYSTEM_PROPERTY, "true");
+
+  @Test
+  public void testSimulateForEntireDSWithTimeout() {
+    setupTestRebalanceForEntireDS();
+    //check if DistributedRegionMXBean is available so that command will not fail
+    final VM manager = Host.getHost(0).getVM(0);
+    manager.invoke(checkRegionMBeans);
+
+    getLogWriter().info("testSimulateForEntireDS verified MBean and executing command");
+
+    String command = "rebalance --simulate=true --time-out=-1";
+
+    CommandResult cmdResult = executeCommand(command);
+
+    getLogWriter().info("testSimulateForEntireDS just after executing " + cmdResult);
+
+    if (cmdResult != null) {
+      String stringResult = commandResultToString(cmdResult);
+      getLogWriter().info("testSimulateForEntireDS stringResult : " + stringResult);
+      assertEquals(Result.Status.OK, cmdResult.getStatus());
+    } else {
+      fail("testRebalanceForIncludeRegionFunction failed as did not get CommandResult");
+    }
+  }
+
+  SerializableRunnable checkRegionMBeans = new SerializableRunnable() {
+    @Override
+    public void run() {
+      final WaitCriterion waitForMaangerMBean = new WaitCriterion() {
+        @Override
+        public boolean done() {
+          final ManagementService service = ManagementService.getManagementService(getCache());
+          final DistributedRegionMXBean bean = service.getDistributedRegionMXBean(
+            Region.SEPARATOR + REBALANCE_REGION_NAME);
+          if (bean == null) {
+            getLogWriter().info("Still probing for checkRegionMBeans ManagerMBean");
+            return false;
+          } else {
+            // verify that bean is proper before executing tests
+            if (bean.getMembers() != null && bean.getMembers().length > 1 && bean.getMemberCount() > 0 && service.getDistributedSystemMXBean().listRegions().length >= 2) {
+              return true;
+            } else {
+              return false;
+            }
+          }
+        }
+
+        @Override
+        public String description() {
+          return "Probing for testRebalanceCommandForSimulateWithNoMember ManagerMBean";
+        }
+      };
+      waitForCriterion(waitForMaangerMBean, 2 * 60 * 1000, 2000, true);
+      DistributedRegionMXBean bean = ManagementService.getManagementService(getCache()).getDistributedRegionMXBean(
+        "/" + REBALANCE_REGION_NAME);
+      assertNotNull(bean);
+    }
+  };
+
+  void setupTestRebalanceForEntireDS() {
+    final VM vm1 = Host.getHost(0).getVM(1);
+    final VM vm2 = Host.getHost(0).getVM(2);
+    setUpJmxManagerOnVm0ThenConnect(null);
+
+    vm1.invoke(new SerializableRunnable() {
+      public void run() {
+
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create(REBALANCE_REGION_NAME);
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+        region = dataRegionFactory.create(REBALANCE_REGION_NAME + "Another1");
+        for (int i = 0; i < 100; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+
+    vm2.invoke(new SerializableRunnable() {
+      public void run() {
+
+        // no need to close cache as it will be closed as part of teardown2
+        Cache cache = getCache();
+
+        RegionFactory<Integer, Integer> dataRegionFactory = cache.createRegionFactory(RegionShortcut.PARTITION);
+        Region region = dataRegionFactory.create(REBALANCE_REGION_NAME);
+        for (int i = 0; i < 100; i++) {
+          region.put("key" + (i + 400), "value" + (i + 400));
+        }
+        region = dataRegionFactory.create(REBALANCE_REGION_NAME + "Another2");
+        for (int i = 0; i < 10; i++) {
+          region.put("key" + (i + 200), "value" + (i + 200));
+        }
+      }
+    });
+  }
+
+}


[33/50] [abbrv] incubator-geode git commit: GEODE-1576: Website needs to be upgraded to include WAN and CQ

Posted by ji...@apache.org.
GEODE-1576: Website needs to be upgraded to include WAN and CQ

upgrade nanoc to 4.2.0


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

Branch: refs/heads/feature/GEODE-1571
Commit: e987e5d90347c7520cd04043ee6d92d633322a76
Parents: 30928d9
Author: Swapnil Bawaskar <sb...@pivotal.io>
Authored: Fri Jun 24 14:58:31 2016 -0700
Committer: Swapnil Bawaskar <sb...@pivotal.io>
Committed: Fri Jun 24 16:05:36 2016 -0700

----------------------------------------------------------------------
 geode-site/website/README.md      | 2 +-
 geode-site/website/lib/default.rb | 4 ++--
 geode-site/website/nanoc.yaml     | 2 ++
 3 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e987e5d9/geode-site/website/README.md
----------------------------------------------------------------------
diff --git a/geode-site/website/README.md b/geode-site/website/README.md
index 2ae17b4..60deb69 100644
--- a/geode-site/website/README.md
+++ b/geode-site/website/README.md
@@ -22,7 +22,7 @@ Install Pygments (Python-based syntax coloring library):
 
 Install Nanoc and other Ruby Gems needed:
 
-    $ sudo gem install nanoc -v 3.8.0
+    $ sudo gem install nanoc -v 4.2.0
     $ sudo gem install pygments.rb htmlentities pandoc-ruby nokogiri rack mime-types adsf
 
 ## How to change/update the website

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e987e5d9/geode-site/website/lib/default.rb
----------------------------------------------------------------------
diff --git a/geode-site/website/lib/default.rb b/geode-site/website/lib/default.rb
index dababa8..95383f2 100644
--- a/geode-site/website/lib/default.rb
+++ b/geode-site/website/lib/default.rb
@@ -22,7 +22,7 @@ include Nanoc::Helpers::Rendering
 require 'pandoc-ruby'
 require 'htmlentities'
 
-class PandocFilter < Nanoc3::Filter
+class PandocFilter < Nanoc::Filter
   identifier :pandoc
   type :text
 
@@ -31,7 +31,7 @@ class PandocFilter < Nanoc3::Filter
   end
 end
 
-class FencedCodeBlock < Nanoc3::Filter
+class FencedCodeBlock < Nanoc::Filter
   identifier :fenced_code_block
 
   def run(content, params={})

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/e987e5d9/geode-site/website/nanoc.yaml
----------------------------------------------------------------------
diff --git a/geode-site/website/nanoc.yaml b/geode-site/website/nanoc.yaml
index 764bb74..25fa01b 100644
--- a/geode-site/website/nanoc.yaml
+++ b/geode-site/website/nanoc.yaml
@@ -46,6 +46,7 @@ prune:
   # .git, .svn etc.
   exclude: [ '.git', '.hg', '.svn', 'CVS' ]
 
+string_pattern_type: legacy
 # The data sources where nanoc loads its data from. This is an array of
 # hashes; each array element represents a single data source. By default,
 # there is only a single data source that reads data from the \u201ccontent/\u201d and
@@ -55,6 +56,7 @@ data_sources:
     # The type is the identifier of the data source. By default, this will be
     # `filesystem_unified`.
     type: filesystem_unified
+    identifier_type: legacy
 
     # The path where items should be mounted (comparable to mount points in
     # Unix-like systems). This is \u201c/\u201d by default, meaning that items will have


[32/50] [abbrv] incubator-geode git commit: GEODE-11: Fixing compilation failure from LuceneQueriesBase

Posted by ji...@apache.org.
GEODE-11: Fixing compilation failure from LuceneQueriesBase


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

Branch: refs/heads/feature/GEODE-1571
Commit: 850e143d35c1cde4975a312349b33b4cbd97d6ce
Parents: 8bbea5b
Author: Dan Smith <up...@apache.org>
Authored: Fri Jun 24 14:58:48 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Fri Jun 24 14:59:20 2016 -0700

----------------------------------------------------------------------
 .../java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/850e143d/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
index 301d5af..1f3795c 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesBase.java
@@ -105,7 +105,7 @@ public abstract class LuceneQueriesBase extends LuceneDUnitTest {
       LuceneQuery query = service.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, index ->  {
         return new TermQuery(new Term("text", "world"));
       });
-      final LuceneQueryResults results = query.search();
+      final PageableLuceneQueryResults results = query.findPages();
       assertEquals(3, results.size());
     });
   }


[39/50] [abbrv] incubator-geode git commit: GEODE-1571: security for containsKey

Posted by ji...@apache.org.
GEODE-1571: security for containsKey


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

Branch: refs/heads/feature/GEODE-1571
Commit: 8493cfee52cb2b2e5e4577a1de120d55d4806258
Parents: 4ac886e
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Jun 24 21:06:33 2016 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri Jun 24 21:06:33 2016 -0700

----------------------------------------------------------------------
 .../internal/cache/tier/sockets/command/ContainsKey.java       | 2 +-
 .../internal/cache/tier/sockets/command/ContainsKey66.java     | 2 +-
 .../IntegratedClientContainsKeyAuthDistributedTest.java        | 6 +++---
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8493cfee/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey.java
index 3692fda..10a9067 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey.java
@@ -117,7 +117,7 @@ public class ContainsKey extends BaseCommand {
       return;
     }
 
-    GeodeSecurityUtil.authorizeDataRead();
+    GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
 
     AuthorizeRequest authzRequest = servConn.getAuthzRequest();
     if (authzRequest != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8493cfee/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey66.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey66.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey66.java
index 5f11033..1d6c52d 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey66.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/ContainsKey66.java
@@ -120,7 +120,7 @@ public class ContainsKey66 extends BaseCommand {
       return;
     }
 
-    GeodeSecurityUtil.authorizeDataRead();
+    GeodeSecurityUtil.authorizeRegionRead(regionName, key.toString());
 
     AuthorizeRequest authzRequest = servConn.getAuthzRequest();
     if (authzRequest != null) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/8493cfee/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
index 093d1ae..87535db 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
@@ -33,16 +33,16 @@ public class IntegratedClientContainsKeyAuthDistributedTest extends AbstractInte
   public void testContainsKey() throws InterruptedException {
 
     AsyncInvocation ai1 = client1.invokeAsync(() -> {
-      Cache cache = SecurityTestUtils.createCacheClient("dataReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
+      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
       assertTrue(region.containsKeyOnServer("key1"));
+      assertNotAuthorized(() -> region.containsKeyOnServer("key3"), "DATA:READ:AuthRegion:key3");
     });
 
     AsyncInvocation ai2 = client2.invokeAsync(() -> {
       Cache cache = SecurityTestUtils.createCacheClient("authRegionReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
-
-      assertNotAuthorized(() -> region.containsKeyOnServer("key3"), "DATA:READ");
+      region.containsKeyOnServer("key3");
     });
     ai1.join();
     ai2.join();


[46/50] [abbrv] incubator-geode git commit: Created GetEntry Test and removed use of SecurityTestUtils

Posted by ji...@apache.org.
Created GetEntry Test and removed use of SecurityTestUtils


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

Branch: refs/heads/feature/GEODE-1571
Commit: 98244ecef5fed7c06481d59852ed20bf6f766e2b
Parents: 536c13b
Author: gmeilen <gr...@gmail.com>
Authored: Mon Jun 27 11:19:50 2016 -0700
Committer: gmeilen <gr...@gmail.com>
Committed: Mon Jun 27 11:19:50 2016 -0700

----------------------------------------------------------------------
 .../cache/tier/sockets/command/GetEntry70.java  |  1 +
 ...ractIntegratedClientAuthDistributedTest.java | 71 +++++++++++++----
 .../security/IntegratedClientAuthDUnitTest.java | 36 ++++++++-
 ...tedClientContainsKeyAuthDistributedTest.java | 27 ++++++-
 ...entDestroyInvalidateAuthDistributedTest.java | 31 +++++---
 ...dClientDestroyRegionAuthDistributedTest.java | 31 ++++++--
 ...gratedClientGetEntryAuthDistributedTest.java | 80 ++++++++++++++++++++
 ...tegratedClientGetPutAuthDistributedTest.java | 34 ++++++---
 ...ntegratedClientQueryAuthDistributedTest.java | 12 ++-
 ...tedClientRegionClearAuthDistributedTest.java | 20 +++--
 ...ientRegisterInterestAuthDistributedTest.java | 76 ++++++++++++++-----
 ...ratedClientRemoveAllAuthDistributedTest.java | 28 ++++---
 ...ntUnregisterInterestAuthDistributedTest.java | 12 ++-
 13 files changed, 369 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetEntry70.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetEntry70.java b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetEntry70.java
index d975f1b..715da7f 100644
--- a/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetEntry70.java
+++ b/geode-core/src/main/java/com/gemstone/gemfire/internal/cache/tier/sockets/command/GetEntry70.java
@@ -24,6 +24,7 @@ import com.gemstone.gemfire.internal.cache.NonLocalRegionEntry;
 import com.gemstone.gemfire.internal.cache.tier.Command;
 import com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection;
 import com.gemstone.gemfire.internal.cache.versions.VersionTag;
+import com.gemstone.gemfire.internal.security.GeodeSecurityUtil;
 
 /**
  * getEntry(key) operation performed on server.

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/AbstractIntegratedClientAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/AbstractIntegratedClientAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/AbstractIntegratedClientAuthDistributedTest.java
index 4a2db57..5243bd5 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/AbstractIntegratedClientAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/AbstractIntegratedClientAuthDistributedTest.java
@@ -16,21 +16,39 @@
  */
 package com.gemstone.gemfire.security;
 
-import static com.gemstone.gemfire.security.SecurityTestUtils.*;
+import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
+import static com.gemstone.gemfire.test.dunit.Assert.fail;
+import static com.gemstone.gemfire.test.dunit.DistributedTestUtils.getDUnitLocatorPort;
+import static com.gemstone.gemfire.test.dunit.LogWriterUtils.getLogWriter;
+import static com.gemstone.gemfire.test.dunit.NetworkUtils.getIPLiteral;
 import static org.assertj.core.api.Assertions.*;
 import static org.junit.Assert.*;
 
+import java.util.Properties;
+
+import com.gemstone.gemfire.cache.AttributesFactory;
+import com.gemstone.gemfire.cache.Cache;
+import com.gemstone.gemfire.cache.CacheFactory;
+import com.gemstone.gemfire.cache.DataPolicy;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.RegionAttributes;
+import com.gemstone.gemfire.cache.RegionShortcut;
+import com.gemstone.gemfire.cache.Scope;
+import com.gemstone.gemfire.cache.server.CacheServer;
 import com.gemstone.gemfire.management.internal.security.JSONAuthorization;
+import com.gemstone.gemfire.security.templates.UserPasswordAuthInit;
 import com.gemstone.gemfire.test.dunit.Host;
+import com.gemstone.gemfire.test.dunit.Invoke;
 import com.gemstone.gemfire.test.dunit.VM;
-import com.gemstone.gemfire.test.dunit.internal.JUnit4DistributedTestCase;
+import com.gemstone.gemfire.test.dunit.cache.internal.JUnit4CacheTestCase;
 
 import org.assertj.core.api.ThrowableAssert.ThrowingCallable;
 import org.junit.After;
 import org.junit.Before;
 
-public class AbstractIntegratedClientAuthDistributedTest extends JUnit4DistributedTestCase {
+public class AbstractIntegratedClientAuthDistributedTest extends JUnit4CacheTestCase {
+
+  protected static final String REGION_NAME = "AuthRegion";
 
   protected VM client1 = null;
   protected VM client2 = null;
@@ -40,14 +58,30 @@ public class AbstractIntegratedClientAuthDistributedTest extends JUnit4Distribut
   @Before
   public void before() throws Exception{
     final Host host = Host.getHost(0);
-    client1 = host.getVM(1);
-    client2 = host.getVM(2);
-    client3 = host.getVM(3);
+    this.client1 = host.getVM(1);
+    this.client2 = host.getVM(2);
+    this.client3 = host.getVM(3);
 
     JSONAuthorization.setUpWithJsonFile("clientServer.json");
-    serverPort =  SecurityTestUtils.createCacheServer(JSONAuthorization.class.getName()+".create");
-    Region region = getCache().getRegion(SecurityTestUtils.REGION_NAME);
-    assertEquals(0, region.size());
+
+    Properties props = new Properties();
+    props.setProperty(SECURITY_CLIENT_AUTHENTICATOR, JSONAuthorization.class.getName()+".create");
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "");
+    props.setProperty(SECURITY_LOG_LEVEL, "finest");
+
+    getSystem(props);
+
+    Cache cache = getCache();
+
+    Region region = cache.createRegionFactory(RegionShortcut.REPLICATE).create(REGION_NAME);
+
+    CacheServer server1 = cache.addCacheServer();
+    server1.setPort(0);
+    server1.start();
+
+    this.serverPort = server1.getPort();
+
     for (int i = 0; i < 5; i++) {
       String key = "key" + i;
       String value = "value" + i;
@@ -56,11 +90,9 @@ public class AbstractIntegratedClientAuthDistributedTest extends JUnit4Distribut
     assertEquals(5, region.size());
   }
 
-  @After
-  public void after(){
-    client1.invoke(() -> closeCache());
-    client2.invoke(() -> closeCache());
-    client3.invoke(() -> closeCache());
+  @Override
+  public void preTearDownCacheTestCase() throws Exception {
+    Invoke.invokeInEveryVM(()->closeCache());
     closeCache();
   }
 
@@ -68,4 +100,15 @@ public class AbstractIntegratedClientAuthDistributedTest extends JUnit4Distribut
     assertThatThrownBy(shouldRaiseThrowable).hasMessageContaining(permString);
   }
 
+  protected Properties createClientProperties(String userName, String password) {
+    Properties props = new Properties();
+    props.setProperty(UserPasswordAuthInit.USER_NAME, userName);
+    props.setProperty(UserPasswordAuthInit.PASSWORD, password);
+    props.setProperty(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName() + ".create");
+    props.setProperty(MCAST_PORT, "0");
+    props.setProperty(LOCATORS, "");
+    props.setProperty(SECURITY_LOG_LEVEL, "finest");
+    return props;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
index 10a8f73..ccdc966 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientAuthDUnitTest.java
@@ -16,6 +16,14 @@
  */
 package com.gemstone.gemfire.security;
 
+import static com.googlecode.catchexception.CatchException.*;
+import static org.assertj.core.api.Assertions.*;
+
+import com.gemstone.gemfire.cache.client.ClientCache;
+import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
+import com.gemstone.gemfire.test.dunit.IgnoredException;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 import org.junit.Test;
@@ -25,14 +33,34 @@ import org.junit.experimental.categories.Category;
 public class IntegratedClientAuthDUnitTest extends AbstractIntegratedClientAuthDistributedTest {
 
   @Test
-  public void testAuthentication(){
-    int port = serverPort;
+  public void authWithCorrectPasswordShouldPass() {
     client1.invoke("logging in super-user with correct password", () -> {
-      SecurityTestUtils.createCacheClient("super-user", "1234567", port, SecurityTestUtils.NO_EXCEPTION);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("super-user", "1234567"))
+          .setPoolSubscriptionEnabled(true)
+          .addPoolServer("localhost", serverPort)
+          .create();
+
+      ClientRegionFactory<String, String> crf = cache.createClientRegionFactory(ClientRegionShortcut.PROXY);
+
+      crf.create(REGION_NAME);
     });
+  }
+
+  @Test
+  public void authWithIncorrectPasswordShouldFail() {
+    IgnoredException.addIgnoredException(AuthenticationFailedException.class.getName());
 
     client2.invoke("logging in super-user with wrong password", () -> {
-      SecurityTestUtils.createCacheClient("super-user", "wrong", port, SecurityTestUtils.AUTHFAIL_EXCEPTION);
+      AuthenticationFailedException expected = new AuthenticationFailedException("Authentication error. Please check your username/password.");
+
+      catchException(new ClientCacheFactory(createClientProperties("super-user", "wrong"))
+          .setPoolSubscriptionEnabled(true)
+          .addPoolServer("localhost", serverPort))
+          .create();
+
+      //throw caughtException(); // TODO: gemfire-mm review as team
+
+      assertThat((Throwable)caughtException()).hasCause(expected);
     });
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
index 87535db..a02b49c 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientContainsKeyAuthDistributedTest.java
@@ -18,20 +18,25 @@ package com.gemstone.gemfire.security;
 
 import static org.junit.Assert.*;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
 import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 @Category(DistributedTest.class)
 public class IntegratedClientContainsKeyAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
 
   @Test
   public void testContainsKey() throws InterruptedException {
-
+//    AsyncInvocation ai1 = client1.invokeAsync("containsKey with permission", () -> {
+//      ClientCache cache = new ClientCacheFactory(createClientProperties("dataReader", "1234567"))
+//          .setPoolSubscriptionEnabled(true)
+//          .addPoolServer("localhost", serverPort)
+//          .create();
+//
     AsyncInvocation ai1 = client1.invokeAsync(() -> {
       Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
@@ -43,7 +48,21 @@ public class IntegratedClientContainsKeyAuthDistributedTest extends AbstractInte
       Cache cache = SecurityTestUtils.createCacheClient("authRegionReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
       final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
       region.containsKeyOnServer("key3");
+      assertTrue(region.containsKeyOnServer("key1"));
+
     });
+
+//    AsyncInvocation ai2 = client2.invokeAsync("containsKey without permission", () -> {
+//      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionReader", "1234567"))
+//        .setPoolSubscriptionEnabled(true)
+//        .addPoolServer("localhost", serverPort)
+//        .create();
+//
+//      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+//      assertNotAuthorized(() -> region.containsKeyOnServer("key3"), "DATA:READ");
+
+//    });
+
     ai1.join();
     ai2.join();
     ai1.checkException();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyInvalidateAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyInvalidateAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyInvalidateAuthDistributedTest.java
index e78ed62..1b3ae21 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyInvalidateAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyInvalidateAuthDistributedTest.java
@@ -18,8 +18,10 @@ package com.gemstone.gemfire.security;
 
 import static org.junit.Assert.*;
 
-import com.gemstone.gemfire.cache.Cache;
 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.client.ClientRegionShortcut;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
@@ -34,15 +36,17 @@ public class IntegratedClientDestroyInvalidateAuthDistributedTest extends Abstra
 
     // Delete one key and invalidate another key with an authorized user.
     AsyncInvocation ai1 = client1.invokeAsync(() -> {
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("dataUser", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
 
-      assertEquals(region.get("key1"), "value1");
-      assertTrue(region.containsKey("key1")); // will only be true after we first get it, then it's cached locally
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+      assertTrue(region.containsKeyOnServer("key1"));
 
       // Destroy key1
       region.destroy("key1");
-      assertFalse(region.containsKey("key1"));
+      assertFalse(region.containsKeyOnServer("key1"));
 
       // Invalidate key2
       assertNotNull("Value of key2 should not be null", region.get("key2"));
@@ -53,22 +57,25 @@ public class IntegratedClientDestroyInvalidateAuthDistributedTest extends Abstra
 
     // Delete one key and invalidate another key with an unauthorized user.
     AsyncInvocation ai2 = client2.invokeAsync(() -> {
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionReader", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
 
-      assertEquals(region.get("key3"), "value3");
-      assertTrue(region.containsKey("key3")); // will only be true after we first get it, then it's cached locally
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+
+      assertTrue(region.containsKeyOnServer("key3"));
 
       // Destroy key1
       assertNotAuthorized(() -> region.destroy("key3"), "DATA:WRITE:AuthRegion");
-      assertTrue(region.containsKey("key3"));
+      assertTrue(region.containsKeyOnServer("key3"));
 
       // Invalidate key2
       assertNotNull("Value of key4 should not be null", region.get("key4"));
       assertNotAuthorized(() -> region.invalidate("key4"), "DATA:WRITE:AuthRegion");
       assertNotNull("Value of key4 should not be null", region.get("key4"));
-      cache.close();
     });
+
     ai1.join();
     ai2.join();
     ai1.checkException();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyRegionAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyRegionAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyRegionAuthDistributedTest.java
index a9cc354..42d55f7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyRegionAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientDestroyRegionAuthDistributedTest.java
@@ -16,8 +16,12 @@
  */
 package com.gemstone.gemfire.security;
 
-import com.gemstone.gemfire.cache.Cache;
+import static org.assertj.core.api.Assertions.*;
+
 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.client.ClientRegionShortcut;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
 import org.junit.Test;
@@ -29,21 +33,34 @@ public class IntegratedClientDestroyRegionAuthDistributedTest extends AbstractIn
   @Test
   public void testDestroyRegion() throws InterruptedException {
     client1.invoke(()-> {
-      Cache cache = SecurityTestUtils.createCacheClient("dataWriter", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("dataWriter", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       assertNotAuthorized(()->region.destroyRegion(), "DATA:MANAGE");
     });
 
     client2.invoke(()-> {
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionManager", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionManager", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       assertNotAuthorized(()->region.destroyRegion(), "DATA:MANAGE");
     });
 
     client3.invoke(()-> {
-      Cache cache = SecurityTestUtils.createCacheClient("super-user", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("super-user", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       region.destroyRegion();
+      assertThat(region.isDestroyed()).isTrue();
     });
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetEntryAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetEntryAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetEntryAuthDistributedTest.java
new file mode 100644
index 0000000..b4eff71
--- /dev/null
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetEntryAuthDistributedTest.java
@@ -0,0 +1,80 @@
+/*
+ * 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.security;
+
+
+import com.gemstone.gemfire.cache.CacheTransactionManager;
+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.client.ClientRegionShortcut;
+import com.gemstone.gemfire.test.dunit.AsyncInvocation;
+import com.gemstone.gemfire.test.junit.categories.DistributedTest;
+
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(DistributedTest.class)
+public class IntegratedClientGetEntryAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest{
+
+  @Test
+  public void testGetEntry() throws InterruptedException {
+    // client1 connects to server as a user not authorized to do any operations
+
+    AsyncInvocation ai1 =  client1.invokeAsync(()->{
+      ClientCache cache = new ClientCacheFactory(createClientProperties("stranger", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      CacheTransactionManager transactionManager = cache.getCacheTransactionManager();
+      transactionManager.begin();
+      try {
+        Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+        assertNotAuthorized(()->region.getEntry("key3"), "DATA:READ:AuthRegion:key3");
+      }
+      finally {
+        transactionManager.commit();
+      }
+
+    });
+
+    AsyncInvocation ai2 =  client2.invokeAsync(()->{
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionReader", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      CacheTransactionManager transactionManager = cache.getCacheTransactionManager();
+      transactionManager.begin();
+      try {
+        Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
+        region.getEntry("key3");
+      }
+      finally {
+        transactionManager.commit();
+      }
+
+    });
+
+    ai1.join();
+    ai2.join();
+    ai1.checkException();
+    ai2.checkException();
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
index 4bd7191..37f9e80 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientGetPutAuthDistributedTest.java
@@ -24,14 +24,16 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.client.ClientCache;
+import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 @Category(DistributedTest.class)
 public class IntegratedClientGetPutAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
 
@@ -47,8 +49,12 @@ public class IntegratedClientGetPutAuthDistributedTest extends AbstractIntegrate
 
     // client1 connects to server as a user not authorized to do any operations
     AsyncInvocation ai1 =  client1.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("stranger", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
 
       assertNotAuthorized(()->region.put("key3", "value3"), "DATA:WRITE:AuthRegion:key3");
       assertNotAuthorized(()->region.get("key3"), "DATA:READ:AuthRegion:key3");
@@ -66,8 +72,12 @@ public class IntegratedClientGetPutAuthDistributedTest extends AbstractIntegrate
 
     // client2 connects to user as a user authorized to use AuthRegion region
     AsyncInvocation ai2 =  client2.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionUser", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
 
       region.put("key3", "value3");
       assertEquals("value3", region.get("key3"));
@@ -86,8 +96,12 @@ public class IntegratedClientGetPutAuthDistributedTest extends AbstractIntegrate
 
     // client3 connects to user as a user authorized to use key1 in AuthRegion region
     AsyncInvocation ai3 =  client3.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("key1User", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
 
       assertNotAuthorized(()->region.put("key2", "value1"), "DATA:WRITE:AuthRegion:key2");
       assertNotAuthorized(()->region.get("key2"), "DATA:READ:AuthRegion:key2");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
index 8651a2f..a6a2129 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientQueryAuthDistributedTest.java
@@ -16,8 +16,10 @@
  */
 package com.gemstone.gemfire.security;
 
-import com.gemstone.gemfire.cache.Cache;
 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.client.ClientRegionShortcut;
 import com.gemstone.gemfire.cache.client.Pool;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
@@ -31,8 +33,12 @@ public class IntegratedClientQueryAuthDistributedTest extends AbstractIntegrated
   @Test
   public void testQuery(){
     client1.invoke(()-> {
-      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("stranger", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
 
       String query = "select * from /AuthRegion";
       assertNotAuthorized(()->region.query(query), "DATA:READ:AuthRegion");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegionClearAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegionClearAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegionClearAuthDistributedTest.java
index b344ce1..44985c7 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegionClearAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegionClearAuthDistributedTest.java
@@ -16,8 +16,10 @@
  */
 package com.gemstone.gemfire.security;
 
-import com.gemstone.gemfire.cache.Cache;
 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.client.ClientRegionShortcut;
 import com.gemstone.gemfire.test.dunit.SerializableRunnable;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
@@ -33,8 +35,12 @@ public class IntegratedClientRegionClearAuthDistributedTest extends AbstractInte
     SerializableRunnable clearUnauthorized = new SerializableRunnable() {
       @Override
       public void run() {
-        Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-        final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+        ClientCache cache = new ClientCacheFactory(createClientProperties("stranger", "1234567"))
+          .setPoolSubscriptionEnabled(true)
+          .addPoolServer("localhost", serverPort)
+          .create();
+
+        Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
         assertNotAuthorized(() -> region.clear(), "DATA:WRITE:AuthRegion");
       }
     };
@@ -44,8 +50,12 @@ public class IntegratedClientRegionClearAuthDistributedTest extends AbstractInte
     SerializableRunnable clearAuthorized = new SerializableRunnable() {
       @Override
       public void run() {
-        Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-        final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+        ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionUser", "1234567"))
+          .setPoolSubscriptionEnabled(true)
+          .addPoolServer("localhost", serverPort)
+          .create();
+
+        Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
         region.clear();
       }
     };

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegisterInterestAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegisterInterestAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegisterInterestAuthDistributedTest.java
index d533a20..373b7dd 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegisterInterestAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRegisterInterestAuthDistributedTest.java
@@ -19,8 +19,10 @@ package com.gemstone.gemfire.security;
 import java.util.ArrayList;
 import java.util.List;
 
-import com.gemstone.gemfire.cache.Cache;
 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.client.ClientRegionShortcut;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
@@ -34,22 +36,34 @@ public class IntegratedClientRegisterInterestAuthDistributedTest extends Abstrac
   public void testRegisterInterest() throws InterruptedException {
     // client1 connects to server as a user not authorized to do any operations
     AsyncInvocation ai1 =  client1.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("stranger", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       assertNotAuthorized(()->region.registerInterest("key3"), "DATA:READ:AuthRegion:key3");
     });
 
     // client2 connects to user as a user authorized to use AuthRegion region
     AsyncInvocation ai2 =  client2.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionUser", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       region.registerInterest("key3");  //  DATA:READ:AuthRegion:key3;
     });
 
     // client3 connects to user as a user authorized to use key1 in AuthRegion region
     AsyncInvocation ai3 =  client3.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("key1User", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       assertNotAuthorized(()->region.registerInterest("key2"), "DATA:READ:AuthRegion:key2");
     });
 
@@ -66,22 +80,34 @@ public class IntegratedClientRegisterInterestAuthDistributedTest extends Abstrac
   public void testRegisterInterestRegex() throws InterruptedException {
     //client1 connects to server as a user not authorized to do any operations
     AsyncInvocation ai1 =  client1.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("stranger", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       assertNotAuthorized(()->region.registerInterestRegex("key.*"), "DATA:READ:AuthRegion");
     });
 
     // client2 connects to user as a user authorized to use AuthRegion region
     AsyncInvocation ai2 =  client2.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionUser", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       region.registerInterestRegex("key[0-9]+");  //  DATA:READ:AuthRegion:key3;
     });
 
     // client3 connects to user as a user authorized to use key1 in AuthRegion region
     AsyncInvocation ai3 =  client3.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("key1User", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       assertNotAuthorized(()->region.registerInterestRegex("key[0-9]+"), "DATA:READ:AuthRegion");
       assertNotAuthorized(()->region.registerInterestRegex("key1"), "DATA:READ:AuthRegion");
     });
@@ -103,22 +129,34 @@ public class IntegratedClientRegisterInterestAuthDistributedTest extends Abstrac
 
     //client1 connects to server as a user not authorized to do any operations
     AsyncInvocation ai1 =  client1.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("stranger", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("stranger", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       assertNotAuthorized(()->region.registerInterest(keys), "DATA:READ:AuthRegion");
     });
 
     // client2 connects to user as a user authorized to use AuthRegion region
     AsyncInvocation ai2 =  client2.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionUser", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       region.registerInterest(keys);  //  DATA:READ:AuthRegion;
     });
 
     // client3 connects to user as a user authorized to use key1 in AuthRegion region
     AsyncInvocation ai3 =  client3.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("key1User", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("key1User", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       assertNotAuthorized(()->region.registerInterest(keys), "DATA:READ:AuthRegion");
     });
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
index edfbdd6..46ed1b8 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientRemoveAllAuthDistributedTest.java
@@ -16,18 +16,20 @@
  */
 package com.gemstone.gemfire.security;
 
-import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.*;
 
 import java.util.Arrays;
 
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import com.gemstone.gemfire.cache.Cache;
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.client.ClientCache;
+import com.gemstone.gemfire.cache.client.ClientCacheFactory;
+import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
 @Category(DistributedTest.class)
 public class IntegratedClientRemoveAllAuthDistributedTest extends AbstractIntegratedClientAuthDistributedTest {
 
@@ -35,14 +37,22 @@ public class IntegratedClientRemoveAllAuthDistributedTest extends AbstractIntegr
   public void testRemoveAll() throws InterruptedException {
 
     AsyncInvocation ai1 = client1.invokeAsync(() -> {
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionReader", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionReader", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       assertNotAuthorized(() -> region.removeAll(Arrays.asList("key1", "key2", "key3", "key4")), "DATA:WRITE:AuthRegion");
     });
 
     AsyncInvocation ai2 = client2.invokeAsync(() -> {
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionWriter", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionWriter", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       region.removeAll(Arrays.asList("key1", "key2", "key3", "key4"));
       assertFalse(region.containsKey("key1"));
       assertNotAuthorized(()->region.containsKeyOnServer("key1"), "DATA:READ:AuthRegion:key1");

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/98244ece/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
----------------------------------------------------------------------
diff --git a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
index aa46882..082474a 100644
--- a/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
+++ b/geode-core/src/test/java/com/gemstone/gemfire/security/IntegratedClientUnregisterInterestAuthDistributedTest.java
@@ -16,8 +16,10 @@
  */
 package com.gemstone.gemfire.security;
 
-import com.gemstone.gemfire.cache.Cache;
 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.client.ClientRegionShortcut;
 import com.gemstone.gemfire.test.dunit.AsyncInvocation;
 import com.gemstone.gemfire.test.junit.categories.DistributedTest;
 
@@ -30,8 +32,12 @@ public class IntegratedClientUnregisterInterestAuthDistributedTest extends Abstr
   public void testUnregisterInterest() throws InterruptedException {
     // client2 connects to user as a user authorized to use AuthRegion region
     AsyncInvocation ai1 =  client2.invokeAsync(()->{
-      Cache cache = SecurityTestUtils.createCacheClient("authRegionUser", "1234567", serverPort, SecurityTestUtils.NO_EXCEPTION);
-      final Region region = cache.getRegion(SecurityTestUtils.REGION_NAME);
+      ClientCache cache = new ClientCacheFactory(createClientProperties("authRegionUser", "1234567"))
+        .setPoolSubscriptionEnabled(true)
+        .addPoolServer("localhost", serverPort)
+        .create();
+
+      Region region = cache.createClientRegionFactory(ClientRegionShortcut.PROXY).create(REGION_NAME);
       region.registerInterest("key3");
       region.unregisterInterest("key3");  //  DATA:READ:AuthRegion:key3;
     });