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 2017/06/24 00:57:43 UTC

geode git commit: GEODE-2924: fix lucene commands security test

Repository: geode
Updated Branches:
  refs/heads/develop 451d12e83 -> 805345014


GEODE-2924: fix lucene commands security test


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

Branch: refs/heads/develop
Commit: 8053450143007ffaa7d92ae23d9e289129b97f56
Parents: 451d12e
Author: Jinmei Liao <ji...@pivotal.io>
Authored: Fri Jun 23 17:56:27 2017 -0700
Committer: Jinmei Liao <ji...@pivotal.io>
Committed: Fri Jun 23 17:57:13 2017 -0700

----------------------------------------------------------------------
 .../geode/internal/util/PasswordUtil.java       |  27 +-
 .../lucene/LuceneClientSecurityDUnitTest.java   |  46 ++-
 .../lucene/LuceneCommandsSecurityDUnitTest.java | 291 -------------------
 .../lucene/LuceneCommandsSecurityTest.java      | 263 +++++++++++++++++
 4 files changed, 296 insertions(+), 331 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/80534501/geode-core/src/main/java/org/apache/geode/internal/util/PasswordUtil.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/util/PasswordUtil.java b/geode-core/src/main/java/org/apache/geode/internal/util/PasswordUtil.java
index 5cc3bcd..2495959 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/util/PasswordUtil.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/util/PasswordUtil.java
@@ -49,23 +49,20 @@ public class PasswordUtil {
    */
   @Deprecated
   public static String decrypt(String password) {
-    String toDecrypt;
     if (password.startsWith("encrypted(") && password.endsWith(")")) {
-      toDecrypt = password.substring(10, password.length() - 1);
-    } else {
-      toDecrypt = password;
+      byte[] decrypted = null;
+      try {
+        String toDecrypt = password.substring(10, password.length() - 1);
+        SecretKeySpec key = new SecretKeySpec(init, "Blowfish");
+        Cipher cipher = Cipher.getInstance("Blowfish");
+        cipher.init(Cipher.DECRYPT_MODE, key);
+        decrypted = cipher.doFinal(hexStringToByteArray(toDecrypt));
+        return new String(decrypted);
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
     }
-    byte[] decrypted;
-    try {
-      SecretKeySpec key = new SecretKeySpec(init, "Blowfish");
-      Cipher cipher = Cipher.getInstance("Blowfish");
-      cipher.init(Cipher.DECRYPT_MODE, key);
-      decrypted = cipher.doFinal(hexStringToByteArray(toDecrypt));
-      return new String(decrypted);
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-    return toDecrypt;
+    return password;
   }
 
   private static byte[] hexStringToByteArray(String s) {

http://git-wip-us.apache.org/repos/asf/geode/blob/80534501/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneClientSecurityDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneClientSecurityDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneClientSecurityDUnitTest.java
index 3e6b541..7ab3365 100644
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneClientSecurityDUnitTest.java
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneClientSecurityDUnitTest.java
@@ -14,8 +14,22 @@
  */
 package org.apache.geode.cache.lucene;
 
+import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.INDEX_NAME;
+import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTH_INIT;
+import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Properties;
+
 import junitparams.JUnitParamsRunner;
 import junitparams.Parameters;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
 import org.apache.geode.cache.Cache;
 import org.apache.geode.cache.RegionShortcut;
 import org.apache.geode.cache.client.ClientCache;
@@ -26,22 +40,8 @@ import org.apache.geode.cache.server.CacheServer;
 import org.apache.geode.security.NotAuthorizedException;
 import org.apache.geode.security.SimpleTestSecurityManager;
 import org.apache.geode.security.templates.UserPasswordAuthInit;
-import org.apache.geode.test.dunit.SerializableRunnableIF;
 import org.apache.geode.test.junit.categories.DistributedTest;
 import org.apache.geode.test.junit.categories.SecurityTest;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-
-import java.io.IOException;
-import java.util.Properties;
-
-import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.INDEX_NAME;
-import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_CLIENT_AUTH_INIT;
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
 
 @Category({DistributedTest.class, SecurityTest.class})
 @RunWith(JUnitParamsRunner.class)
@@ -50,7 +50,7 @@ public class LuceneClientSecurityDUnitTest extends LuceneQueriesAccessorBase {
   @Test
   @Parameters(method = "getSearchIndexUserNameAndExpectedResponses")
   public void verifySearchIndexPermissions(
-      LuceneCommandsSecurityDUnitTest.UserNameAndExpectedResponse user) {
+      LuceneCommandsSecurityTest.UserNameAndExpectedResponse user) {
     // Start server
     int serverPort = dataStore1.invoke(() -> startCacheServer());
 
@@ -58,8 +58,7 @@ public class LuceneClientSecurityDUnitTest extends LuceneQueriesAccessorBase {
     accessor.invoke(() -> startClient(user.getUserName(), serverPort));
 
     // Attempt query
-    accessor.invoke(
-        () -> executeTextSearch(user.getExpectAuthorizationError(), user.getExpectedResponse()));
+    accessor.invoke(() -> executeTextSearch(user.getExpectAuthorizationError()));
   }
 
   private int startCacheServer() throws IOException {
@@ -86,8 +85,7 @@ public class LuceneClientSecurityDUnitTest extends LuceneQueriesAccessorBase {
     clientCache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create(REGION_NAME);
   }
 
-  private void executeTextSearch(boolean expectAuthorizationError, String expectedResponse)
-      throws LuceneQueryException {
+  private void executeTextSearch(boolean expectAuthorizationError) throws LuceneQueryException {
     LuceneService service = LuceneServiceProvider.get(getCache());
     LuceneQuery<Integer, TestObject> query =
         service.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "test", "text");
@@ -97,14 +95,12 @@ public class LuceneClientSecurityDUnitTest extends LuceneQueriesAccessorBase {
     } catch (ServerOperationException e) {
       assertTrue(e.getCause() != null && e.getCause() instanceof NotAuthorizedException);
       assertTrue(expectAuthorizationError);
-      assertTrue(e.getLocalizedMessage().contains(expectedResponse));
     }
   }
 
-  protected LuceneCommandsSecurityDUnitTest.UserNameAndExpectedResponse[] getSearchIndexUserNameAndExpectedResponses() {
-    return new LuceneCommandsSecurityDUnitTest.UserNameAndExpectedResponse[] {
-        new LuceneCommandsSecurityDUnitTest.UserNameAndExpectedResponse("nopermissions", true,
-            "nopermissions not authorized for DATA:WRITE"),
-        new LuceneCommandsSecurityDUnitTest.UserNameAndExpectedResponse("datawrite", false, null)};
+  protected LuceneCommandsSecurityTest.UserNameAndExpectedResponse[] getSearchIndexUserNameAndExpectedResponses() {
+    return new LuceneCommandsSecurityTest.UserNameAndExpectedResponse[] {
+        new LuceneCommandsSecurityTest.UserNameAndExpectedResponse("nopermissions", true),
+        new LuceneCommandsSecurityTest.UserNameAndExpectedResponse("datawrite", false)};
   }
 }

http://git-wip-us.apache.org/repos/asf/geode/blob/80534501/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityDUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityDUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityDUnitTest.java
deleted file mode 100644
index 7cc6709..0000000
--- a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityDUnitTest.java
+++ /dev/null
@@ -1,291 +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 org.apache.geode.cache.lucene;
-
-import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.INDEX_NAME;
-import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import java.io.Serializable;
-import java.util.Properties;
-
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.cache.lucene.internal.cli.LuceneCliStrings;
-import org.apache.geode.management.cli.Result;
-import org.apache.geode.management.internal.cli.i18n.CliStrings;
-import org.apache.geode.management.internal.cli.result.CommandResult;
-import org.apache.geode.management.internal.cli.result.ErrorResultData;
-import org.apache.geode.management.internal.cli.result.ResultBuilder;
-import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
-import org.apache.geode.security.SimpleTestSecurityManager;
-import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
-import org.apache.geode.test.dunit.rules.LocatorServerStartupRule;
-import org.apache.geode.test.dunit.rules.MemberVM;
-import org.apache.geode.test.junit.categories.DistributedTest;
-import org.apache.geode.test.junit.categories.SecurityTest;
-
-@Category({DistributedTest.class, SecurityTest.class})
-@RunWith(JUnitParamsRunner.class)
-public class LuceneCommandsSecurityDUnitTest {
-
-  @Rule
-  public LocatorServerStartupRule locatorServer = new LocatorServerStartupRule();
-
-  @Rule
-  public GfshShellConnectionRule gfshShell = new GfshShellConnectionRule();
-
-  private MemberVM locator;
-
-  @Before
-  public void before() throws Exception {
-    // start the locator
-    Properties props = new Properties();
-    props.setProperty(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName());
-    this.locator = this.locatorServer.startLocatorVM(0, props);
-
-    // start the server
-    props = new Properties();
-    props.setProperty("security-username", "clusterManage");
-    props.setProperty("security-password", "clusterManage");
-    this.locatorServer.startServerVM(1, props, this.locator.getPort());
-  }
-
-  protected UserNameAndExpectedResponse[] getCreateIndexUserNameAndExpectedResponses() {
-    return new UserNameAndExpectedResponse[] {
-        new UserNameAndExpectedResponse("noPermissions", true,
-            "Unauthorized. Reason : noPermissions not authorized for CLUSTER:MANAGE:QUERY"),
-        new UserNameAndExpectedResponse("clusterManageQuery", false,
-            "Successfully created lucene index")};
-  }
-
-  @Test
-  @Parameters(method = "getCreateIndexUserNameAndExpectedResponses")
-  public void verifyCreateIndexPermissions(UserNameAndExpectedResponse user) throws Exception {
-    // Connect gfsh
-    this.gfshShell.secureConnectAndVerify(this.locator.getPort(),
-        GfshShellConnectionRule.PortType.locator, user.getUserName(), user.getUserName());
-
-    // Attempt to create lucene index
-    CommandResult result = this.gfshShell.executeCommand(getCreateIndexCommand());
-
-    // Verify result
-    verifyResult(user, result);
-  }
-
-  protected UserNameAndExpectedResponse[] getSearchIndexUserNameAndExpectedResponses() {
-    return new UserNameAndExpectedResponse[] {
-        new UserNameAndExpectedResponse("noPermissions", true,
-            "Unauthorized. Reason : noPermissions not authorized for DATA:READ:region"),
-        new UserNameAndExpectedResponse("dataRead", false, "No results")};
-  }
-
-  @Test
-  @Parameters(method = "getSearchIndexUserNameAndExpectedResponses")
-  public void verifySearchIndexPermissions(UserNameAndExpectedResponse user) throws Exception {
-    // Create index and region
-    createIndexAndRegion();
-
-    // Connect gfsh
-    this.gfshShell.secureConnectAndVerify(this.locator.getPort(),
-        GfshShellConnectionRule.PortType.locator, user.getUserName(), user.getUserName());
-
-    // Attempt to search lucene index
-    CommandResult result = this.gfshShell.executeCommand(getSearchIndexCommand());
-
-    // Verify result
-    verifyResult(user, result);
-  }
-
-  protected UserNameAndExpectedResponse[] getListIndexesUserNameAndExpectedResponses() {
-    return new UserNameAndExpectedResponse[] {
-        new UserNameAndExpectedResponse("noPermissions", true,
-            "Unauthorized. Reason : noPermissions not authorized for CLUSTER:READ:QUERY"),
-        new UserNameAndExpectedResponse("clusterReadQuery", false, "Index Name")};
-  }
-
-  @Test
-  @Parameters(method = "getListIndexesUserNameAndExpectedResponses")
-  public void verifyListIndexesPermissions(UserNameAndExpectedResponse user) throws Exception {
-    // Create index and region
-    createIndexAndRegion();
-
-    // Connect gfsh
-    this.gfshShell.secureConnectAndVerify(this.locator.getPort(),
-        GfshShellConnectionRule.PortType.locator, user.getUserName(), user.getUserName());
-
-    // Attempt to search lucene index
-    CommandResult result = this.gfshShell.executeCommand(getListIndexesCommand());
-
-    // Verify result
-    verifyResult(user, result);
-  }
-
-  protected UserNameAndExpectedResponse[] getDescribeIndexUserNameAndExpectedResponses() {
-    return new UserNameAndExpectedResponse[] {
-        new UserNameAndExpectedResponse("noPermissions", true,
-            "Unauthorized. Reason : noPermissions not authorized for CLUSTER:READ:QUERY"),
-        new UserNameAndExpectedResponse("clusterReadQuery", false, "Index Name")};
-  }
-
-  @Test
-  @Parameters(method = "getDescribeIndexUserNameAndExpectedResponses")
-  public void verifyDescribeIndexPermissions(UserNameAndExpectedResponse user) throws Exception {
-    // Create index and region
-    createIndexAndRegion();
-
-    // Connect gfsh
-    this.gfshShell.secureConnectAndVerify(this.locator.getPort(),
-        GfshShellConnectionRule.PortType.locator, user.getUserName(), user.getUserName());
-
-    // Attempt to search lucene index
-    CommandResult result = this.gfshShell.executeCommand(getDescribeIndexCommand());
-
-    // Verify result
-    verifyResult(user, result);
-  }
-
-  protected UserNameAndExpectedResponse[] getDestroyIndexUserNameAndExpectedResponses() {
-    return new UserNameAndExpectedResponse[] {
-        new UserNameAndExpectedResponse("noPermissions", true,
-            "Unauthorized. Reason : noPermissions not authorized for CLUSTER:MANAGE:QUERY"),
-        new UserNameAndExpectedResponse("clusterManageQuery", false,
-            "Successfully destroyed lucene index")};
-  }
-
-  @Test
-  @Parameters(method = "getDestroyIndexUserNameAndExpectedResponses")
-  public void verifyDestroyIndexPermissions(UserNameAndExpectedResponse user) throws Exception {
-    // Create index and region
-    createIndexAndRegion();
-
-    // Connect gfsh
-    this.gfshShell.secureConnectAndVerify(this.locator.getPort(),
-        GfshShellConnectionRule.PortType.locator, user.getUserName(), user.getUserName());
-
-    // Attempt to search lucene index
-    CommandResult result = this.gfshShell.executeCommand(getDestroyIndexCommand());
-
-    // Verify result
-    verifyResult(user, result);
-  }
-
-  private void createIndexAndRegion() throws Exception {
-    // Connect gfsh to locator with permissions necessary to create an index and region
-    this.gfshShell.secureConnectAndVerify(this.locator.getPort(),
-        GfshShellConnectionRule.PortType.locator, "cluster,data", "cluster,data");
-
-    // Create lucene index
-    this.gfshShell.executeAndVerifyCommand(getCreateIndexCommand());
-
-    // Create region
-    this.gfshShell.executeAndVerifyCommand(getCreateRegionCommand());
-
-    // Disconnect gfsh
-    this.gfshShell.disconnect();
-  }
-
-  private void verifyResult(UserNameAndExpectedResponse user, CommandResult result) {
-    if (user.getExpectAuthorizationError()) {
-      assertTrue(result.getResultData() instanceof ErrorResultData);
-      assertEquals(ResultBuilder.ERRORCODE_UNAUTHORIZED,
-          ((ErrorResultData) result.getResultData()).getErrorCode());
-    } else {
-      assertEquals(Result.Status.OK, result.getStatus());
-    }
-    assertTrue(this.gfshShell.getGfshOutput().contains(user.getExpectedResponse()));
-  }
-
-  private String getCreateIndexCommand() {
-    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_CREATE_INDEX);
-    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
-    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
-    csb.addOption(LuceneCliStrings.LUCENE_CREATE_INDEX__FIELD, "field1");
-    return csb.toString();
-  }
-
-  private String getCreateRegionCommand() {
-    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_REGION);
-    csb.addOption(CliStrings.CREATE_REGION__REGION, REGION_NAME);
-    csb.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT,
-        RegionShortcut.PARTITION_REDUNDANT.name());
-    return csb.toString();
-  }
-
-  private String getSearchIndexCommand() {
-    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_SEARCH_INDEX);
-    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
-    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
-    csb.addOption(LuceneCliStrings.LUCENE_SEARCH_INDEX__QUERY_STRING, "field1:value1");
-    csb.addOption(LuceneCliStrings.LUCENE_SEARCH_INDEX__DEFAULT_FIELD, "field1");
-    return csb.toString();
-  }
-
-  private String getListIndexesCommand() {
-    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
-    return csb.toString();
-  }
-
-  private String getDescribeIndexCommand() {
-    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESCRIBE_INDEX);
-    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
-    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
-    return csb.toString();
-  }
-
-  private String getDestroyIndexCommand() {
-    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESTROY_INDEX);
-    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
-    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
-    return csb.toString();
-  }
-
-  public static class UserNameAndExpectedResponse implements Serializable {
-
-    private final String userName;
-
-    private final boolean expectAuthorizationError;
-
-    private final String expectedResponse;
-
-    public UserNameAndExpectedResponse(String userName, boolean expectAuthorizationError,
-        String expectedResponse) {
-      this.userName = userName;
-      this.expectAuthorizationError = expectAuthorizationError;
-      this.expectedResponse = expectedResponse;
-    }
-
-    public String getUserName() {
-      return this.userName;
-    }
-
-    public boolean getExpectAuthorizationError() {
-      return this.expectAuthorizationError;
-    }
-
-    public String getExpectedResponse() {
-      return this.expectedResponse;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/geode/blob/80534501/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityTest.java
new file mode 100644
index 0000000..c195611
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityTest.java
@@ -0,0 +1,263 @@
+/*
+ * 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 org.apache.geode.cache.lucene;
+
+import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.INDEX_NAME;
+import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.Serializable;
+
+import junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.junit.After;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.lucene.internal.cli.LuceneCliStrings;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.i18n.CliStrings;
+import org.apache.geode.management.internal.cli.result.CommandResult;
+import org.apache.geode.management.internal.cli.result.ErrorResultData;
+import org.apache.geode.management.internal.cli.result.ResultBuilder;
+import org.apache.geode.management.internal.cli.util.CommandStringBuilder;
+import org.apache.geode.security.SimpleTestSecurityManager;
+import org.apache.geode.test.dunit.rules.GfshShellConnectionRule;
+import org.apache.geode.test.dunit.rules.ServerStarterRule;
+import org.apache.geode.test.junit.categories.IntegrationTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+
+@Category({IntegrationTest.class, SecurityTest.class})
+@RunWith(JUnitParamsRunner.class)
+public class LuceneCommandsSecurityTest {
+
+  @ClassRule
+  public static ServerStarterRule server = new ServerStarterRule()
+      .withSecurityManager(SimpleTestSecurityManager.class).withJMXManager().withAutoStart();
+
+  @ClassRule
+  public static GfshShellConnectionRule gfshShell = new GfshShellConnectionRule();
+
+  @After
+  public void after() throws Exception {
+    gfshShell.disconnect();
+  }
+
+
+  protected UserNameAndExpectedResponse[] getCreateIndexUserNameAndExpectedResponses() {
+    return new UserNameAndExpectedResponse[] {
+        new UserNameAndExpectedResponse("noPermissions", true),
+        new UserNameAndExpectedResponse("clusterManageQuery,clusterWriteDisk", false)};
+  }
+
+  @Test
+  @Parameters(method = "getCreateIndexUserNameAndExpectedResponses")
+  public void verifyCreateIndexPermissions(UserNameAndExpectedResponse user) throws Exception {
+    // Connect gfsh
+    this.gfshShell.secureConnectAndVerify(this.server.getJmxPort(),
+        GfshShellConnectionRule.PortType.jmxManger, user.getUserName(), user.getUserName());
+
+    // Attempt to create lucene index
+    CommandResult result = this.gfshShell.executeCommand(getCreateIndexCommand());
+
+    // Verify result
+    verifyResult(user, result);
+  }
+
+  protected UserNameAndExpectedResponse[] getSearchIndexUserNameAndExpectedResponses() {
+    return new UserNameAndExpectedResponse[] {
+        new UserNameAndExpectedResponse("noPermissions", true),
+        new UserNameAndExpectedResponse("dataRead", false)};
+  }
+
+  @Test
+  @Parameters(method = "getSearchIndexUserNameAndExpectedResponses")
+  public void verifySearchIndexPermissions(UserNameAndExpectedResponse user) throws Exception {
+    // Create index and region
+    // createIndexAndRegion();
+
+    // Connect gfsh
+    this.gfshShell.secureConnectAndVerify(this.server.getJmxPort(),
+        GfshShellConnectionRule.PortType.jmxManger, user.getUserName(), user.getUserName());
+
+    // Attempt to search lucene index
+    CommandResult result = this.gfshShell.executeCommand(getSearchIndexCommand());
+
+    // Verify result
+    verifyResult(user, result);
+  }
+
+  protected UserNameAndExpectedResponse[] getListIndexesUserNameAndExpectedResponses() {
+    return new UserNameAndExpectedResponse[] {
+        new UserNameAndExpectedResponse("noPermissions", true),
+        new UserNameAndExpectedResponse("clusterReadQuery", false)};
+  }
+
+  @Test
+  @Parameters(method = "getListIndexesUserNameAndExpectedResponses")
+  public void verifyListIndexesPermissions(UserNameAndExpectedResponse user) throws Exception {
+    // Create index and region
+    // createIndexAndRegion();
+
+    // Connect gfsh
+    this.gfshShell.secureConnectAndVerify(this.server.getJmxPort(),
+        GfshShellConnectionRule.PortType.jmxManger, user.getUserName(), user.getUserName());
+
+    // Attempt to search lucene index
+    CommandResult result = this.gfshShell.executeCommand(getListIndexesCommand());
+
+    // Verify result
+    verifyResult(user, result);
+  }
+
+  protected UserNameAndExpectedResponse[] getDescribeIndexUserNameAndExpectedResponses() {
+    return new UserNameAndExpectedResponse[] {
+        new UserNameAndExpectedResponse("noPermissions", true),
+        new UserNameAndExpectedResponse("clusterReadQuery", false)};
+  }
+
+  @Test
+  @Parameters(method = "getDescribeIndexUserNameAndExpectedResponses")
+  public void verifyDescribeIndexPermissions(UserNameAndExpectedResponse user) throws Exception {
+    // Create index and region
+    // createIndexAndRegion();
+
+    // Connect gfsh
+    this.gfshShell.secureConnectAndVerify(this.server.getJmxPort(),
+        GfshShellConnectionRule.PortType.jmxManger, user.getUserName(), user.getUserName());
+
+    // Attempt to search lucene index
+    CommandResult result = this.gfshShell.executeCommand(getDescribeIndexCommand());
+
+    // Verify result
+    verifyResult(user, result);
+  }
+
+  protected UserNameAndExpectedResponse[] getDestroyIndexUserNameAndExpectedResponses() {
+    return new UserNameAndExpectedResponse[] {
+        new UserNameAndExpectedResponse("noPermissions", true),
+        new UserNameAndExpectedResponse("clusterManageQuery", false)};
+  }
+
+  @Test
+  @Parameters(method = "getDestroyIndexUserNameAndExpectedResponses")
+  public void verifyDestroyIndexPermissions(UserNameAndExpectedResponse user) throws Exception {
+    // Create index and region
+    // createIndexAndRegion();
+
+    // Connect gfsh
+    this.gfshShell.secureConnectAndVerify(this.server.getJmxPort(),
+        GfshShellConnectionRule.PortType.jmxManger, user.getUserName(), user.getUserName());
+
+    // Attempt to search lucene index
+    CommandResult result = this.gfshShell.executeCommand(getDestroyIndexCommand());
+
+    // Verify result
+    verifyResult(user, result);
+  }
+
+  private void createIndexAndRegion() throws Exception {
+    // Connect gfsh to locator with permissions necessary to create an index and region
+    this.gfshShell.secureConnectAndVerify(this.server.getJmxPort(),
+        GfshShellConnectionRule.PortType.jmxManger, "cluster,data", "cluster,data");
+
+    // Create lucene index
+    this.gfshShell.executeAndVerifyCommand(getCreateIndexCommand());
+
+    // Create region
+    this.gfshShell.executeAndVerifyCommand(getCreateRegionCommand());
+
+    // Disconnect gfsh
+    this.gfshShell.disconnect();
+  }
+
+  private void verifyResult(UserNameAndExpectedResponse user, CommandResult result) {
+    if (user.getExpectAuthorizationError()) {
+      assertTrue(result.getResultData() instanceof ErrorResultData);
+      assertEquals(ResultBuilder.ERRORCODE_UNAUTHORIZED,
+          ((ErrorResultData) result.getResultData()).getErrorCode());
+    } else {
+      assertEquals(Result.Status.OK, result.getStatus());
+    }
+  }
+
+  private String getCreateIndexCommand() {
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_CREATE_INDEX);
+    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE_CREATE_INDEX__FIELD, "field1");
+    return csb.toString();
+  }
+
+  private String getCreateRegionCommand() {
+    CommandStringBuilder csb = new CommandStringBuilder(CliStrings.CREATE_REGION);
+    csb.addOption(CliStrings.CREATE_REGION__REGION, REGION_NAME);
+    csb.addOption(CliStrings.CREATE_REGION__REGIONSHORTCUT,
+        RegionShortcut.PARTITION_REDUNDANT.name());
+    return csb.toString();
+  }
+
+  private String getSearchIndexCommand() {
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_SEARCH_INDEX);
+    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE_SEARCH_INDEX__QUERY_STRING, "field1:value1");
+    csb.addOption(LuceneCliStrings.LUCENE_SEARCH_INDEX__DEFAULT_FIELD, "field1");
+    return csb.toString();
+  }
+
+  private String getListIndexesCommand() {
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
+    return csb.toString();
+  }
+
+  private String getDescribeIndexCommand() {
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESCRIBE_INDEX);
+    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
+    return csb.toString();
+  }
+
+  private String getDestroyIndexCommand() {
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_DESTROY_INDEX);
+    csb.addOption(LuceneCliStrings.LUCENE__INDEX_NAME, INDEX_NAME);
+    csb.addOption(LuceneCliStrings.LUCENE__REGION_PATH, REGION_NAME);
+    return csb.toString();
+  }
+
+  public static class UserNameAndExpectedResponse implements Serializable {
+
+    private final String userName;
+
+    private final boolean expectAuthorizationError;
+
+    public UserNameAndExpectedResponse(String userName, boolean expectAuthorizationError) {
+      this.userName = userName;
+      this.expectAuthorizationError = expectAuthorizationError;
+    }
+
+    public String getUserName() {
+      return this.userName;
+    }
+
+    public boolean getExpectAuthorizationError() {
+      return this.expectAuthorizationError;
+    }
+  }
+}