You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ds...@apache.org on 2017/04/21 23:42:33 UTC

[37/51] [abbrv] geode git commit: GEODE-2605: Modified gfsh search lucene to require DATA:WRITE privilege to match client

GEODE-2605: Modified gfsh search lucene to require DATA:WRITE privilege to match client


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

Branch: refs/heads/feature/GEODE-2097
Commit: 9b49bc8fc53cb2520215e90930b2c9a27a918921
Parents: 8ea0f90
Author: Barry Oglesby <bo...@pivotal.io>
Authored: Mon Apr 17 12:45:31 2017 -0700
Committer: Barry Oglesby <bo...@pivotal.io>
Committed: Wed Apr 19 14:26:44 2017 -0700

----------------------------------------------------------------------
 .../internal/cli/LuceneIndexCommands.java       |   7 +-
 .../lucene/LuceneClientSecurityDUnitTest.java   | 128 ++++++++
 .../lucene/LuceneCommandsSecurityDUnitTest.java | 303 +++++++++++++++++++
 3 files changed, 433 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/geode/blob/9b49bc8f/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
index 3fa34e7..4bfa868 100755
--- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
+++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/cli/LuceneIndexCommands.java
@@ -250,7 +250,6 @@ public class LuceneIndexCommands extends AbstractCommandsSupport {
 
   @SuppressWarnings("unchecked")
   protected List<LuceneIndexDetails> getIndexDetails(LuceneIndexInfo indexInfo) throws Exception {
-    this.securityService.authorizeRegionManage(indexInfo.getRegionPath());
     final ResultCollector<?, ?> rc =
         executeFunctionOnRegion(describeIndexFunction, indexInfo, true);
     final List<LuceneIndexDetails> funcResults = (List<LuceneIndexDetails>) rc.getResult();
@@ -262,7 +261,7 @@ public class LuceneIndexCommands extends AbstractCommandsSupport {
       help = LuceneCliStrings.LUCENE_SEARCH_INDEX__HELP)
   @CliMetaData(shellOnly = false,
       relatedTopic = {CliStrings.TOPIC_GEODE_REGION, CliStrings.TOPIC_GEODE_DATA})
-  @ResourceOperation(resource = Resource.CLUSTER, operation = Operation.READ)
+  @ResourceOperation(resource = Resource.DATA, operation = Operation.WRITE)
   public Result searchIndex(@CliOption(key = LuceneCliStrings.LUCENE__INDEX_NAME, mandatory = true,
       help = LuceneCliStrings.LUCENE_SEARCH_INDEX__NAME__HELP) final String indexName,
 
@@ -531,8 +530,6 @@ public class LuceneIndexCommands extends AbstractCommandsSupport {
 
   private List<LuceneSearchResults> getSearchResults(final LuceneQueryInfo queryInfo)
       throws Exception {
-    securityService.authorizeRegionManage(queryInfo.getRegionPath());
-
     final String[] groups = {};
     final ResultCollector<?, ?> rc = this.executeSearch(queryInfo);
     final List<Set<LuceneSearchResults>> functionResults =
@@ -588,7 +585,7 @@ public class LuceneIndexCommands extends AbstractCommandsSupport {
 
   @CliAvailabilityIndicator({LuceneCliStrings.LUCENE_SEARCH_INDEX,
       LuceneCliStrings.LUCENE_CREATE_INDEX, LuceneCliStrings.LUCENE_DESCRIBE_INDEX,
-      LuceneCliStrings.LUCENE_LIST_INDEX})
+      LuceneCliStrings.LUCENE_LIST_INDEX, LuceneCliStrings.LUCENE_DESTROY_INDEX})
   public boolean indexCommandsAvailable() {
     return (!CliUtil.isGfshVM() || (getGfsh() != null && getGfsh().isConnectedAndReady()));
   }

http://git-wip-us.apache.org/repos/asf/geode/blob/9b49bc8f/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
new file mode 100644
index 0000000..67103ff
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneClientSecurityDUnitTest.java
@@ -0,0 +1,128 @@
+/*
+ * 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 junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.geode.cache.Cache;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.client.ClientCacheFactory;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.cache.client.ServerOperationException;
+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.SerializableCallableIF;
+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.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)
+public class LuceneClientSecurityDUnitTest extends LuceneQueriesAccessorBase {
+
+  @Test
+  @Parameters(method = "getSearchIndexUserNameAndExpectedResponses")
+  public void verifySearchIndexPermissions(
+      LuceneCommandsSecurityDUnitTest.UserNameAndExpectedResponse user) {
+    // Start server
+    int serverPort = dataStore1.invoke(startCacheServer());
+
+    // Create index and region
+    dataStore1.invoke(() -> createIndexAndRegion());
+
+    // Start client
+    accessor.invoke(() -> startClient(user.getUserName(), serverPort));
+
+    // Attempt query
+    accessor.invoke(
+        () -> executeTextSearch(user.getExpectAuthorizationError(), user.getExpectedResponse()));
+  }
+
+  private SerializableCallableIF<Integer> startCacheServer() {
+    return () -> {
+      Properties props = new Properties();
+      props.setProperty(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName());
+      final Cache cache = getCache(props);
+      final CacheServer server = cache.addCacheServer();
+      server.setPort(0);
+      server.start();
+      LuceneService luceneService = LuceneServiceProvider.get(cache);
+      luceneService.createIndexFactory().addField("text").create(INDEX_NAME, REGION_NAME);
+      cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
+      return server.getPort();
+    };
+  }
+
+  private SerializableRunnableIF createIndexAndRegion() {
+    return () -> {
+      Cache cache = getCache();
+      LuceneService luceneService = LuceneServiceProvider.get(cache);
+      luceneService.createIndexFactory().setFields("field1").create(INDEX_NAME, REGION_NAME);
+      cache.createRegionFactory(RegionShortcut.PARTITION).create(REGION_NAME);
+    };
+  }
+
+  private SerializableRunnableIF startClient(String userName, int serverPort) {
+    return () -> {
+      Properties props = new Properties();
+      props.setProperty("security-username", userName);
+      props.setProperty("security-password", userName);
+      props.setProperty(SECURITY_CLIENT_AUTH_INIT, UserPasswordAuthInit.class.getName());
+      ClientCacheFactory clientCacheFactory = new ClientCacheFactory(props);
+      clientCacheFactory.addPoolServer("localhost", serverPort);
+      ClientCache clientCache = getClientCache(clientCacheFactory);
+      clientCache.createClientRegionFactory(ClientRegionShortcut.CACHING_PROXY).create(REGION_NAME);
+    };
+  }
+
+  private SerializableRunnableIF executeTextSearch(boolean expectAuthorizationError,
+      String expectedResponse) {
+    return () -> {
+      LuceneService service = LuceneServiceProvider.get(getCache());
+      LuceneQuery<Integer, TestObject> query =
+          service.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "test", "text");
+      try {
+        query.findKeys();
+        assertFalse(expectAuthorizationError);
+      } 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)};
+  }
+}

http://git-wip-us.apache.org/repos/asf/geode/blob/9b49bc8f/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
new file mode 100644
index 0000000..ad734e8
--- /dev/null
+++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/LuceneCommandsSecurityDUnitTest.java
@@ -0,0 +1,303 @@
+/*
+ * 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 junitparams.JUnitParamsRunner;
+import junitparams.Parameters;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.lucene.internal.cli.LuceneCliStrings;
+import org.apache.geode.cache.lucene.internal.cli.LuceneIndexCommands;
+import org.apache.geode.management.cli.Result;
+import org.apache.geode.management.internal.cli.CommandManager;
+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.cache.internal.JUnit4CacheTestCase;
+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;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+
+import java.io.Serializable;
+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_MANAGER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+@Category({DistributedTest.class, SecurityTest.class})
+@RunWith(JUnitParamsRunner.class)
+public class LuceneCommandsSecurityDUnitTest extends JUnit4CacheTestCase {
+
+  @Rule
+  public LocatorServerStartupRule locatorServer = new LocatorServerStartupRule();
+
+  @Rule
+  public GfshShellConnectionRule gfshShell = new GfshShellConnectionRule();
+
+  private MemberVM locator;
+
+  private MemberVM server;
+
+  @Before
+  public void before() throws Exception {
+    startLocator();
+    startServer();
+  }
+
+  private void startLocator() throws Exception {
+    Properties props = new Properties();
+    props.setProperty(SECURITY_MANAGER, SimpleTestSecurityManager.class.getName());
+    this.locator = this.locatorServer.startLocatorVM(0, props);
+  }
+
+  private void startServer() throws Exception {
+    Properties props = new Properties();
+    props.setProperty("security-username", "clustermanage");
+    props.setProperty("security-password", "clustermanage");
+    this.server = this.locatorServer.startServerVM(1, props, this.locator.getPort());
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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);
+  }
+
+  @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, "datamanage", "datamanage");
+
+    // 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() throws Exception {
+    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
+    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() throws Exception {
+    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() throws Exception {
+    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
+    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() throws Exception {
+    CommandStringBuilder csb = new CommandStringBuilder(LuceneCliStrings.LUCENE_LIST_INDEX);
+    return csb.toString();
+  }
+
+  private String getDescribeIndexCommand() throws Exception {
+    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() throws Exception {
+    CommandManager.getInstance().add(LuceneIndexCommands.class.newInstance());
+    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();
+  }
+
+  protected UserNameAndExpectedResponse[] getCreateIndexUserNameAndExpectedResponses() {
+    return new UserNameAndExpectedResponse[] {
+        new UserNameAndExpectedResponse("nopermissions", true,
+            "Unauthorized. Reason : nopermissions not authorized for DATA:MANAGE:region"),
+        new UserNameAndExpectedResponse("datamanageregion", false,
+            "Successfully created lucene index")};
+  }
+
+  protected UserNameAndExpectedResponse[] getSearchIndexUserNameAndExpectedResponses() {
+    return new UserNameAndExpectedResponse[] {
+        new UserNameAndExpectedResponse("nopermissions", true,
+            "Unauthorized. Reason : nopermissions not authorized for DATA:WRITE"),
+        new UserNameAndExpectedResponse("datawrite", false, "No results")};
+  }
+
+  protected UserNameAndExpectedResponse[] getListIndexesUserNameAndExpectedResponses() {
+    return new UserNameAndExpectedResponse[] {
+        new UserNameAndExpectedResponse("nopermissions", true,
+            "Unauthorized. Reason : nopermissions not authorized for CLUSTER:READ"),
+        new UserNameAndExpectedResponse("clusterread", false, "Index Name")};
+  }
+
+  protected UserNameAndExpectedResponse[] getDescribeIndexUserNameAndExpectedResponses() {
+    return new UserNameAndExpectedResponse[] {
+        new UserNameAndExpectedResponse("nopermissions", true,
+            "Unauthorized. Reason : nopermissions not authorized for CLUSTER:READ"),
+        new UserNameAndExpectedResponse("clusterread", false, "Index Name")};
+  }
+
+  protected UserNameAndExpectedResponse[] getDestroyIndexUserNameAndExpectedResponses() {
+    return new UserNameAndExpectedResponse[] {
+        new UserNameAndExpectedResponse("nopermissions", true,
+            "Unauthorized. Reason : nopermissions not authorized for DATA:MANAGE:region"),
+        new UserNameAndExpectedResponse("datamanageregion", false,
+            "Successfully destroyed lucene index")};
+  }
+
+  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;
+    }
+  }
+}