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:21:12 UTC

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

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": [