You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by jj...@apache.org on 2019/10/25 10:05:46 UTC

[geode] branch develop updated: GEODE-7313: Refactor Query Security Tests (#4218)

This is an automated email from the ASF dual-hosted git repository.

jjramos pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/geode.git


The following commit(s) were added to refs/heads/develop by this push:
     new 480cdea  GEODE-7313: Refactor Query Security Tests (#4218)
480cdea is described below

commit 480cdeaefc311aac7728cb12852127df6421273c
Author: Juan José Ramos <ju...@users.noreply.github.com>
AuthorDate: Fri Oct 25 11:05:12 2019 +0100

    GEODE-7313: Refactor Query Security Tests (#4218)
    
    - Fixed warnings.
    - Replaced usages of 'junit.Assert' by 'assertj'.
    - Refactored the tests to use new rules instead of deprecated code.
    - Added extra tests to verify method invocation authorization in OQL.
---
 .../security/query/IndexSecurityDUnitTest.java     | 102 -------
 .../query/IndexSecurityDistributedTest.java        | 123 +++++++++
 ...rictedButMethodsDoNotExistQueriesDUnitTest.java |  36 ---
 ...nedQuerySecurityRestrictedQueriesDUnitTest.java |  36 ---
 ...tionedQuerySecurityAllowedQueriesDUnitTest.java |  35 ---
 .../PdxQuerySecurityAllowedQueriesDUnitTest.java   |  69 -----
 .../query/PdxQuerySecurityDistributedTest.java     | 107 +++++++
 ...PdxQuerySecurityRestrictedQueriesDUnitTest.java |  72 -----
 .../QuerySecurityAllowedQueriesDUnitTest.java      | 286 -------------------
 ...QuerySecurityAllowedQueriesDistributedTest.java | 256 +++++++++++++++++
 ...curityAuthorizedUserBindParameterDUnitTest.java |  52 ----
 .../QuerySecurityDistinctQueryDistributedTest.java |  99 -------
 ...erySecurityForbiddenQueriesDistributedTest.java | 307 +++++++++++++++++++++
 .../QuerySecurityRestrictedQueriesDUnitTest.java   | 266 ------------------
 ...rityRetrictedButMethodsDoNotExistDUnitTest.java | 125 ---------
 ...rityUnauthorizedUserBindParameterDUnitTest.java |  72 -----
 ...rySecurityWithBindParameterDistributedTest.java | 126 +++++++++
 .../dunit/CqSecurityAuthorizedUserDUnitTest.java   | 261 ------------------
 ...SecurityPartitionedAuthorizedUserDUnitTest.java |  64 -----
 ...curityPartitionedUnauthorizedUserDUnitTest.java |  37 ---
 .../dunit/CqSecurityUnauthorizedUserDUnitTest.java | 213 --------------
 ...tyExecutionContextTamperingDistributedTest.java | 137 +++++++++
 .../query/internal/ExecutionContextTamperer.java   |  27 +-
 .../CqSecurityAllowedUsersDistributedTest.java     | 235 ++++++++++++++++
 .../CqSecurityForbiddenUsersDistributedTest.java   | 172 ++++++++++++
 .../geode/security/query/TestCqListener.java       |  34 ++-
 .../AbstractQuerySecurityDistributedTest.java      | 168 +++++++++++
 .../geode/security/query/QuerySecurityBase.java    | 238 ----------------
 28 files changed, 1662 insertions(+), 2093 deletions(-)

diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/IndexSecurityDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/IndexSecurityDUnitTest.java
deleted file mode 100644
index f3f3a00..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/IndexSecurityDUnitTest.java
+++ /dev/null
@@ -1,102 +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.security.query;
-
-import static org.apache.geode.internal.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.cache.query.Index;
-import org.apache.geode.cache.query.IndexInvalidException;
-import org.apache.geode.cache.query.QueryService;
-import org.apache.geode.security.query.data.QueryTestObject;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class IndexSecurityDUnitTest extends QuerySecurityBase {
-  @Override
-  public RegionShortcut getRegionType() {
-    return RegionShortcut.REPLICATE;
-  }
-
-  @Parameterized.Parameters
-  public static Object[] usersAllowed() {
-    return new Object[] {"dataWriter"};
-  }
-
-  @Parameterized.Parameter
-  public String user;
-
-
-  @Before
-  public void configureSpecificUserAndKeyAndValues() {
-    createClientCache(specificUserClient, user, userPerms.getUserPassword(user));
-    createProxyRegion(specificUserClient, regionName);
-
-    keys = new Object[] {"key-0", "key-1", "key-2"};
-    values = new Object[] {new QueryTestObject(1, "Mary"), new QueryTestObject(2, "Joe"),
-        new QueryTestObject(3, "Joe")};
-  }
-
-  @Test
-  public void indexCreatedButPutWithNoReadCredentialsShouldNotThrowSecurityException()
-      throws Exception {
-    QueryService queryService = server.getCache().getQueryService();
-    Index idIndex = queryService.createIndex("IdIndex", "id", "/" + regionName);
-    putIntoRegion(specificUserClient, keys, values, regionName);
-  }
-
-  @Test
-  public void indexCreatedWithRegionEntriesButPutWithNoReadCredentialsShouldNotThrowSecurityException()
-      throws Exception {
-    QueryService queryService = server.getCache().getQueryService();
-    Index idIndex = queryService.createIndex("IdIndex", "e.id", "/" + regionName + ".entries e");
-    putIntoRegion(specificUserClient, keys, values, regionName);
-  }
-
-  @Test
-  public void indexCreatedWithMethodInvocationOnPrepopulatedRegionShouldThrowSecurityException()
-      throws Exception {
-    QueryService queryService = server.getCache().getQueryService();
-    putIntoRegion(superUserClient, keys, values, regionName);
-
-    try {
-      queryService.createIndex("IdIndex", "e.getName()", "/" + regionName + " e");
-      fail("Index creation should have failed due to method invocation");
-    } catch (IndexInvalidException e) {
-      assertTrue(e.getMessage().contains("Unauthorized access to method: getName"));
-    }
-  }
-
-  @Test
-  public void indexCreatedWithMethodInvocationOnUnpopulatedRegionAndPutShouldMarkIndexInvalid()
-      throws Exception {
-    QueryService queryService = server.getCache().getQueryService();
-    Index index = queryService.createIndex("IdIndex", "e.getName()", "/" + regionName + " e");
-    putIntoRegion(superUserClient, keys, values, regionName);
-    assertFalse(index.isValid());
-  }
-
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/IndexSecurityDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/IndexSecurityDistributedTest.java
new file mode 100644
index 0000000..ca98447
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/query/IndexSecurityDistributedTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.security.query;
+
+import static org.apache.geode.cache.RegionShortcut.PARTITION;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.query.Index;
+import org.apache.geode.cache.query.IndexInvalidException;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.security.query.data.QueryTestObject;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.junit.categories.OQLIndexTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+
+@RunWith(Parameterized.class)
+@Category({SecurityTest.class, OQLIndexTest.class})
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+public class IndexSecurityDistributedTest extends AbstractQuerySecurityDistributedTest {
+
+  @Parameterized.Parameters(name = "User:{0}, RegionType:{1}")
+  public static Object[] usersAndRegionTypes() {
+    return new Object[][] {
+        {"dataWriter", REPLICATE}, {"dataWriter", PARTITION},
+    };
+  }
+
+  @Parameterized.Parameter
+  public String user;
+
+  @Parameterized.Parameter(1)
+  public RegionShortcut regionShortcut;
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp(user, regionShortcut);
+
+    keys = new Object[] {"key-0", "key-1", "key-2"};
+    values = new Object[] {
+        new QueryTestObject(1, "Mary"),
+        new QueryTestObject(2, "Joe"),
+        new QueryTestObject(3, "Joe")
+    };
+  }
+
+  @Test
+  public void indexCreatedOnPublicFieldFollowedByPutWithNoReadCredentialsShouldNotThrowSecurityException() {
+    server.invoke(() -> {
+      assertThat(ClusterStartupRule.getCache()).isNotNull();
+      QueryService queryService = ClusterStartupRule.getCache().getQueryService();
+      queryService.createIndex("IdIndex", "id", "/" + regionName);
+    });
+
+    putIntoRegion(specificUserClient, keys, values, regionName);
+  }
+
+  @Test
+  public void indexCreatedOnPrivateFieldAccessibleThroughAccessorMethodFollowedByPutWithNoReadCredentialsShouldNotThrowSecurityException() {
+    server.invoke(() -> {
+      assertThat(ClusterStartupRule.getCache()).isNotNull();
+      QueryService queryService = ClusterStartupRule.getCache().getQueryService();
+      queryService.createIndex("IdIndex", "e.id", "/" + regionName + ".entries e");
+    });
+
+    putIntoRegion(specificUserClient, keys, values, regionName);
+  }
+
+  @Test
+  public void indexCreationWithMethodInvocationOnPopulatedRegionShouldThrowSecurityException() {
+    putIntoRegion(superUserClient, keys, values, regionName);
+
+    server.invoke(() -> {
+      assertThat(ClusterStartupRule.getCache()).isNotNull();
+      QueryService queryService = ClusterStartupRule.getCache().getQueryService();
+      assertThatThrownBy(
+          () -> queryService.createIndex("IdIndex", "e.getName()", "/" + regionName + " e"))
+              .isInstanceOf(IndexInvalidException.class)
+              .hasMessageContaining("Unauthorized access to method: getName");
+    });
+  }
+
+  @Test
+  public void indexCreationWithMethodInvocationOnEmptyRegionFollowedByPutShouldMarkIndexAsInvalid() {
+    server.invoke(() -> {
+      assertThat(ClusterStartupRule.getCache()).isNotNull();
+      QueryService queryService = ClusterStartupRule.getCache().getQueryService();
+      queryService.createIndex("IdIndex", "e.getName()", "/" + regionName + " e");
+    });
+
+    putIntoRegion(superUserClient, keys, values, regionName);
+
+    server.invoke(() -> {
+      assertThat(ClusterStartupRule.getCache()).isNotNull();
+      QueryService queryService = ClusterStartupRule.getCache().getQueryService();
+      Region region = ClusterStartupRule.getCache().getRegion(regionName);
+      Index index = queryService.getIndex(region, "IdIndex");
+      assertThat(index.isValid()).isFalse();
+    });
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedQuerySecurityRestrictedButMethodsDoNotExistQueriesDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedQuerySecurityRestrictedButMethodsDoNotExistQueriesDUnitTest.java
deleted file mode 100644
index fd2e576..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedQuerySecurityRestrictedButMethodsDoNotExistQueriesDUnitTest.java
+++ /dev/null
@@ -1,36 +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.security.query;
-
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class PartitionedQuerySecurityRestrictedButMethodsDoNotExistQueriesDUnitTest
-    extends QuerySecurityRetrictedButMethodsDoNotExistDUnitTest {
-
-  @Override
-  public RegionShortcut getRegionType() {
-    return RegionShortcut.PARTITION;
-  }
-
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedQuerySecurityRestrictedQueriesDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedQuerySecurityRestrictedQueriesDUnitTest.java
deleted file mode 100644
index 8130ae5..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedQuerySecurityRestrictedQueriesDUnitTest.java
+++ /dev/null
@@ -1,36 +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.security.query;
-
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class PartitionedQuerySecurityRestrictedQueriesDUnitTest
-    extends QuerySecurityRestrictedQueriesDUnitTest {
-
-  @Override
-  public RegionShortcut getRegionType() {
-    return RegionShortcut.PARTITION;
-  }
-
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxPartitionedQuerySecurityAllowedQueriesDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxPartitionedQuerySecurityAllowedQueriesDUnitTest.java
deleted file mode 100644
index f289325..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxPartitionedQuerySecurityAllowedQueriesDUnitTest.java
+++ /dev/null
@@ -1,35 +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.security.query;
-
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class PdxPartitionedQuerySecurityAllowedQueriesDUnitTest
-    extends PdxQuerySecurityAllowedQueriesDUnitTest {
-  @Override
-  public RegionShortcut getRegionType() {
-    return RegionShortcut.PARTITION;
-  }
-
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxQuerySecurityAllowedQueriesDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxQuerySecurityAllowedQueriesDUnitTest.java
deleted file mode 100644
index b00bd74..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxQuerySecurityAllowedQueriesDUnitTest.java
+++ /dev/null
@@ -1,69 +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.security.query;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.security.query.data.PdxQueryTestObject;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class PdxQuerySecurityAllowedQueriesDUnitTest extends QuerySecurityBase {
-  @Parameterized.Parameters
-  public static Object[] usersAllowed() {
-    return new Object[] {"dataReader", "dataReaderRegion", "clusterManagerDataReader",
-        "clusterManagerDataReaderRegion", "super-user"};
-  }
-
-  @Parameterized.Parameter
-  public String user;
-
-  @Before
-  public void configureSpecificClientAndKeyAndValues() {
-    createClientCache(specificUserClient, user, userPerms.getUserPassword(user));
-    createProxyRegion(specificUserClient, regionName);
-
-    keys = new Object[] {"key-0", "key-1"};
-    values = new Object[] {new PdxQueryTestObject(0, "John"), new PdxQueryTestObject(3, "Beth")};
-    putIntoRegion(superUserClient, keys, values, regionName);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectOnPublicFieldQuery() {
-    String query = "select * from /" + regionName + " r where r.id = 3";
-    List<Object> expectedResults = Arrays.asList(values[1]);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectWithPdxFieldNamedGetQuery() {
-    server.getCache().setReadSerializedForTest(true);
-    String query = "select * from /" + regionName + " r where r.getName = 'Beth'";
-    List<Object> expectedResults = Arrays.asList(values[1]);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxQuerySecurityDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxQuerySecurityDistributedTest.java
new file mode 100644
index 0000000..42fdc25
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxQuerySecurityDistributedTest.java
@@ -0,0 +1,107 @@
+/*
+ * 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.security.query;
+
+import static org.apache.geode.cache.RegionShortcut.PARTITION;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.security.query.data.PdxQueryTestObject;
+import org.apache.geode.test.junit.categories.OQLQueryTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+
+@RunWith(Parameterized.class)
+@Category({SecurityTest.class, OQLQueryTest.class})
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+public class PdxQuerySecurityDistributedTest extends AbstractQuerySecurityDistributedTest {
+
+  @Parameterized.Parameters(name = "User:{0}, RegionType:{1}")
+  public static Object[] usersAndRegionTypes() {
+    return new Object[][] {
+        {"super-user", REPLICATE}, {"super-user", PARTITION},
+        {"dataReader", REPLICATE}, {"dataReader", PARTITION},
+        {"dataReaderRegion", REPLICATE}, {"dataReaderRegion", PARTITION},
+        {"clusterManagerDataReader", REPLICATE}, {"clusterManagerDataReader", PARTITION},
+        {"clusterManagerDataReaderRegion", REPLICATE}, {"clusterManagerDataReaderRegion", PARTITION}
+    };
+  }
+
+  @Parameterized.Parameter
+  public String user;
+
+  @Parameterized.Parameter(1)
+  public RegionShortcut regionShortcut;
+
+  private String regexForExpectedExceptions = ".*Unauthorized access.*";
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp(user, regionShortcut);
+
+    keys = new Object[] {"key-0", "key-1"};
+    values = new Object[] {
+        new PdxQueryTestObject(0, "John"),
+        new PdxQueryTestObject(3, "Beth")
+    };
+
+    putIntoRegion(superUserClient, keys, values, regionName);
+  }
+
+  @Test
+  public void queryWithPublicFieldAccessOnWhereClauseShouldNotThrowSecurityException() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.id = 3";
+    List<Object> expectedResults = Collections.singletonList(values[1]);
+    executeQueryAndAssertExpectedResults(specificUserClient, query, expectedResults);
+  }
+
+  @Test
+  public void queryWithPdxNamedFieldAccessOnWhereClauseShouldNotThrowSecurityException() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.getName = 'Beth'";
+    List<Object> expectedResults = Collections.singletonList(values[1]);
+    executeQueryAndAssertExpectedResults(specificUserClient, query, expectedResults);
+  }
+
+  @Test
+  public void queryWithMethodInvocationShouldThrowSecurityExceptionForPdxObjects() {
+    String query1 = "SELECT r.getAge FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query1,
+        regexForExpectedExceptions);
+
+    String query2 = "SELECT * FROM /" + regionName + " r WHERE r.name = 'Beth'";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query2,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queryWithPermanentlyForbiddenMethodShouldThrowSecurityExceptionForPdxObjects() {
+    String query1 = "SELECT r.getClass FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query1,
+        regexForExpectedExceptions);
+
+    String query2 = "SELECT * FROM /" + regionName + " r WHERE r.getClass() != 3";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query2,
+        regexForExpectedExceptions);
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxQuerySecurityRestrictedQueriesDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxQuerySecurityRestrictedQueriesDUnitTest.java
deleted file mode 100644
index 6a2db14..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PdxQuerySecurityRestrictedQueriesDUnitTest.java
+++ /dev/null
@@ -1,72 +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.security.query;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.security.query.data.PdxQueryTestObject;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class PdxQuerySecurityRestrictedQueriesDUnitTest extends QuerySecurityBase {
-  @Parameterized.Parameters
-  public static Object[] usersAllowed() {
-    return new Object[] {"dataReader", "dataReaderRegion", "clusterManagerDataReader",
-        "clusterManagerDataReaderRegion", "super-user"};
-  }
-
-  @Parameterized.Parameter
-  public String user;
-
-  private String regexForExpectedExceptions = ".*Unauthorized access.*";
-
-  @Before
-  public void configureCache() {
-    createClientCache(specificUserClient, user, userPerms.getUserPassword(user));
-    createProxyRegion(specificUserClient, regionName);
-
-    keys = new Object[] {"key-0", "key-1"};
-    values = new Object[] {new PdxQueryTestObject(0, "John"), new PdxQueryTestObject(3, "Beth")};
-    putIntoRegion(superUserClient, keys, values, regionName);
-  }
-
-  @Test
-  public void selectWhenInvokingMethodOnPdxObjectQueryShouldBeRestricted() {
-    String query = "select r.getClass from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void invokingMethodOnPdxObjectShouldBeRestricted() {
-    String query = "select r.getAge from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void selectWithPdxFieldNoExistingPublicFieldQueryShouldBeRestricted() {
-    String query = "select * from /" + regionName + " r where r.name = 'Beth'";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityAllowedQueriesDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityAllowedQueriesDUnitTest.java
deleted file mode 100644
index 1c6b3f2..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityAllowedQueriesDUnitTest.java
+++ /dev/null
@@ -1,286 +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.security.query;
-
-import java.util.Arrays;
-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.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.security.query.data.QueryTestObject;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class QuerySecurityAllowedQueriesDUnitTest extends QuerySecurityBase {
-
-  @Parameterized.Parameters
-  public static Object[] usersAllowed() {
-    return new Object[] {"dataReader", "dataReaderRegion", "clusterManagerDataReader",
-        "clusterManagerDataReaderRegion", "super-user"};
-  }
-
-  @Parameterized.Parameter
-  public String user;
-
-  @Before
-  public void configureCache() {
-    createClientCache(specificUserClient, user, userPerms.getUserPassword(user));
-    createProxyRegion(specificUserClient, regionName);
-
-    keys = new Object[] {"key-0", "key-1"};
-    values = new Object[] {new QueryTestObject(1, "John"), new QueryTestObject(3, "Beth")};
-    putIntoRegion(superUserClient, keys, values, regionName);
-  }
-
-
-  @Test
-  public void checkUserAuthorizationsForSelectAllQuery() {
-    String query = "select * from /" + regionName;
-    List<Object> expectedResults = Arrays.asList(values);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  /* ----- Public Field Tests ----- */
-  @Test
-  public void checkUserAuthorizationsForSelectByPublicFieldQuery() {
-    String query = "select * from /" + regionName + " r where r.id = 1";
-    List<Object> expectedResults = Arrays.asList(values[0]);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectPublicFieldQuery() {
-    String query = "select r.id from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList(1, 3);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectCountOfPublicFieldQuery() {
-    String query = "select count(r.id) from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList(2);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectMaxOfPublicFieldQuery() {
-    String query = "select max(r.id) from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList(3);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectMinOfPublicFieldQuery() {
-    String query = "select min(r.id) from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList(1);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectPublicFieldFromRegionByPublicFieldFromRegionQuery() {
-    String query = "select * from /" + regionName + " r1 where r1.id in (select r2.id from /"
-        + regionName + " r2)";
-    List<Object> expectedResults = Arrays.asList(values);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectRegionContainsKeyQuery() {
-    String query = "select * from /" + regionName + ".containsKey('" + keys[0] + "')";
-    List<Object> expectedResults = Arrays.asList(true);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectValuesQuery() {
-    String query = "select * from /" + regionName + ".values";
-    List<Object> expectedResults = Arrays.asList(values);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectKeySetQuery() {
-    String query = "select * from /" + regionName + ".keySet";
-    List<Object> expectedResults = Arrays.asList(keys);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectEntriesQuery() {
-    String query = "select e.getKey from /" + regionName + ".entries e";
-    List<Object> expectedResults = Arrays.asList(keys);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectKeyFromEntrySetQuery() {
-    String query = "select e.key from /" + regionName + ".entrySet e";
-    List<Object> expectedResults = Arrays.asList(keys);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectGetKeyFromEntrySetQuery() {
-    String query = "select e.getKey from /" + regionName + ".entrySet e";
-    List<Object> expectedResults = Arrays.asList(keys);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectValueFromEntrySetQuery() {
-    String query = "select e.value from /" + regionName + ".entrySet e";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        Arrays.asList(values));
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectGetValueFromEntrySetQuery() {
-    String query = "select e.getValue from /" + regionName + ".entrySet e";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        Arrays.asList(values));
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectByToStringQuery() {
-    String query = "select * from /" + regionName + " r where r.toString = 'Test_Object'";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        Arrays.asList(values));
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectToStringQuery() {
-    String query = "select r.toString() from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList("Test_Object", "Test_Object");
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectByToStringToUpperCaseQuery() {
-    String query =
-        "select * from /" + regionName + " r where r.toString().toUpperCase = 'TEST_OBJECT'";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        Arrays.asList(values));
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectByToStringToLowerCaseQuery() {
-    String query =
-        "select * from /" + regionName + " r where r.toString().toLowerCase = 'test_object'";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        Arrays.asList(values));
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectIntValueQuery() {
-    String query = "select r.id.intValue() from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList(1, 3);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectLongValueQuery() {
-    String query = "select r.id.longValue() from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList(1L, 3L);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectDoubleValueQuery() {
-    String query = "select r.id.doubleValue() from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList(1d, 3d);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectShortValueQuery() {
-    String query = "select r.id.shortValue() from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList((short) 1, (short) 3);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectToDateQuery() {
-    String query =
-        "SELECT * FROM /" + regionName + " where dateField = to_date('08/08/2018', 'MM/dd/yyyy')";
-
-    QueryTestObject obj1 = new QueryTestObject(0, "John");
-    obj1.setDateField("08/08/2018");
-    QueryTestObject obj2 = new QueryTestObject(3, "Beth");
-    obj2.setDateField("08/08/2018");
-    Object[] values = {obj1, obj2};
-    putIntoRegion(superUserClient, keys, values, regionName);
-
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        Arrays.asList(values));
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectByMapFieldQuery() {
-    QueryTestObject valueObject1 = new QueryTestObject(1, "John");
-    Map<Object, Object> map1 = new HashMap<>();
-    map1.put("intData", 1);
-    map1.put(1, 98);
-    map1.put("strData1", "ABC");
-    map1.put("strData2", "ZZZ");
-    valueObject1.mapField = map1;
-    QueryTestObject valueObject2 = new QueryTestObject(3, "Beth");
-    Map<Object, Object> map2 = new HashMap<>();
-    map2.put("intData", 99);
-    map2.put(1, 99);
-    map2.put("strData1", "XYZ");
-    map2.put("strData2", "ZZZ");
-    valueObject2.mapField = map2;
-    values = new Object[] {valueObject1, valueObject2};
-    putIntoRegion(superUserClient, keys, values, regionName);
-
-    String query1 = String.format(
-        "SELECT * FROM /%s WHERE mapField.get('intData') = 1 AND mapField.get(1) = 98 AND mapField.get('strData1') = 'ABC' AND mapField.get('strData2') = 'ZZZ'",
-        regionName);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query1, regionName,
-        Arrays.asList(new Object[] {valueObject1}));
-
-    String query2 =
-        String.format("SELECT * FROM /%s WHERE mapField.get('strData2') = 'ZZZ'", regionName);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query2, regionName,
-        Arrays.asList(values));
-  }
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityAllowedQueriesDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityAllowedQueriesDistributedTest.java
new file mode 100644
index 0000000..30ffe83
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityAllowedQueriesDistributedTest.java
@@ -0,0 +1,256 @@
+/*
+ * 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.security.query;
+
+import static org.apache.geode.cache.RegionShortcut.PARTITION;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+
+import java.util.Arrays;
+import java.util.Collections;
+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.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.security.query.data.QueryTestObject;
+import org.apache.geode.test.junit.categories.OQLQueryTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+
+@RunWith(Parameterized.class)
+@Category({SecurityTest.class, OQLQueryTest.class})
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+public class QuerySecurityAllowedQueriesDistributedTest
+    extends AbstractQuerySecurityDistributedTest {
+
+  @Parameterized.Parameters(name = "User:{0}, RegionType:{1}")
+  public static Object[] usersAndRegionTypes() {
+    return new Object[][] {
+        {"super-user", REPLICATE}, {"super-user", PARTITION},
+        {"dataReader", REPLICATE}, {"dataReader", PARTITION},
+        {"dataReaderRegion", REPLICATE}, {"dataReaderRegion", PARTITION},
+        {"clusterManagerDataReader", REPLICATE}, {"clusterManagerDataReader", PARTITION},
+        {"clusterManagerDataReaderRegion", REPLICATE}, {"clusterManagerDataReaderRegion", PARTITION}
+    };
+  }
+
+  @Parameterized.Parameter
+  public String user;
+
+  @Parameterized.Parameter(1)
+  public RegionShortcut regionShortcut;
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp(user, regionShortcut);
+
+    keys = new Object[] {"key-0", "key-1"};
+    values = new Object[] {
+        new QueryTestObject(1, "John"),
+        new QueryTestObject(3, "Beth")
+    };
+
+    putIntoRegion(superUserClient, keys, values, regionName);
+  }
+
+  @Test
+  public void queryWithSelectStarShouldNotThrowSecurityException() {
+    String query = "SELECT * FROM /" + regionName;
+    List<Object> expectedResults = Arrays.asList(values);
+    executeQueryAndAssertExpectedResults(specificUserClient, query, expectedResults);
+  }
+
+  /* ----- Public Field Tests ----- */
+  @Test
+  public void queryWithPublicFieldOnWhereClauseShouldNotThrowSecurityException() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.id = 1";
+    List<Object> expectedResults = Collections.singletonList(values[0]);
+    executeQueryAndAssertExpectedResults(specificUserClient, query, expectedResults);
+  }
+
+  @Test
+  public void queryingByPublicFieldOnSelectClauseShouldNotThrowSecurityException() {
+    String query = "SELECT r.id FROM /" + regionName + " r";
+    List<Object> expectedResults = Arrays.asList(1, 3);
+    executeQueryAndAssertExpectedResults(specificUserClient, query, expectedResults);
+  }
+
+  @Test
+  public void queriesWithPublicFieldUsedWithinAggregateFunctionsShouldNotThrowSecurityException() {
+    String queryCount = "SELECT COUNT(r.id) FROM /" + regionName + " r";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryCount,
+        Collections.singletonList(2));
+
+    String queryMax = "SELECT MAX(r.id) FROM /" + regionName + " r";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryMax,
+        Collections.singletonList(3));
+
+    String queryMin = "SELECT MIN(r.id) FROM /" + regionName + " r";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryMin,
+        Collections.singletonList(1));
+
+    String queryAvg = "SELECT AVG(r.id) FROM /" + regionName + " r";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryAvg,
+        Collections.singletonList(2));
+
+    String querySum = "SELECT SUM(r.id) FROM /" + regionName + " r";
+    executeQueryAndAssertExpectedResults(specificUserClient, querySum,
+        Collections.singletonList(4));
+  }
+
+  @Test
+  public void queryWithPublicFieldUsedWithinDistinctClauseShouldNotThrowSecurityException() {
+    String query = "<TRACE> SELECT DISTINCT * from /" + regionName
+        + " WHERE id IN SET(1, 3) ORDER BY id asc LIMIT 2";
+    executeQueryAndAssertExpectedResults(specificUserClient, query, Arrays.asList(values));
+  }
+
+  @Test
+  public void queryWithByPublicFieldOnInnerQueriesShouldNotThrowSecurityException() {
+    String query = "SELECT * FROM /" + regionName + " r1 WHERE r1.id IN (SELECT r2.id FROM /"
+        + regionName + " r2)";
+    List<Object> expectedResults = Arrays.asList(values);
+    executeQueryAndAssertExpectedResults(specificUserClient, query, expectedResults);
+  }
+
+  /* ----- Default Allowed Methods Tests ----- */
+  @Test
+  public void queriesWithAllowedRegionMethodInvocationsShouldNotThrowSecurityException() {
+    String queryValues = "SELECT * FROM /" + regionName + ".values";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryValues, Arrays.asList(values));
+
+    String queryKeySet = "SELECT * FROM /" + regionName + ".keySet";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryKeySet, Arrays.asList(keys));
+
+    String queryContainsKey = "SELECT * FROM /" + regionName + ".containsKey('" + keys[0] + "')";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryContainsKey,
+        Collections.singletonList(true));
+
+    String queryEntrySet = "SELECT * FROM /" + regionName + ".get('" + keys[0] + "')";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryEntrySet,
+        Collections.singletonList(values[0]));
+  }
+
+  @Test
+  public void queriesWithAllowedRegionEntryMethodInvocationsShouldNotThrowSecurityException() {
+    List<Object> expectedKeys = Arrays.asList(keys);
+    String queryKeyEntrySet = "SELECT e.key FROM /" + regionName + ".entrySet e";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryKeyEntrySet, expectedKeys);
+    String queryGetKeyEntrySet = "SELECT e.getKey FROM /" + regionName + ".entrySet e";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryGetKeyEntrySet, expectedKeys);
+    String queryKeyEntries = "SELECT e.key FROM /" + regionName + ".entries e";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryKeyEntries, expectedKeys);
+    String queryGetKeyEntries = "SELECT e.getKey FROM /" + regionName + ".entries e";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryGetKeyEntries, expectedKeys);
+
+    List<Object> expectedValues = Arrays.asList(values);
+    String queryValueEntrySet = "SELECT e.value FROM /" + regionName + ".entrySet e";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryValueEntrySet, expectedValues);
+    String queryGetValueEntrySet = "SELECT e.getValue FROM /" + regionName + ".entrySet e";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryGetValueEntrySet, expectedValues);
+    String queryValueEntries = "SELECT e.value FROM /" + regionName + ".entries e";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryValueEntries, expectedValues);
+    String queryGetValueEntries = "SELECT e.getValue FROM /" + regionName + ".entries e";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryGetValueEntries, expectedValues);
+  }
+
+  @Test
+  public void queriesWithAllowedStringMethodInvocationsShouldNotThrowSecurityException() {
+    String toStringWhere = "SELECT * FROM /" + regionName + " r WHERE r.toString = 'Test_Object'";
+    executeQueryAndAssertExpectedResults(specificUserClient, toStringWhere, Arrays.asList(values));
+
+    String toStringSelect = "SELECT r.toString() FROM /" + regionName + " r";
+    executeQueryAndAssertExpectedResults(specificUserClient, toStringSelect,
+        Arrays.asList("Test_Object", "Test_Object"));
+
+    String toUpperCaseWhere =
+        "SELECT * FROM /" + regionName + " r WHERE r.toString().toUpperCase = 'TEST_OBJECT'";
+    executeQueryAndAssertExpectedResults(specificUserClient, toUpperCaseWhere,
+        Arrays.asList(values));
+
+    String toLowerCaseWhere =
+        "SELECT * FROM /" + regionName + " r WHERE r.toString().toLowerCase = 'test_object'";
+    executeQueryAndAssertExpectedResults(specificUserClient, toLowerCaseWhere,
+        Arrays.asList(values));
+  }
+
+  @Test
+  public void queriesWithAllowedNumberMethodInvocationsShouldNotThrowSecurityException() {
+    String intValue = "SELECT r.id.intValue() FROM /" + regionName + " r";
+    executeQueryAndAssertExpectedResults(specificUserClient, intValue, Arrays.asList(1, 3));
+
+    String longValue = "SELECT r.id.longValue() FROM /" + regionName + " r";
+    executeQueryAndAssertExpectedResults(specificUserClient, longValue, Arrays.asList(1L, 3L));
+
+    String doubleValue = "SELECT r.id.doubleValue() FROM /" + regionName + " r";
+    executeQueryAndAssertExpectedResults(specificUserClient, doubleValue, Arrays.asList(1d, 3d));
+
+    String shortValue = "SELECT r.id.shortValue() FROM /" + regionName + " r";
+    executeQueryAndAssertExpectedResults(specificUserClient, shortValue,
+        Arrays.asList((short) 1, (short) 3));
+  }
+
+  @Test
+  public void queriesWithAllowedDateMethodInvocationsShouldNotThrowSecurityException() {
+    String query =
+        "SELECT * FROM /" + regionName + " WHERE dateField = to_date('08/08/2018', 'MM/dd/yyyy')";
+    QueryTestObject obj1 = new QueryTestObject(0, "John");
+    obj1.setDateField("08/08/2018");
+    QueryTestObject obj2 = new QueryTestObject(3, "Beth");
+    obj2.setDateField("08/08/2018");
+    Object[] values = {obj1, obj2};
+    putIntoRegion(superUserClient, keys, values, regionName);
+
+    executeQueryAndAssertExpectedResults(specificUserClient, query, Arrays.asList(values));
+  }
+
+  @Test
+  public void queriesWithAllowedMapMethodInvocationsShouldNotThrowSecurityException() {
+    QueryTestObject valueObject1 = new QueryTestObject(1, "John");
+    Map<Object, Object> map1 = new HashMap<>();
+    map1.put("intData", 1);
+    map1.put(1, 98);
+    map1.put("strData1", "ABC");
+    map1.put("strData2", "ZZZ");
+    valueObject1.mapField = map1;
+
+    QueryTestObject valueObject2 = new QueryTestObject(3, "Beth");
+    Map<Object, Object> map2 = new HashMap<>();
+    map2.put("intData", 99);
+    map2.put(1, 99);
+    map2.put("strData1", "XYZ");
+    map2.put("strData2", "ZZZ");
+    valueObject2.mapField = map2;
+
+    values = new Object[] {valueObject1, valueObject2};
+    putIntoRegion(superUserClient, keys, values, regionName);
+
+    String query1 = String.format(
+        "SELECT * FROM /%s WHERE mapField.get('intData') = 1 AND mapField.get(1) = 98 AND mapField.get('strData1') = 'ABC' AND mapField.get('strData2') = 'ZZZ'",
+        regionName);
+    executeQueryAndAssertExpectedResults(specificUserClient, query1,
+        Arrays.asList(new Object[] {valueObject1}));
+
+    String query2 =
+        String.format("SELECT * FROM /%s WHERE mapField.get('strData2') = 'ZZZ'", regionName);
+    executeQueryAndAssertExpectedResults(specificUserClient, query2, Arrays.asList(values));
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityAuthorizedUserBindParameterDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityAuthorizedUserBindParameterDUnitTest.java
deleted file mode 100644
index 356468b..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityAuthorizedUserBindParameterDUnitTest.java
+++ /dev/null
@@ -1,52 +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.security.query;
-
-import java.util.Arrays;
-import java.util.HashSet;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.cache.Region;
-import org.apache.geode.security.query.data.QueryTestObject;
-import org.apache.geode.test.junit.categories.SecurityTest;
-
-@Category({SecurityTest.class})
-public class QuerySecurityAuthorizedUserBindParameterDUnitTest extends QuerySecurityBase {
-
-  @Before
-  public void configureCache() {
-    String user = "dataReaderRegion";
-    createClientCache(specificUserClient, user, userPerms.getUserPassword(user));
-    createProxyRegion(specificUserClient, regionName);
-
-    keys = new Object[] {"key-0", "key-1"};
-    values = new Object[] {new QueryTestObject(1, "John"), new QueryTestObject(3, "Beth")};
-    putIntoRegion(superUserClient, keys, values, regionName);
-  }
-
-  @Test
-  public void userWithRegionAccessAndPassingInWrappedBindParameterShouldReturnCorrectResults() {
-    String query = "select v from $1 r, r.values() v";
-    specificUserClient.invoke(() -> {
-      Region region = getClientCache().getRegion(regionName);
-      HashSet hashset = new HashSet();
-      hashset.add(region);
-      assertQueryResults(getClientCache(), query, new Object[] {hashset}, Arrays.asList(values));
-    });
-  }
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityDistinctQueryDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityDistinctQueryDistributedTest.java
deleted file mode 100644
index d4cab5c..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityDistinctQueryDistributedTest.java
+++ /dev/null
@@ -1,99 +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.security.query;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-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.RegionShortcut;
-import org.apache.geode.cache.client.ServerOperationException;
-import org.apache.geode.cache.query.QueryService;
-import org.apache.geode.cache.query.SelectResults;
-import org.apache.geode.cache.query.security.RestrictedMethodAuthorizer;
-import org.apache.geode.security.NotAuthorizedException;
-import org.apache.geode.security.query.data.PdxTrade;
-import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.junit.categories.SecurityTest;
-
-/**
- * This test verifies client/server distinct order-by queries with integrated security
- */
-@Category(SecurityTest.class)
-@RunWith(JUnitParamsRunner.class)
-public class QuerySecurityDistinctQueryDistributedTest extends QuerySecurityBase {
-
-  private static final int NUM_ENTRIES = 1000;
-
-  @Override
-  public RegionShortcut getRegionType() {
-    return RegionShortcut.PARTITION;
-  }
-
-  @Test
-  @Parameters({"99", "100", "499", "500", "999", "1000", "1500"})
-  public void verifyDistinctOrderByQueryWithLimits(int limit) {
-    // Do puts from the client
-    putIntoRegion(superUserClient, NUM_ENTRIES);
-
-    // Execute query from the client and validate the size of the results
-    superUserClient.invoke(() -> {
-      String query = "<trace> select distinct * from /" + regionName
-          + " where cusip = 'PVTL' order by id asc limit " + limit;
-      QueryService queryService = getClientCache().getQueryService();
-      Object results = queryService.newQuery(query).execute();
-      assertThat(results).isInstanceOf(SelectResults.class);
-      assertThat(((SelectResults) results).size()).isEqualTo(Math.min(limit, NUM_ENTRIES));
-    });
-  }
-
-  @Test
-  public void verifyDistinctOrderByQueryOnMethodFails() {
-    // Do puts from the client
-    putIntoRegion(superUserClient, 1);
-
-    // Execute query from the client and validate the size of the results
-    superUserClient.invoke(() -> {
-      String methodName = "getCusip";
-      String query = "<trace> select distinct * from /" + regionName
-          + " where " + methodName + " = 'PVTL' order by id asc";
-      QueryService queryService = getClientCache().getQueryService();
-      try {
-        queryService.newQuery(query).execute();
-      } catch (Exception e) {
-        assertThat(e).isInstanceOf(ServerOperationException.class);
-        assertThat(e.getCause()).isInstanceOf(NotAuthorizedException.class);
-        assertThat(e.getMessage()).contains(
-            RestrictedMethodAuthorizer.UNAUTHORIZED_STRING + methodName);
-      }
-    });
-  }
-
-  private void putIntoRegion(VM clientVM, int numEntries) {
-    keys = new Object[numEntries];
-    values = new Object[numEntries];
-    for (int i = 0; i < numEntries; i++) {
-      String key = String.valueOf(i);
-      Object value = new PdxTrade(key, "PVTL", 100, 30);
-      keys[i] = key;
-      values[i] = value;
-    }
-    putIntoRegion(clientVM, keys, values, regionName);
-  }
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityForbiddenQueriesDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityForbiddenQueriesDistributedTest.java
new file mode 100644
index 0000000..cc98597
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityForbiddenQueriesDistributedTest.java
@@ -0,0 +1,307 @@
+/*
+ * 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.security.query;
+
+import static org.apache.geode.cache.RegionShortcut.PARTITION;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.security.query.data.QueryTestObject;
+import org.apache.geode.test.junit.categories.OQLQueryTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+
+@RunWith(Parameterized.class)
+@Category({SecurityTest.class, OQLQueryTest.class})
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+public class QuerySecurityForbiddenQueriesDistributedTest
+    extends AbstractQuerySecurityDistributedTest {
+
+  @Parameterized.Parameters(name = "User:{0}, RegionType:{1}")
+  public static Object[] usersAndRegionTypes() {
+    return new Object[][] {
+        {"super-user", REPLICATE}, {"super-user", PARTITION},
+        {"dataReader", REPLICATE}, {"dataReader", PARTITION},
+        {"dataReaderRegion", REPLICATE}, {"dataReaderRegion", PARTITION},
+        {"dataReaderRegionKey", REPLICATE}, {"dataReaderRegionKey", PARTITION},
+        {"clusterManagerDataReader", REPLICATE}, {"clusterManagerDataReader", PARTITION},
+        {"clusterManagerDataReaderRegion", REPLICATE}, {"clusterManagerDataReaderRegion", PARTITION}
+    };
+  }
+
+  @Parameterized.Parameter
+  public String user;
+
+  @Parameterized.Parameter(1)
+  public RegionShortcut regionShortcut;
+
+  private String regexForExpectedExceptions =
+      ".*Unauthorized access.*|.*dataReaderRegionKey not authorized for DATA:READ:region.*";
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp(user, regionShortcut);
+
+    keys = new Object[] {"key-0", "key-1"};
+    values = new Object[] {
+        new QueryTestObject(1, "John"),
+        new QueryTestObject(3, "Beth")
+    };
+
+    putIntoRegion(superUserClient, keys, values, regionName);
+  }
+
+  /* ----- Implicit Getter Tests ----- */
+  @Test
+  public void queryWithImplicitMethodInvocationOnWhereClauseShouldThrowSecurityException() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.name = 'Beth'";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queryWithImplicitMethodInvocationOnSelectClauseShouldThrowSecurityException() {
+    String query = "SELECT r.name FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queriesWithImplicitMethodInvocationUsedWithinAggregateFunctionsShouldThrowSecurityException() {
+    String queryCount = "SELECT COUNT(r.name) FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryCount,
+        regexForExpectedExceptions);
+
+    String queryMax = "SELECT MAX(r.name) FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryMax,
+        regexForExpectedExceptions);
+
+    String queryMin = "SELECT MIN(r.name) FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryMin,
+        regexForExpectedExceptions);
+
+    String queryAvg = "SELECT AVG(r.name) FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryAvg,
+        regexForExpectedExceptions);
+
+    String querySum = "SELECT SUM(r.name) FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, querySum,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queryWithImplicitMethodInvocationUsedWithinDistinctClauseShouldThrowSecurityException() {
+    String query = "<TRACE> SELECT DISTINCT * from /" + regionName
+        + " r WHERE r.name IN SET('John', 'Beth') ORDER BY r.id asc LIMIT 2";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queryWithImplicitMethodInvocationOnInnerQueriesShouldThrowSecurityException() {
+    String query = "SELECT * FROM /" + regionName + " r1 WHERE r1.name IN (SELECT r2.name FROM /"
+        + regionName + " r2)";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query,
+        regexForExpectedExceptions);
+  }
+
+  /* ----- Direct Getter Tests ----- */
+  @Test
+  public void queryWithExplicitMethodInvocationOnWhereClauseShouldThrowSecurityException() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.getName = 'Beth'";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queryWithExplicitMethodInvocationOnSelectClauseShouldThrowSecurityException() {
+    String query = "SELECT r.getName FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queriesWithExplicitMethodInvocationUsedWithinAggregateFunctionsShouldThrowSecurityException() {
+    String queryCount = "SELECT COUNT(r.getName) FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryCount,
+        regexForExpectedExceptions);
+
+    String queryMax = "SELECT MAX(r.getName) FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryMax,
+        regexForExpectedExceptions);
+
+    String queryMin = "SELECT MIN(r.getName) FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryMin,
+        regexForExpectedExceptions);
+
+    String queryAvg = "SELECT AVG(r.getName) FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryAvg,
+        regexForExpectedExceptions);
+
+    String querySum = "SELECT SUM(r.getName) FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, querySum,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queryWithExplicitMethodInvocationUsedWithinDistinctClauseShouldThrowSecurityException() {
+    String query = "<TRACE> SELECT DISTINCT * from /" + regionName
+        + " r WHERE r.getName IN SET('John', 'Beth') ORDER BY r.id asc LIMIT 2";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queryWithExplicitMethodInvocationOnInnerQueriesShouldThrowSecurityException() {
+    String query = "SELECT * FROM /" + regionName
+        + " r1 WHERE r1.getName IN (SELECT r2.getName FROM /" + regionName + " r2)";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query,
+        regexForExpectedExceptions);
+  }
+
+  /* ----- Region Methods ----- */
+  @Test
+  public void queriesWithAllowedRegionMethodInvocationsShouldThrowSecurityExceptionForNonAuthorizedUsers() {
+    Assume.assumeTrue(user.equals("dataReaderRegionKey"));
+
+    String queryValues = "SELECT * FROM /" + regionName + ".values";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryValues,
+        regexForExpectedExceptions);
+
+    String queryKeySet = "SELECT * FROM /" + regionName + ".keySet";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryKeySet,
+        regexForExpectedExceptions);
+
+    String queryContainsKey = "SELECT * FROM /" + regionName + ".containsKey('" + keys[0] + "')";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryContainsKey,
+        regexForExpectedExceptions);
+
+    String queryEntrySet = "SELECT * FROM /" + regionName + ".get('" + keys[0] + "')";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryEntrySet,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queriesWithRegionMutatorMethodInvocationsShouldThrowSecurityException() {
+    String queryCreate = "SELECT * FROM /" + regionName + ".create('key2', 15)";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryCreate,
+        regexForExpectedExceptions);
+    assertRegionData(superUserClient, Arrays.asList(values));
+
+    String queryPut = "SELECT * FROM /" + regionName + ".put('key-2', 'something')";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryPut,
+        regexForExpectedExceptions);
+    assertRegionData(superUserClient, Arrays.asList(values));
+
+    String queryRemove = "SELECT * FROM /" + regionName + ".remove('key-0')";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryRemove,
+        regexForExpectedExceptions);
+    assertRegionData(superUserClient, Arrays.asList(values));
+
+    String queryDestroy = "SELECT * FROM /" + regionName + ".destroyRegion()";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, queryDestroy,
+        regexForExpectedExceptions);
+  }
+
+  /* ----- Other Forbidden Methods ----- */
+  @Test
+  public void queryWithGetClassShouldThrowSecurityException() {
+    String query1 = "SELECT * FROM /" + regionName + " r WHERE r.getClass != '1'";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query1,
+        regexForExpectedExceptions);
+
+    String query2 = "SELECT r.getClass FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query2,
+        regexForExpectedExceptions);
+
+    String query3 = "SELECT r.getClass() FROM /" + regionName + " r";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query3,
+        regexForExpectedExceptions);
+
+    String query4 = "SELECT * FROM /" + regionName + " r WHERE r.getClass != 'blah'";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query4,
+        regexForExpectedExceptions);
+
+    String query5 = "SELECT * FROM /" + regionName + " r WHERE r.getClass() != '1'";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query5,
+        regexForExpectedExceptions);
+
+    String query6 = "SELECT * FROM /" + regionName + " r WHERE r.Class != '1'";
+    executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(specificUserClient, query6,
+        regexForExpectedExceptions);
+  }
+
+  @Test
+  public void queryWithExplicitNonExistingMethodInvocationShouldReturnUndefined() {
+    Assume.assumeFalse(user.equals("dataReaderRegionKey"));
+    String query = "SELECT r.getInterestListRegex() FROM /" + regionName + " r";
+    List<Object> expectedResults = Arrays.asList(QueryService.UNDEFINED, QueryService.UNDEFINED);
+    executeQueryAndAssertExpectedResults(specificUserClient, query, expectedResults);
+  }
+
+  @Test
+  public void queryWithCloneMethodOnQRegionShouldReturnEmptyResult() {
+    Assume.assumeFalse(user.equals("dataReaderRegionKey"));
+    String query = "SELECT * FROM /" + regionName + ".clone";
+    List<Object> expectedResults = Collections.emptyList();
+    executeQueryAndAssertExpectedResults(specificUserClient, query, expectedResults);
+  }
+
+  @Test
+  public void queryWithExplicitNonExistingMethodInvocationOnQRegionShouldReturnEmptyResult() {
+    Assume.assumeFalse(user.equals("dataReaderRegionKey"));
+    String query = "SELECT * FROM /" + regionName + ".getKey('" + keys[0] + "')";
+    List<Object> expectedResults = Collections.emptyList();
+    executeQueryAndAssertExpectedResults(specificUserClient, query, expectedResults);
+  }
+
+  @Test
+  public void queryWithExplicitCreateMethodInvocationOnRegionShouldReturnUndefinedAndDoNotModifyRegion() {
+    Assume.assumeFalse(user.equals("dataReaderRegionKey"));
+    String query = "SELECT r.create('key2', 15) FROM /" + regionName + " r";
+    List<Object> expectedResults = Arrays.asList(QueryService.UNDEFINED, QueryService.UNDEFINED);
+    executeQueryAndAssertExpectedResults(specificUserClient, query, expectedResults);
+    assertRegionData(specificUserClient, Arrays.asList(values));
+  }
+
+  @Test
+  public void queryWithExplicitMutatorMethodInvocationsOnRegionShouldReturnEmptyResultAndDoNotModifyRegion() {
+    Assume.assumeFalse(user.equals("dataReaderRegionKey"));
+    String queryDestroy = "SELECT * FROM /" + regionName + ".destroyKey('" + keys[0] + "')";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryDestroy, Collections.emptyList());
+    assertRegionData(superUserClient, Arrays.asList(values));
+
+    String queryPutIfAbsent = "SELECT * FROM /" + regionName + ".putIfAbsent('key-2', 'something')";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryPutIfAbsent,
+        Collections.emptyList());
+    assertRegionData(superUserClient, Arrays.asList(values));
+
+    String queryReplace = "SELECT * FROM /" + regionName + ".replace('key-0', 'something')";
+    executeQueryAndAssertExpectedResults(specificUserClient, queryReplace, Collections.emptyList());
+    assertRegionData(superUserClient, Arrays.asList(values));
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityRestrictedQueriesDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityRestrictedQueriesDUnitTest.java
deleted file mode 100644
index 70265a6..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityRestrictedQueriesDUnitTest.java
+++ /dev/null
@@ -1,266 +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.security.query;
-
-import static org.junit.Assert.fail;
-
-import java.util.Arrays;
-
-import junitparams.Parameters;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.RegionDestroyedException;
-import org.apache.geode.cache.query.QueryService;
-import org.apache.geode.cache.query.TypeMismatchException;
-import org.apache.geode.security.query.data.QueryTestObject;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class QuerySecurityRestrictedQueriesDUnitTest extends QuerySecurityBase {
-
-  @Parameterized.Parameters
-  public static Object[] usersAllowed() {
-    return new Object[] {"dataReader", "dataReaderRegion", "clusterManagerDataReader",
-        "clusterManagerDataReaderRegion", "super-user"};
-  }
-
-  @Parameterized.Parameter
-  public String user;
-
-  @Before
-  public void configureCache() {
-    createClientCache(specificUserClient, user, userPerms.getUserPassword(user));
-    createProxyRegion(specificUserClient, regionName);
-
-    keys = new Object[] {"key-0", "key-1"};
-    values = new Object[] {new QueryTestObject(1, "John"), new QueryTestObject(3, "Beth")};
-    putIntoRegion(superUserClient, keys, values, regionName);
-  }
-
-  protected String regexForExpectedExceptions = ".*Unauthorized access.*";
-
-
-  /* ----- Implicit Getter Tests ----- */
-  @Test
-  public void checkUserAuthorizationsForSelectByImplicitGetterQuery() {
-    String query = "select * from /" + regionName + " r where r.name = 'Beth'";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectImplicitGetterQuery() {
-    String query = "select r.name from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectCountOfImplicitGetterQuery() {
-    String query = "select count(r.name) from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectMaxOfImplicitGetterQuery() {
-    String query = "select max(r.name) from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectMinOfImplicitGetterQuery() {
-    String query = "select min(r.name) from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectImplicitGetterFromRegionByImplicitGetterFromRegionQuery() {
-    String query = "select * from /" + regionName + " r1 where r1.name in (select r2.name from /"
-        + regionName + " r2)";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-  /* ----- Implicit Getter Tests ----- */
-
-  /* ----- Direct Getter Tests ----- */
-  @Test
-  public void checkUserAuthorizationsForSelectByDirectGetterQuery() {
-    String query = "select * from /" + regionName + " r where r.getName = 'Beth'";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectDirectGetterQuery() {
-    String query = "select r.getName() from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectCountOfDirectGetterQuery() {
-    String query = "select count(r.getId) from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectMaxOfDirectGetterQuery() {
-    String query = "select max(r.getId()) from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectMinOfDirectGetterQuery() {
-    String query = "select min(getId()) from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectDirectGetterFromRegionByDirectGetterFromRegionQuery() {
-    String query = "select * from /" + regionName
-        + " r1 where r1.getName in (select r2.getName from /" + regionName + " r2)";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectRegionContainsValueQuery() {
-    String query = "select * from /" + regionName + ".containsValue('value')";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void RegionMethodInvocationShouldThrowSecurityExceptionNotTypeMismatch() {
-    String query = "select * from /" + regionName + ".containsValue('value')";
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      try {
-        queryService.newQuery(query).execute();
-        fail();
-      } catch (Exception e) {
-        e.printStackTrace();
-        if (!e.getMessage().matches(regexForExpectedExceptions)) {
-          Throwable cause = e.getCause();
-          while (cause != null || cause instanceof TypeMismatchException) {
-            if (cause.getMessage().matches(regexForExpectedExceptions)) {
-              return;
-            }
-            cause = cause.getCause();
-          }
-          e.printStackTrace();
-          fail();
-        }
-      }
-    });
-  }
-
-  @Test
-  public void usersWhoCanExecuteQueryShouldNotInvokeRegionCreateForSelectRegionCreateQuery()
-      throws Exception {
-    String query = "select * from /" + regionName + ".create('key2', 15)";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-    executeAndConfirmRegionMatches(specificUserClient, regionName, Arrays.asList(values));
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectRegionDestroyRegionQuery() {
-    String query = "select * from /" + regionName + ".destroyRegion()";
-    try {
-      executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-          regexForExpectedExceptions);
-    } catch (Throwable throwable) {
-      if (!(throwable.getCause().getCause() instanceof RegionDestroyedException)) {
-        throw throwable;
-      }
-    }
-  }
-
-  @Test
-  public void usersWhoCanExecuteQueryShouldNotInvokeRegionPutForSelectRegionPutQuery()
-      throws Exception {
-    String query = "select * from /" + regionName + ".put('key-2', 'something')";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-    executeAndConfirmRegionMatches(specificUserClient, regionName, Arrays.asList(values));
-  }
-
-  @Test
-  @Parameters(method = "getAllUsersWhoCanExecuteQuery")
-  public void usersWhoCanExecuteQueryShouldNotInvokedRegionRemoveForSelectRegionRemoveQuery()
-      throws Exception {
-    String query = "select * from /" + regionName + ".remove('key-0')";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-    executeAndConfirmRegionMatches(specificUserClient, regionName, Arrays.asList(values));
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectByGetClassQuery() {
-    String query = "select * from /" + regionName + " r where r.getClass != '1'";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectGetClassRegionQuery() {
-    String query = "select r.getClass from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectGetClassWithParenthesisRegionQuery() {
-    String query = "select r.getClass() from /" + regionName + " r";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectByClassQuery() {
-    String query = "select * from /" + regionName + " r where r.getClass != 'blah'";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectByGetClassWithParenthesisQuery() {
-    String query = "select * from /" + regionName + " r where r.getClass() != '1'";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-
-
-  @Test
-  public void checkUserAuthorizationsForSelectByCapitalClassQuery() {
-    String query = "select * from /" + regionName + " r where r.Class != '1'";
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        regexForExpectedExceptions);
-  }
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityRetrictedButMethodsDoNotExistDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityRetrictedButMethodsDoNotExistDUnitTest.java
deleted file mode 100644
index 3b219ed..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityRetrictedButMethodsDoNotExistDUnitTest.java
+++ /dev/null
@@ -1,125 +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.security.query;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.query.QueryService;
-import org.apache.geode.security.query.data.QueryTestObject;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class QuerySecurityRetrictedButMethodsDoNotExistDUnitTest extends QuerySecurityBase {
-
-  @Parameterized.Parameters
-  public static Object[] usersAllowed() {
-    return new Object[] {"dataReader", "dataReaderRegion", "clusterManagerDataReader",
-        "clusterManagerDataReaderRegion", "super-user"};
-  }
-
-  @Parameterized.Parameter
-  public String user;
-
-  @Before
-  public void configureCache() {
-    createClientCache(specificUserClient, user, userPerms.getUserPassword(user));
-    createProxyRegion(specificUserClient, regionName);
-
-    keys = new Object[] {"key-0", "key-1"};
-    values = new Object[] {new QueryTestObject(1, "John"), new QueryTestObject(3, "Beth")};
-    putIntoRegion(superUserClient, keys, values, regionName);
-  }
-
-  @Test
-  public void executingMethodThatDoesNotExistOnResultsWillReturnUndefinedAsAResult() {
-    String query = "select r.getInterestListRegex() from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList(QueryService.UNDEFINED, QueryService.UNDEFINED);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void cloneExecutedOnQRegionWillReturnEmptyResults() {
-    String query = "select * from /" + regionName + ".clone";
-    List<Object> expectedResults = Arrays.asList();
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void executingMethodThatDoesNotExistOnQRegionWillReturnEmptyResult() {
-    String query = "select * from /" + regionName + ".getKey('" + keys[0] + "')";
-    List<Object> expectedResults = Arrays.asList();
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-  @Test
-  public void executingCreateOnRegionWillResultInUndefinedButNotModifyRegion() throws Exception {
-    String query = "select r.create('key2', 15) from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList(QueryService.UNDEFINED, QueryService.UNDEFINED);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-    executeAndConfirmRegionMatches(specificUserClient, regionName, Arrays.asList(values));
-  }
-
-  @Test
-  public void executingDestroyKeyOnRegionWillReturnEmptyResultsAndNotModifyRegion()
-      throws Exception {
-    String query = "select * from /" + regionName + ".destroyKey('" + keys[0] + "')";
-    List<Object> expectedResults = Arrays.asList();
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-    executeAndConfirmRegionMatches(specificUserClient, regionName, Arrays.asList(values));
-  }
-
-  @Test
-  public void executingPutIfAbsentOnRegionWillReturnEmptyResultsAndNotModifyRegion()
-      throws Exception {
-    String query = "select * from /" + regionName + ".putIfAbsent('key-2', 'something')";
-    List<Object> expectedResults = Arrays.asList();
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-    executeAndConfirmRegionMatches(specificUserClient, regionName, Arrays.asList(values));
-  }
-
-  @Test
-  public void executingReplaceOnRegionWillReturnEmptyResultsAndNotModifyRegion() throws Exception {
-    String query = "select * from /" + regionName + ".replace('key-0', 'something')";
-    List<Object> expectedResults = Arrays.asList();
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-    executeAndConfirmRegionMatches(specificUserClient, regionName, Arrays.asList(values));
-  }
-
-  @Test
-  public void checkUserAuthorizationsForSelectGetInterestListRegexParenRegionQuery() {
-    String query = "select r.getInterestListRegex() from /" + regionName + " r";
-    List<Object> expectedResults = Arrays.asList(QueryService.UNDEFINED, QueryService.UNDEFINED);
-    executeQueryWithCheckForAccessPermissions(specificUserClient, query, regionName,
-        expectedResults);
-  }
-
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityUnauthorizedUserBindParameterDUnitTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityUnauthorizedUserBindParameterDUnitTest.java
deleted file mode 100644
index 96415b1..0000000
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityUnauthorizedUserBindParameterDUnitTest.java
+++ /dev/null
@@ -1,72 +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.security.query;
-
-import java.util.HashSet;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.query.internal.QRegion;
-import org.apache.geode.cache.query.internal.index.DummyQRegion;
-import org.apache.geode.security.query.data.QueryTestObject;
-import org.apache.geode.test.junit.categories.SecurityTest;
-
-@Category({SecurityTest.class})
-public class QuerySecurityUnauthorizedUserBindParameterDUnitTest extends QuerySecurityBase {
-
-  @Before
-  public void configureCache() {
-    String user = "dataReaderRegionKey";
-    createClientCache(specificUserClient, user, userPerms.getUserPassword(user));
-    createProxyRegion(specificUserClient, regionName);
-
-    keys = new Object[] {"key-0", "key-1"};
-    values = new Object[] {new QueryTestObject(1, "John"), new QueryTestObject(3, "Beth")};
-    putIntoRegion(superUserClient, keys, values, regionName);
-  }
-
-  @Test
-  public void userWithoutRegionAccessAndPassingInWrappedBindParameterShouldThrowException() {
-    String query = "select v from $1 r, r.values() v";
-    String regexForExpectedException = ".*values.*";
-    specificUserClient.invoke(() -> {
-      Region region = getClientCache().getRegion(regionName);
-      HashSet<Region> hashSet = new HashSet<>();
-      hashSet.add(region);
-      assertExceptionOccurred(getClientCache().getQueryService(), query, new Object[] {hashSet},
-          regexForExpectedException);
-    });
-  }
-
-  // If DummyQRegion is ever serializable, then this test will fail and a security hole with query
-  // will have been opened
-  // That means a user could wrap a region in a dummy region and bypass the
-  // RestrictedMethodInvocationAuthorizer
-  @Test
-  public void userWithoutRegionAccessAndPassingInWrappedInDummyQRegionBindParameterShouldThrowSerializationException() {
-    String query = "select v from $1 r, r.values() v";
-    String regexForExpectedException = ".*failed serializing object.*";
-    specificUserClient.invoke(() -> {
-      Region region = getClientCache().getRegion(regionName);
-      HashSet<QRegion> hashset = new HashSet<>();
-      hashset.add(new DummyQRegion(region));
-      assertExceptionOccurred(getClientCache().getQueryService(), query, new Object[] {hashset},
-          regexForExpectedException);
-    });
-  }
-}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityWithBindParameterDistributedTest.java b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityWithBindParameterDistributedTest.java
new file mode 100644
index 0000000..fc57a1b
--- /dev/null
+++ b/geode-core/src/distributedTest/java/org/apache/geode/security/query/QuerySecurityWithBindParameterDistributedTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.security.query;
+
+import static org.apache.geode.cache.RegionShortcut.PARTITION;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.NotSerializableException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.internal.QRegion;
+import org.apache.geode.cache.query.internal.index.DummyQRegion;
+import org.apache.geode.security.NotAuthorizedException;
+import org.apache.geode.security.query.data.QueryTestObject;
+import org.apache.geode.test.junit.categories.OQLQueryTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+
+@RunWith(Parameterized.class)
+@Category({SecurityTest.class, OQLQueryTest.class})
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+public class QuerySecurityWithBindParameterDistributedTest
+    extends AbstractQuerySecurityDistributedTest {
+  @Parameterized.Parameters(name = "RegionType:{0}")
+  public static Object[] regionTypes() {
+    return new Object[] {
+        REPLICATE, PARTITION
+    };
+  }
+
+  @Parameterized.Parameter
+  public RegionShortcut regionShortcut;
+
+  private final String queryString = "SELECT v FROM $1 r, r.values() v";
+
+  private void assertExceptionOccurred(QueryService qs, Object[] bindParams,
+      String authErrorRegexp) {
+    assertThatThrownBy(() -> qs.newQuery(queryString).execute(bindParams))
+        .hasMessageMatching(authErrorRegexp)
+        .hasCauseInstanceOf(NotAuthorizedException.class);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUpSuperUserClientAndServer(regionShortcut);
+
+    keys = new Object[] {"key-0", "key-1"};
+    values = new Object[] {
+        new QueryTestObject(1, "John"),
+        new QueryTestObject(3, "Beth")
+    };
+
+    putIntoRegion(superUserClient, keys, values, regionName);
+  }
+
+  @Test
+  public void queryWithRegionAsBindParameterShouldNotThrowSecurityExceptionWhenUserHasTheCorrectPrivileges()
+      throws Exception {
+    setUpSpecificClient("dataReaderRegion");
+
+    specificUserClient.invoke(() -> {
+      Region region = getClientCache().getRegion(regionName);
+      HashSet<Region> hashSet = new HashSet<>(Collections.singletonList(region));
+      assertQueryResults(getClientCache(), queryString, new Object[] {hashSet},
+          Arrays.asList(values));
+    });
+  }
+
+  // If DummyQRegion is ever serializable, then this test will fail and a security hole with query
+  // will have been opened
+  // That means a user could wrap a region in a dummy region and bypass the
+  // RestrictedMethodInvocationAuthorizer
+  @Test
+  public void queryWithQRegionAsBindParameterShouldThrowSerializationException() throws Exception {
+    setUpSpecificClient("dataReaderRegionKey");
+    String regexForExpectedException = ".*failed serializing object.*";
+
+    specificUserClient.invoke(() -> {
+      Region region = getClientCache().getRegion(regionName);
+      HashSet<QRegion> hashSet = new HashSet<>(Collections.singletonList(new DummyQRegion(region)));
+
+      assertThatThrownBy(() -> getClientCache().getQueryService().newQuery(queryString)
+          .execute(new Object[] {hashSet}))
+              .hasMessageMatching(regexForExpectedException)
+              .hasCauseInstanceOf(NotSerializableException.class);
+    });
+  }
+
+  @Test
+  public void queryWithRegionAsBindParameterShouldThrowSecurityExceptionWhenUserDoesNotHaveTheCorrectPrivileges()
+      throws Exception {
+    setUpSpecificClient("dataReaderRegionKey");
+    String regexForExpectedException = ".*values.*";
+
+    specificUserClient.invoke(() -> {
+      Region region = getClientCache().getRegion(regionName);
+      HashSet<Region> hashSet = new HashSet<>(Collections.singletonList(region));
+      assertExceptionOccurred(getClientCache().getQueryService(), new Object[] {hashSet},
+          regexForExpectedException);
+    });
+  }
+}
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityAuthorizedUserDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityAuthorizedUserDUnitTest.java
deleted file mode 100644
index 7586ad2..0000000
--- a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityAuthorizedUserDUnitTest.java
+++ /dev/null
@@ -1,261 +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.query.cq.dunit;
-
-import static org.apache.geode.internal.Assert.assertTrue;
-import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.query.CqAttributesFactory;
-import org.apache.geode.cache.query.CqEvent;
-import org.apache.geode.cache.query.CqException;
-import org.apache.geode.cache.query.CqListener;
-import org.apache.geode.cache.query.CqQuery;
-import org.apache.geode.cache.query.QueryService;
-import org.apache.geode.cache.query.RegionNotFoundException;
-import org.apache.geode.security.query.QuerySecurityBase;
-import org.apache.geode.security.query.data.QueryTestObject;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class CqSecurityAuthorizedUserDUnitTest extends QuerySecurityBase {
-
-  @Parameterized.Parameters
-  public static Object[] usersAllowed() {
-    return new Object[] {"dataReader", "dataReaderRegion", "clusterManagerDataReader",
-        "clusterManagerDataReaderRegion", "super-user"};
-  }
-
-  @Parameterized.Parameter
-  public String user;
-
-  @Before
-  public void configureSpecificClientAndKeyAndValues() {
-    createClientCache(specificUserClient, user, userPerms.getUserPassword(user));
-    createProxyRegion(specificUserClient, regionName);
-
-    keys = new Object[] {"key-0"};
-    values = new Object[] {new QueryTestObject(0, "John")};
-  }
-
-  // Variables that need to be shared across invoke calls.
-  protected static CqSecurityTestCqListener cqListener = null;
-
-  private String regexForExpectedExceptions = ".*Unauthorized access.*";
-
-  @Test
-  public void cqExecuteNoMethodInvocationWithUsersWithCqPermissionsWithPrepopulatedRegionShouldBeAllowed()
-      throws Exception {
-    putIntoRegion(superUserClient, keys, values, regionName);
-    String query = "select * from /" + regionName + " r where r.id = 0";
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityAuthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      cq.execute();
-    });
-
-    putIntoRegion(superUserClient, keys, new Object[] {new QueryTestObject(0, "Bethany")},
-        regionName);
-
-    specificUserClient.invoke(() -> {
-      await()
-          .untilAsserted(() -> assertEquals(1, cqListener.getNumEvent()));
-    });
-  }
-
-  @Test
-  public void cqExecuteWithMethodInvocationWithUsersWithCqPermissionsWithPrepopulatedRegionIsGettingExceptionInReplicatedRegion()
-      throws Exception {
-    putIntoRegion(superUserClient, keys, values, regionName);
-    String query = "select * from /" + regionName + " r where r.name = 'Beth'";
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityAuthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      executeCqButExpectException(cq, user, regexForExpectedExceptions);
-    });
-  }
-
-  @Test
-  public void cqExecuteWithInitialResultsWithMethodInvocationWithUsersWithCqPermissionsWithPrepopulatedRegionShouldBeDeniedBecauseOfInvocation()
-      throws Exception {
-    putIntoRegion(superUserClient, keys, values, regionName);
-    String query = "select * from /" + regionName + " r where r.name = 'Beth'";
-
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityAuthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      executeCqWithInitialResultsButExpectException(cq, user, regexForExpectedExceptions);
-    });
-  }
-
-
-  @Test
-  public void cqExecuteWithInitialResultsWithMethodInvocationWithUnpopulatedRegionAndFollowedByAPutShouldTriggerCqError()
-      throws Exception {
-    String query = "select * from /" + regionName + " r where r.name = 'Beth'";
-
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityAuthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      cq.executeWithInitialResults();
-    });
-
-    Object[] keys = {"key-0"};
-    Object[] values = {new QueryTestObject(1, "Mary")};
-    putIntoRegion(superUserClient, keys, values, regionName);
-
-    specificUserClient.invoke(() -> {
-      await()
-          .untilAsserted(() -> assertEquals(1, cqListener.getNumErrors()));
-    });
-  }
-
-  @Test
-  public void cqExecuteWithMethodInvocationWithUnpopulatedRegionAndFollowedByAPutShouldTriggerCqError()
-      throws Exception {
-    String query = "select * from /" + regionName + " r where r.name = 'Beth'";;
-
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityAuthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      cq.execute();
-    });
-
-    Object[] keys = {"key-0"};
-    Object[] values = {new QueryTestObject(1, "Mary")};
-    putIntoRegion(superUserClient, keys, values, regionName);
-
-    specificUserClient.invoke(() -> {
-      await()
-          .untilAsserted(() -> assertEquals(1, cqListener.getNumErrors()));
-    });
-  }
-
-  @Test
-  public void cqCanBeClosedByTheCreator() throws Exception {
-    String query = "select * from /" + regionName + " r where r.id = 0";
-
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityAuthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      cq.execute();
-      cq.close();
-      assertTrue(cq.isClosed());
-    });
-    assertEquals(0, server.getCache().getCqService().getAllCqs().size());
-  }
-
-
-  protected CqQuery createCq(QueryService queryService, String query, CqListener cqListener)
-      throws CqException {
-    CqAttributesFactory cqaf = new CqAttributesFactory();
-    cqaf.addCqListener(cqListener);
-    CqQuery cq = queryService.newCq(query, cqaf.create());
-    return cq;
-  }
-
-  protected void executeCqButExpectException(CqQuery cq, String user,
-      String regexForExpectedException) {
-    try {
-      cq.execute();
-      fail("Expected an exception when executing cq:" + cq.getQueryString() + " with user:" + user);
-    } catch (RegionNotFoundException | CqException e) {
-      if (!e.getMessage().matches(regexForExpectedException)) {
-        Throwable cause = e.getCause();
-        while (cause != null) {
-          if (cause.getMessage().matches(regexForExpectedException)) {
-            return;
-          }
-          cause = cause.getCause();
-        }
-        e.printStackTrace();
-        fail("Exception thrown did not match:" + regexForExpectedException + ".  Instead was:" + e);
-      }
-    }
-  }
-
-  private void executeCqWithInitialResultsButExpectException(CqQuery cq, String user,
-      String regexForExpectedException) {
-    try {
-      cq.executeWithInitialResults();
-      fail("Expected an exception when executing cq:" + cq + " with user:" + user);
-    } catch (RegionNotFoundException | CqException e) {
-      e.printStackTrace();
-      if (!e.getMessage().matches(regexForExpectedException)) {
-        Throwable cause = e.getCause();
-        while (cause != null) {
-          if (cause.getMessage() != null && cause.getMessage().matches(regexForExpectedException)) {
-            return;
-          }
-          cause = cause.getCause();
-        }
-        e.printStackTrace();
-        fail("Exception thrown did not match:" + regexForExpectedException + ".  Instead was:" + e);
-      }
-    }
-
-  }
-
-  public class CqSecurityTestCqListener implements CqListener {
-
-    private int numEvents = 0;
-    private int numErrors = 0;
-
-    @Override
-    public void onEvent(CqEvent aCqEvent) {
-      numEvents++;
-    }
-
-    @Override
-    public void onError(CqEvent aCqEvent) {
-      numErrors++;
-    }
-
-    public int getNumEvent() {
-      return numEvents;
-    }
-
-    public int getNumErrors() {
-      return numErrors;
-    }
-
-    @Override
-    public void close() {
-
-    }
-  }
-}
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityPartitionedAuthorizedUserDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityPartitionedAuthorizedUserDUnitTest.java
deleted file mode 100644
index af1d2b3..0000000
--- a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityPartitionedAuthorizedUserDUnitTest.java
+++ /dev/null
@@ -1,64 +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.query.cq.dunit;
-
-import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.cache.query.CqQuery;
-import org.apache.geode.cache.query.QueryService;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class CqSecurityPartitionedAuthorizedUserDUnitTest
-    extends CqSecurityAuthorizedUserDUnitTest {
-  @Override
-  public RegionShortcut getRegionType() {
-    return RegionShortcut.PARTITION;
-  }
-
-  @Override
-  @Test
-  public void cqExecuteWithMethodInvocationWithUsersWithCqPermissionsWithPrepopulatedRegionIsGettingExceptionInReplicatedRegion()
-      throws Exception {
-    putIntoRegion(superUserClient, keys, values, regionName);
-
-    String query = "select * from /" + regionName + " r where r.name = 'Beth'";
-
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityAuthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      cq.execute();
-    });
-
-    putIntoRegion(superUserClient, keys, values, regionName);
-
-    specificUserClient.invoke(() -> {
-      await()
-          .untilAsserted(() -> assertEquals(1, cqListener.getNumErrors()));
-    });
-  }
-}
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityPartitionedUnauthorizedUserDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityPartitionedUnauthorizedUserDUnitTest.java
deleted file mode 100644
index 2853725..0000000
--- a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityPartitionedUnauthorizedUserDUnitTest.java
+++ /dev/null
@@ -1,37 +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.query.cq.dunit;
-
-
-
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class CqSecurityPartitionedUnauthorizedUserDUnitTest
-    extends CqSecurityUnauthorizedUserDUnitTest {
-  @Override
-  public RegionShortcut getRegionType() {
-    return RegionShortcut.PARTITION;
-  }
-
-}
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityUnauthorizedUserDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityUnauthorizedUserDUnitTest.java
deleted file mode 100644
index aa3edf0..0000000
--- a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/dunit/CqSecurityUnauthorizedUserDUnitTest.java
+++ /dev/null
@@ -1,213 +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.query.cq.dunit;
-
-import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.junit.Assume.assumeTrue;
-
-import java.util.Arrays;
-import java.util.List;
-
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.query.CqAttributesFactory;
-import org.apache.geode.cache.query.CqEvent;
-import org.apache.geode.cache.query.CqException;
-import org.apache.geode.cache.query.CqListener;
-import org.apache.geode.cache.query.CqQuery;
-import org.apache.geode.cache.query.QueryService;
-import org.apache.geode.cache.query.RegionNotFoundException;
-import org.apache.geode.security.query.QuerySecurityBase;
-import org.apache.geode.security.query.data.QueryTestObject;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class CqSecurityUnauthorizedUserDUnitTest extends QuerySecurityBase {
-
-  @Parameterized.Parameters
-  public static Object[] usersAllowed() {
-    return new Object[] {"stranger", "dataReaderRegionKey", "clusterManagerQuery",
-        "clusterManagerDataReaderRegionKey", "dataWriter"};
-  }
-
-  @Parameterized.Parameter
-  public String user;
-
-  @Before
-  public void configureSpecificClientAndKeyAndValues() {
-    createClientCache(specificUserClient, user, userPerms.getUserPassword(user));
-    createProxyRegion(specificUserClient, regionName);
-
-    keys = new Object[] {"key-0"};
-    values = new Object[] {new QueryTestObject(0, "John")};
-  }
-
-  // Variables that need to be shared across invoke calls.
-  protected static CqSecurityTestCqListener cqListener = null;
-
-  public List<String> getAllUsersOnlyAllowedWrite() {
-    return Arrays.asList("dataWriter");
-  }
-
-
-  private String regexForExpectedExceptions = ".*DATA:READ:.*";
-
-  @Test
-  public void cqExecuteNoMethodInvocationWithUsersWithoutCqPermissionsWithPrepopulatedRegionShouldThrowSecurityException()
-      throws Exception {
-    putIntoRegion(superUserClient, keys, values, regionName);
-    String query = "select * from /" + regionName + " r where r.id = 0";
-
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityUnauthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      executeCqButExpectException(cq, user, regexForExpectedExceptions);
-    });
-  }
-
-  @Test
-  public void cqExecuteWithInitialResultsWithMethodInvocationWithoutPermissionWithUnpopulatedRegionThrowSecurityException()
-      throws Exception {
-    String query = "select * from /" + regionName + " r where r.name = 'Beth'";
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityUnauthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      executeCqWithInitialResultsButExpectException(cq, user, regexForExpectedExceptions);
-    });
-  }
-
-  @Test
-  public void cqExecuteWithOutPermissionsWithUnpopulatedRegionShouldNotAllowCq() throws Exception {
-    String query = "select * from /" + regionName + " r where r.name = 'Beth'";
-    specificUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityUnauthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      executeCqButExpectException(cq, user, regexForExpectedExceptions);
-    });
-  }
-
-  @Test
-  public void cqCreatedByAllowedUserButPutDoneByUnallowedReaderShouldStillExecuteWithCqEvent()
-      throws Exception {
-    assumeTrue(user.equals("dataWriter"));
-    String query = "select * from /" + regionName + " r where r.id = 1";
-    superUserClient.invoke(() -> {
-      QueryService queryService = getClientCache().getQueryService();
-      CqSecurityTestCqListener cqListener = new CqSecurityTestCqListener();
-      CqSecurityUnauthorizedUserDUnitTest.cqListener = cqListener;
-      CqQuery cq = createCq(queryService, query, cqListener);
-      cq.execute();
-    });
-
-    Object[] keys = {"key-0"};
-    Object[] values = {new QueryTestObject(1, "Mary")};
-    putIntoRegion(specificUserClient, keys, values, regionName);
-
-    superUserClient.invoke(() -> {
-      await()
-          .untilAsserted(() -> assertEquals(1, cqListener.getNumEvent()));
-    });
-  }
-
-
-  protected CqQuery createCq(QueryService queryService, String query, CqListener cqListener)
-      throws CqException {
-    CqAttributesFactory cqaf = new CqAttributesFactory();
-    cqaf.addCqListener(cqListener);
-    CqQuery cq = queryService.newCq(query, cqaf.create());
-    return cq;
-  }
-
-  protected void executeCqButExpectException(CqQuery cq, String user,
-      String regexForExpectedException) {
-    try {
-      cq.execute();
-      fail("Expected an exception when executing cq:" + cq.getQueryString() + " with user:" + user);
-    } catch (RegionNotFoundException | CqException e) {
-      if (!e.getMessage().matches(regexForExpectedException)) {
-        Throwable cause = e.getCause();
-        while (cause != null) {
-          if (cause.getMessage().matches(regexForExpectedException)) {
-            return;
-          }
-          cause = cause.getCause();
-        }
-        e.printStackTrace();
-        fail("Exception thrown did not match:" + regexForExpectedException + ".  Instead was:" + e);
-      }
-    }
-  }
-
-  private void executeCqWithInitialResultsButExpectException(CqQuery cq, String user,
-      String regexForExpectedException) {
-    try {
-      cq.executeWithInitialResults();
-      fail("Expected an exception when executing cq:" + cq + " with user:" + user);
-    } catch (RegionNotFoundException | CqException e) {
-      e.printStackTrace();
-      if (!e.getMessage().matches(regexForExpectedException)) {
-        Throwable cause = e.getCause();
-        while (cause != null) {
-          if (cause.getMessage() != null && cause.getMessage().matches(regexForExpectedException)) {
-            return;
-          }
-          cause = cause.getCause();
-        }
-        e.printStackTrace();
-        fail("Exception thrown did not match:" + regexForExpectedException + ".  Instead was:" + e);
-      }
-    }
-  }
-
-  public class CqSecurityTestCqListener implements CqListener {
-
-    private int numEvents = 0;
-
-    @Override
-    public void onEvent(CqEvent aCqEvent) {
-      numEvents++;
-    }
-
-    @Override
-    public void onError(CqEvent aCqEvent) {
-
-    }
-
-    public int getNumEvent() {
-      return numEvents;
-    }
-
-
-    @Override
-    public void close() {
-
-    }
-  }
-}
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/internal/CqSecurityExecutionContextTamperingDistributedTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/internal/CqSecurityExecutionContextTamperingDistributedTest.java
new file mode 100644
index 0000000..a3012cf
--- /dev/null
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/cq/internal/CqSecurityExecutionContextTamperingDistributedTest.java
@@ -0,0 +1,137 @@
+/*
+ * 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.query.cq.internal;
+
+import static org.apache.geode.cache.RegionShortcut.PARTITION;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.io.Serializable;
+
+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.junit.runners.Parameterized;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.cache.query.CqAttributesFactory;
+import org.apache.geode.cache.query.CqQuery;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.internal.ExecutionContext;
+import org.apache.geode.cache.query.internal.ExecutionContextTamperer;
+import org.apache.geode.examples.SimpleSecurityManager;
+import org.apache.geode.internal.cache.InternalCache;
+import org.apache.geode.security.query.TestCqListener;
+import org.apache.geode.security.query.data.QueryTestObject;
+import org.apache.geode.test.dunit.rules.ClientVM;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+import org.apache.geode.test.junit.categories.OQLQueryTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+
+/**
+ * Verifies that users can tamper the {@link ExecutionContext}.
+ * It needs to be part of the {@link org.apache.geode.cache.query.cq.internal} package because the
+ * method {@link CqQueryImpl#getQueryExecutionContext()} has package access.
+ */
+@RunWith(Parameterized.class)
+@Category({SecurityTest.class, OQLQueryTest.class})
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+public class CqSecurityExecutionContextTamperingDistributedTest implements Serializable {
+  private MemberVM server;
+  private ClientVM client;
+  protected final String regionName = "region";
+  private static TestCqListener cqListener = null;
+
+  @Rule
+  public ClusterStartupRule cluster = new ClusterStartupRule();
+
+  @Parameterized.Parameters(name = "RegionType:{1}, Accessor:{0}")
+  public static Object[] regionTypes() {
+    return new Object[][] {
+        {"name", REPLICATE}, {"name", PARTITION},
+        {"getName", REPLICATE}, {"getName", PARTITION},
+    };
+  }
+
+  @Parameterized.Parameter
+  public String attributeAccessor;
+
+  @Parameterized.Parameter(1)
+  public RegionShortcut regionShortcut;
+
+  @Before
+  public void setUp() throws Exception {
+    server = cluster.startServerVM(1, cf -> cf
+        .withSecurityManager(SimpleSecurityManager.class)
+        .withProperty(SERIALIZABLE_OBJECT_FILTER, "org.apache.geode.security.query.data.*")
+        .withRegion(regionShortcut, regionName));
+
+    client = cluster
+        .startClientVM(2, ccf -> ccf.withCredential("dataRead", "dataRead")
+            .withPoolSubscription(true)
+            .withServerConnection(server.getPort())
+            .withProperty(SERIALIZABLE_OBJECT_FILTER, "org.apache.geode.security.query.data.*"));
+
+    client.invoke(() -> {
+      assertThat(ClusterStartupRule.getClientCache()).isNotNull();
+      ClusterStartupRule.getClientCache().createClientRegionFactory(ClientRegionShortcut.PROXY)
+          .create(regionName);
+    });
+  }
+
+  @Test
+  public void executionContextShouldNotBeModifiableForCqQueriesWithMethodInvocations() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r." + attributeAccessor + " = 'Beth'";
+
+    client.invoke(() -> {
+      TestCqListener cqListener = new TestCqListener();
+      assertThat(ClusterStartupRule.getClientCache()).isNotNull();
+      QueryService queryService = ClusterStartupRule.getClientCache().getQueryService();
+      CqSecurityExecutionContextTamperingDistributedTest.cqListener = cqListener;
+      CqAttributesFactory cqAttributesFactory = new CqAttributesFactory();
+      cqAttributesFactory.addCqListener(cqListener);
+
+      CqQuery cq = queryService.newCq(query, cqAttributesFactory.create());
+      cq.execute();
+    });
+
+    server.invoke(() -> {
+      assertThat(ClusterStartupRule.getCache()).isNotNull();
+      InternalCache internalCache = ClusterStartupRule.getCache();
+      assertThat(internalCache.getCqService().getAllCqs().size()).isEqualTo(1);
+      CqQueryImpl cqQueryImpl =
+          (CqQueryImpl) internalCache.getCqService().getAllCqs().iterator().next();
+      ExecutionContextTamperer.tamperContextCache(cqQueryImpl.getQueryExecutionContext(),
+          "org.apache.geode.security.query.data.QueryTestObject.getName", true);
+
+      Region<String, QueryTestObject> region = ClusterStartupRule.getCache().getRegion(regionName);
+      region.put("1", new QueryTestObject(1, "Beth"));
+    });
+
+    client.invoke(() -> {
+      assertThat(CqSecurityExecutionContextTamperingDistributedTest.cqListener.getNumEvents())
+          .isEqualTo(0);
+      assertThat(CqSecurityExecutionContextTamperingDistributedTest.cqListener.getNumErrors())
+          .isEqualTo(1);
+    });
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedQuerySecurityAllowedQueriesDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/internal/ExecutionContextTamperer.java
similarity index 52%
rename from geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedQuerySecurityAllowedQueriesDUnitTest.java
rename to geode-cq/src/distributedTest/java/org/apache/geode/cache/query/internal/ExecutionContextTamperer.java
index 4ee3297..fcb0a9a 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedQuerySecurityAllowedQueriesDUnitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/cache/query/internal/ExecutionContextTamperer.java
@@ -12,25 +12,16 @@
  * or implied. See the License for the specific language governing permissions and limitations under
  * the License.
  */
-package org.apache.geode.security.query;
+package org.apache.geode.cache.query.internal;
 
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
-
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class PartitionedQuerySecurityAllowedQueriesDUnitTest
-    extends QuerySecurityAllowedQueriesDUnitTest {
+/**
+ * Test class to manually try to modify the ExecutionContext cache after a CQ has been created.
+ * It needs to be part of the {@link org.apache.geode.cache.query.internal} package because the
+ * method {@link ExecutionContext#cachePut(Object, Object)} has package access.
+ */
+public class ExecutionContextTamperer {
 
-  @Override
-  public RegionShortcut getRegionType() {
-    return RegionShortcut.PARTITION;
+  public static void tamperContextCache(ExecutionContext context, Object key, Object value) {
+    context.cachePut(key, value);
   }
-
 }
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/security/query/CqSecurityAllowedUsersDistributedTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/security/query/CqSecurityAllowedUsersDistributedTest.java
new file mode 100644
index 0000000..75cc143
--- /dev/null
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/security/query/CqSecurityAllowedUsersDistributedTest.java
@@ -0,0 +1,235 @@
+/*
+ * 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.security.query;
+
+import static org.apache.geode.cache.RegionShortcut.PARTITION;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.query.CqAttributesFactory;
+import org.apache.geode.cache.query.CqException;
+import org.apache.geode.cache.query.CqQuery;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.RegionNotFoundException;
+import org.apache.geode.security.query.data.QueryTestObject;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.junit.categories.OQLQueryTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+
+@RunWith(Parameterized.class)
+@Category({SecurityTest.class, OQLQueryTest.class})
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+public class CqSecurityAllowedUsersDistributedTest extends AbstractQuerySecurityDistributedTest {
+
+  @Parameterized.Parameters(name = "User:{0}, RegionType:{1}")
+  public static Object[] usersAndRegionTypes() {
+    return new Object[][] {
+        {"super-user", REPLICATE}, {"super-user", PARTITION},
+        {"dataReader", REPLICATE}, {"dataReader", PARTITION},
+        {"dataReaderRegion", REPLICATE}, {"dataReaderRegion", PARTITION},
+        {"clusterManagerDataReader", REPLICATE}, {"clusterManagerDataReader", PARTITION},
+        {"clusterManagerDataReaderRegion", REPLICATE}, {"clusterManagerDataReaderRegion", PARTITION}
+    };
+  }
+
+  @Parameterized.Parameter
+  public String user;
+
+  @Parameterized.Parameter(1)
+  public RegionShortcut regionShortcut;
+
+  // Variables that need to be shared across invoke calls.
+  private static TestCqListener cqListener = null;
+  private String regexForExpectedExceptions = ".*Unauthorized access.*";
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp(user, regionShortcut);
+
+    keys = new Object[] {"key-0"};
+    values = new Object[] {
+        new QueryTestObject(0, "John")
+    };
+  }
+
+  private CqQuery createCq(String query) throws CqException {
+    TestCqListener cqListener = new TestCqListener();
+    QueryService queryService = getClientCache().getQueryService();
+    CqSecurityAllowedUsersDistributedTest.cqListener = cqListener;
+    CqAttributesFactory cqAttributesFactory = new CqAttributesFactory();
+    cqAttributesFactory.addCqListener(cqListener);
+
+    return queryService.newCq(query, cqAttributesFactory.create());
+  }
+
+  private void executeCqAndAssertExceptionThrown(String query) {
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+
+      assertThatThrownBy(cq::execute)
+          .as("Expected an exception when executing cq " + cq.getQueryString() + " with user "
+              + user)
+          .isInstanceOfAny(RegionNotFoundException.class, CqException.class)
+          .hasMessageMatching(regexForExpectedExceptions);
+    });
+  }
+
+  private void executeCqWithInitialResultsAndAssertExceptionThrown(String query) {
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+
+      assertThatThrownBy(cq::executeWithInitialResults)
+          .as("Expected an exception when executing cq " + cq.getQueryString() + " with user "
+              + user)
+          .isInstanceOfAny(RegionNotFoundException.class, CqException.class)
+          .hasMessageMatching(regexForExpectedExceptions);
+    });
+  }
+
+  private void executeCqAndAssertThatOnErrorIsInvokedOnNextMatchingEvent(String query) {
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+      cq.execute();
+    });
+
+    putIntoRegion(superUserClient, keys, values, regionName);
+    specificUserClient.invoke(
+        () -> await().untilAsserted(() -> assertThat(cqListener.getNumErrors()).isEqualTo(1)));
+  }
+
+  private void executeCqWithInitialResultsAndAssertThatOnErrorIsInvokedOnNextMatchingEvent(
+      String query) {
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+      cq.executeWithInitialResults();
+    });
+
+    Object[] keys = {"key-0"};
+    Object[] values = {new QueryTestObject(1, "Mary")};
+    putIntoRegion(superUserClient, keys, values, regionName);
+    specificUserClient.invoke(
+        () -> await().untilAsserted(() -> assertThat(cqListener.getNumErrors()).isEqualTo(1)));
+  }
+
+  @Test
+  public void cqQueryWithPublicFieldOnNonEmptyRegionShouldNotThrowException() {
+    putIntoRegion(superUserClient, keys, values, regionName);
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.id = 0";
+
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+      cq.execute();
+    });
+
+    Object[] newValues = new Object[] {new QueryTestObject(0, "Bethany")};
+    putIntoRegion(superUserClient, keys, newValues, regionName);
+    specificUserClient.invoke(
+        () -> await().untilAsserted(() -> assertThat(cqListener.getNumEvents()).isEqualTo(1)));
+  }
+
+  @Test
+  public void cqQueryWithImplicitMethodInvocationOnEmptyRegionShouldNotThrowExceptionDuringExecuteWithInitialResultsAndInvokeOnErrorForNextMatchingEvent() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.name = 'Beth'";
+
+    executeCqWithInitialResultsAndAssertThatOnErrorIsInvokedOnNextMatchingEvent(query);
+  }
+
+  @Test
+  public void cqQueryWithExplicitMethodInvocationOnEmptyRegionShouldNotThrowExceptionDuringExecuteWithInitialResultsAndInvokeOnErrorForNextMatchingEvent() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.getName = 'Beth'";
+
+    executeCqWithInitialResultsAndAssertThatOnErrorIsInvokedOnNextMatchingEvent(query);
+  }
+
+  @Test
+  public void cqQueryWithImplicitMethodInvocationOnNonEmptyReplicateRegionShouldThrowExceptionDuringExecute() {
+    Assume.assumeTrue(regionShortcut.equals(REPLICATE));
+    putIntoRegion(superUserClient, keys, values, regionName);
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.name = 'Beth'";
+
+    executeCqAndAssertExceptionThrown(query);
+  }
+
+  @Test
+  public void cqQueryWithExplicitMethodInvocationOnNonEmptyReplicateRegionShouldThrowExceptionDuringExecute() {
+    Assume.assumeTrue(regionShortcut.equals(REPLICATE));
+    putIntoRegion(superUserClient, keys, values, regionName);
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.getName = 'Beth'";
+
+    executeCqAndAssertExceptionThrown(query);
+  }
+
+  @Test
+  public void cqQueryWithImplicitMethodInvocationOnNonEmptyPartitionRegionShouldNotThrowExceptionDuringExecuteAndInvokeOnErrorForNextMatchingEvent() {
+    Assume.assumeTrue(regionShortcut.equals(PARTITION));
+    putIntoRegion(superUserClient, keys, values, regionName);
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.name = 'Beth'";
+
+    executeCqAndAssertThatOnErrorIsInvokedOnNextMatchingEvent(query);
+  }
+
+  @Test
+  public void cqQueryWithExplicitMethodInvocationOnNonEmptyPartitionRegionShouldNotThrowExceptionDuringExecuteAndInvokeOnErrorForNextMatchingEvent() {
+    Assume.assumeTrue(regionShortcut.equals(PARTITION));
+    putIntoRegion(superUserClient, keys, values, regionName);
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.getName = 'Beth'";
+
+    executeCqAndAssertThatOnErrorIsInvokedOnNextMatchingEvent(query);
+  }
+
+  @Test
+  public void cqQueryWithImplicitMethodInvocationOnNonEmptyRegionShouldThrowExceptionDuringExecuteWithInitialResults() {
+    putIntoRegion(superUserClient, keys, values, regionName);
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.name = 'Beth'";
+
+    executeCqWithInitialResultsAndAssertExceptionThrown(query);
+  }
+
+  @Test
+  public void cqQueryWithExplicitMethodInvocationOnNonEmptyRegionShouldThrowExceptionDuringExecuteWithInitialResults() {
+    putIntoRegion(superUserClient, keys, values, regionName);
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.getName = 'Beth'";
+
+    executeCqWithInitialResultsAndAssertExceptionThrown(query);
+  }
+
+  @Test
+  public void cqCanBeClosedByTheCreator() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.id = 0";
+
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+      cq.execute();
+      cq.close();
+      assertThat(cq.isClosed()).isTrue();
+    });
+
+    server.invoke(() -> {
+      assertThat(ClusterStartupRule.getCache()).isNotNull();
+      assertThat(ClusterStartupRule.getCache().getCqService().getAllCqs().size()).isEqualTo(0);
+    });
+  }
+}
diff --git a/geode-cq/src/distributedTest/java/org/apache/geode/security/query/CqSecurityForbiddenUsersDistributedTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/security/query/CqSecurityForbiddenUsersDistributedTest.java
new file mode 100644
index 0000000..269116d
--- /dev/null
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/security/query/CqSecurityForbiddenUsersDistributedTest.java
@@ -0,0 +1,172 @@
+/*
+ * 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.security.query;
+
+import static org.apache.geode.cache.RegionShortcut.PARTITION;
+import static org.apache.geode.cache.RegionShortcut.REPLICATE;
+import static org.apache.geode.test.awaitility.GeodeAwaitility.await;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.junit.Assume.assumeTrue;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.query.CqAttributesFactory;
+import org.apache.geode.cache.query.CqException;
+import org.apache.geode.cache.query.CqQuery;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.RegionNotFoundException;
+import org.apache.geode.security.query.data.QueryTestObject;
+import org.apache.geode.test.junit.categories.OQLQueryTest;
+import org.apache.geode.test.junit.categories.SecurityTest;
+import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+
+@RunWith(Parameterized.class)
+@Category({SecurityTest.class, OQLQueryTest.class})
+@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
+public class CqSecurityForbiddenUsersDistributedTest extends AbstractQuerySecurityDistributedTest {
+
+  @Parameterized.Parameters(name = "User:{0}, RegionType:{1}")
+  public static Object[] usersAndRegionTypes() {
+    return new Object[][] {
+        {"stranger", REPLICATE}, {"stranger", PARTITION},
+        {"dataWriter", REPLICATE}, {"dataWriter", PARTITION},
+        {"dataReaderRegionKey", REPLICATE}, {"dataReaderRegionKey", PARTITION},
+        {"clusterManagerQuery", REPLICATE}, {"clusterManagerQuery", PARTITION},
+        {"clusterManagerDataReaderRegionKey", REPLICATE},
+        {"clusterManagerDataReaderRegionKey", PARTITION}
+    };
+  }
+
+  @Parameterized.Parameter
+  public String user;
+
+  @Parameterized.Parameter(1)
+  public RegionShortcut regionShortcut;
+
+  // Variables that need to be shared across invoke calls.
+  private static TestCqListener cqListener = null;
+  private String regexForExpectedExceptions = ".*DATA:READ:.*";
+
+  @Before
+  public void setUp() throws Exception {
+    super.setUp(user, regionShortcut);
+
+    keys = new Object[] {"key-0"};
+    values = new Object[] {new QueryTestObject(0, "John")};
+
+    putIntoRegion(superUserClient, keys, values, regionName);
+  }
+
+  private CqQuery createCq(String query) throws CqException {
+    TestCqListener cqListener = new TestCqListener();
+    QueryService queryService = getClientCache().getQueryService();
+    CqSecurityForbiddenUsersDistributedTest.cqListener = cqListener;
+    CqAttributesFactory cqAttributesFactory = new CqAttributesFactory();
+    cqAttributesFactory.addCqListener(cqListener);
+
+    return queryService.newCq(query, cqAttributesFactory.create());
+  }
+
+  private void executeCqAndAssertException(CqQuery cq, String user,
+      String regexForExpectedException) {
+    assertThatThrownBy(cq::execute)
+        .as("Expected an exception when executing cq " + cq.getQueryString() + " with user " + user)
+        .isInstanceOfAny(RegionNotFoundException.class, CqException.class)
+        .hasMessageMatching(regexForExpectedException);
+  }
+
+  private void executeCqWithInitialResultsAndAssertException(CqQuery cq, String user,
+      String regexForExpectedException) {
+    assertThatThrownBy(cq::executeWithInitialResults)
+        .as("Expected an exception when executing cq " + cq.getQueryString() + " with user " + user)
+        .isInstanceOfAny(RegionNotFoundException.class, CqException.class)
+        .hasMessageMatching(regexForExpectedException);
+  }
+
+  @Test
+  public void cqQueryWithPublicFieldOnNonEmptyRegionShouldThrowExceptionDuringExecute() {
+    putIntoRegion(superUserClient, keys, values, regionName);
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.id = 0";
+
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+      executeCqAndAssertException(cq, user, regexForExpectedExceptions);
+    });
+  }
+
+  @Test
+  public void cqQueryWithImplicitMethodInvocationOnNonEmptyRegionShouldThrowExceptionDuringExecute() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.name = 'Beth'";
+
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+      executeCqAndAssertException(cq, user, regexForExpectedExceptions);
+    });
+  }
+
+  @Test
+  public void cqQueryWithExplicitMethodInvocationOnNonEmptyRegionShouldThrowExceptionDuringExecute() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.getName = 'Beth'";
+
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+      executeCqAndAssertException(cq, user, regexForExpectedExceptions);
+    });
+  }
+
+  @Test
+  public void cqQueryWithImplicitMethodInvocationOnNonEmptyRegionShouldThrowExceptionDuringExecuteWithInitialResults() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.name = 'Beth'";
+
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+      executeCqWithInitialResultsAndAssertException(cq, user, regexForExpectedExceptions);
+    });
+  }
+
+  @Test
+  public void cqQueryWithExplicitMethodInvocationOnNonEmptyRegionShouldThrowExceptionDuringExecuteWithInitialResults() {
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.getName = 'Beth'";
+
+    specificUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+      executeCqWithInitialResultsAndAssertException(cq, user, regexForExpectedExceptions);
+    });
+  }
+
+  @Test
+  public void cqCreatedByAllowedUserButPutDoneByForbiddenReaderShouldStillInvokeListener() {
+    assumeTrue(user.equals("dataWriter"));
+    String query = "SELECT * FROM /" + regionName + " r WHERE r.id = 1";
+
+    superUserClient.invoke(() -> {
+      CqQuery cq = createCq(query);
+      cq.execute();
+    });
+
+    Object[] keys = {"key-0"};
+    Object[] values = {new QueryTestObject(1, "Mary")};
+    putIntoRegion(specificUserClient, keys, values, regionName);
+
+    superUserClient.invoke(
+        () -> await().untilAsserted(() -> assertThat(cqListener.getNumEvents()).isEqualTo(1)));
+  }
+}
diff --git a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedIndexSecurityDUnitTest.java b/geode-cq/src/distributedTest/java/org/apache/geode/security/query/TestCqListener.java
similarity index 57%
rename from geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedIndexSecurityDUnitTest.java
rename to geode-cq/src/distributedTest/java/org/apache/geode/security/query/TestCqListener.java
index 12746e5..8049567 100644
--- a/geode-core/src/distributedTest/java/org/apache/geode/security/query/PartitionedIndexSecurityDUnitTest.java
+++ b/geode-cq/src/distributedTest/java/org/apache/geode/security/query/TestCqListener.java
@@ -14,21 +14,31 @@
  */
 package org.apache.geode.security.query;
 
-import org.junit.experimental.categories.Category;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import org.apache.geode.cache.query.CqEvent;
+import org.apache.geode.cache.query.CqListener;
 
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.test.junit.categories.SecurityTest;
-import org.apache.geode.test.junit.runners.CategoryWithParameterizedRunnerFactory;
+public class TestCqListener implements CqListener {
+  private int numEvents = 0;
+  private int numErrors = 0;
+
+  public int getNumEvents() {
+    return numEvents;
+  }
+
+  public int getNumErrors() {
+    return numErrors;
+  }
 
-@Category({SecurityTest.class})
-@RunWith(Parameterized.class)
-@Parameterized.UseParametersRunnerFactory(CategoryWithParameterizedRunnerFactory.class)
-public class PartitionedIndexSecurityDUnitTest extends IndexSecurityDUnitTest {
   @Override
-  public RegionShortcut getRegionType() {
-    return RegionShortcut.PARTITION;
+  public void onEvent(CqEvent aCqEvent) {
+    numEvents++;
   }
 
+  @Override
+  public void onError(CqEvent aCqEvent) {
+    numErrors++;
+  }
+
+  @Override
+  public void close() {}
 }
diff --git a/geode-dunit/src/main/java/org/apache/geode/security/query/AbstractQuerySecurityDistributedTest.java b/geode-dunit/src/main/java/org/apache/geode/security/query/AbstractQuerySecurityDistributedTest.java
new file mode 100644
index 0000000..144641e
--- /dev/null
+++ b/geode-dunit/src/main/java/org/apache/geode/security/query/AbstractQuerySecurityDistributedTest.java
@@ -0,0 +1,168 @@
+/*
+ * 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.security.query;
+
+import static org.apache.geode.distributed.ConfigurationProperties.SERIALIZABLE_OBJECT_FILTER;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.IntStream;
+
+import org.junit.Rule;
+
+import org.apache.geode.cache.Region;
+import org.apache.geode.cache.RegionShortcut;
+import org.apache.geode.cache.client.ClientCache;
+import org.apache.geode.cache.client.ClientRegionShortcut;
+import org.apache.geode.cache.client.Pool;
+import org.apache.geode.cache.client.PoolManager;
+import org.apache.geode.cache.query.FunctionDomainException;
+import org.apache.geode.cache.query.NameResolutionException;
+import org.apache.geode.cache.query.Query;
+import org.apache.geode.cache.query.QueryInvocationTargetException;
+import org.apache.geode.cache.query.QueryService;
+import org.apache.geode.cache.query.TypeMismatchException;
+import org.apache.geode.security.NotAuthorizedException;
+import org.apache.geode.security.TestSecurityManager;
+import org.apache.geode.test.dunit.rules.ClientVM;
+import org.apache.geode.test.dunit.rules.ClusterStartupRule;
+import org.apache.geode.test.dunit.rules.MemberVM;
+
+public abstract class AbstractQuerySecurityDistributedTest implements Serializable {
+  protected Object[] keys;
+  protected Object[] values;
+  protected MemberVM server;
+  protected ClientVM superUserClient, specificUserClient;
+  protected final String regionName = "region";
+  private transient UserPermissions userPerms = new UserPermissions();
+
+  @Rule
+  public ClusterStartupRule cluster = new ClusterStartupRule();
+
+  protected void setUpSuperUserClientAndServer(RegionShortcut regionShortcut) throws Exception {
+    server = cluster.startServerVM(1, cf -> cf
+        .withSecurityManager(TestSecurityManager.class)
+        .withProperty(TestSecurityManager.SECURITY_JSON,
+            "org/apache/geode/management/internal/security/clientServer.json")
+        .withProperty(SERIALIZABLE_OBJECT_FILTER, "org.apache.geode.security.query.data.*")
+        .withRegion(regionShortcut, regionName));
+
+    String superUserPassword = userPerms.getUserPassword("super-user");
+    superUserClient =
+        cluster.startClientVM(2, ccf -> ccf.withCredential("super-user", superUserPassword)
+            .withPoolSubscription(true)
+            .withServerConnection(server.getPort())
+            .withProperty(SERIALIZABLE_OBJECT_FILTER, "org.apache.geode.security.query.data.*"));
+    superUserClient.invoke(() -> {
+      assertThat(ClusterStartupRule.getClientCache()).isNotNull();
+      ClusterStartupRule.getClientCache().createClientRegionFactory(ClientRegionShortcut.PROXY)
+          .create(regionName);
+    });
+  }
+
+  protected void setUpSpecificClient(String specificUser) throws Exception {
+    String specificUserPassword = userPerms.getUserPassword(specificUser);
+    specificUserClient =
+        cluster.startClientVM(3, ccf -> ccf.withCredential(specificUser, specificUserPassword)
+            .withPoolSubscription(true)
+            .withServerConnection(server.getPort())
+            .withProperty(SERIALIZABLE_OBJECT_FILTER, "org.apache.geode.security.query.data.*"));
+    specificUserClient.invoke(() -> {
+      assertThat(ClusterStartupRule.getClientCache()).isNotNull();
+      ClusterStartupRule.getClientCache().createClientRegionFactory(ClientRegionShortcut.PROXY)
+          .create(regionName);
+    });
+  }
+
+  public void setUp(String specificUser, RegionShortcut regionShortcut) throws Exception {
+    setUpSuperUserClientAndServer(regionShortcut);
+    setUpSpecificClient(specificUser);
+  }
+
+  public ClientCache getClientCache() {
+    return ClusterStartupRule.clientCacheRule.getCache();
+  }
+
+  protected void putIntoRegion(ClientVM vm, Object[] keys, Object[] values, String regionName) {
+    vm.invoke(() -> {
+      Region<Object, Object> region = getClientCache().getRegion(regionName);
+      assertThat(values.length)
+          .as("The list of keys does not have the same length as the list of values.")
+          .isEqualTo(keys.length);
+
+      IntStream.range(0, keys.length).forEach(i -> region.put(keys[i], values[i]));
+    });
+  }
+
+  private void assertExceptionOccurred(QueryService qs, String query, String authErrorRegexp) {
+    assertThatThrownBy(() -> qs.newQuery(query).execute())
+        .hasMessageMatching(authErrorRegexp)
+        .hasCauseInstanceOf(NotAuthorizedException.class);
+  }
+
+  @SuppressWarnings("unchecked")
+  protected void assertQueryResults(ClientCache clientCache, String queryString,
+      Object[] bindParameters, List<Object> expectedResults) throws FunctionDomainException,
+      TypeMismatchException, NameResolutionException, QueryInvocationTargetException {
+    Collection results;
+    Query query = clientCache.getQueryService().newQuery(queryString);
+
+    if (bindParameters == null) {
+      results = (Collection) query.execute();
+    } else {
+      results = (Collection) query.execute(bindParameters);
+    }
+
+    assertThat(results).isNotNull();
+    assertThat(results.size())
+        .as("Query results size did not match expected for " + query)
+        .isEqualTo(expectedResults.size());
+
+    results.forEach((i) -> assertThat(expectedResults.contains(i))
+        .as("Result:" + i + " was not found in the expectedResults")
+        .isTrue());
+  }
+
+  private void assertQueryResults(ClientCache clientCache, String query,
+      List<Object> expectedResults) throws FunctionDomainException, TypeMismatchException,
+      NameResolutionException, QueryInvocationTargetException {
+    assertQueryResults(clientCache, query, null, expectedResults);
+  }
+
+  protected void assertRegionData(ClientVM vm, List<Object> expectedRegionResults) {
+    vm.invoke(() -> assertQueryResults(getClientCache(), "SELECT * FROM /" + regionName,
+        expectedRegionResults));
+  }
+
+  protected void executeQueryAndAssertExpectedResults(ClientVM vm, String query,
+      List<Object> expectedSuccessfulQueryResults) {
+    vm.invoke(() -> assertQueryResults(getClientCache(), query, expectedSuccessfulQueryResults));
+  }
+
+  protected void executeQueryAndAssertThatNoAuthorizedExceptionWasThrown(ClientVM vm, String query,
+      String regexForExpectedExceptions) {
+    vm.invoke(() -> {
+      Region region = getClientCache().getRegion(regionName);
+      assertExceptionOccurred(getClientCache().getQueryService(), query,
+          regexForExpectedExceptions);
+
+      Pool pool = PoolManager.find(region);
+      assertExceptionOccurred(pool.getQueryService(), query, regexForExpectedExceptions);
+    });
+  }
+}
diff --git a/geode-dunit/src/main/java/org/apache/geode/security/query/QuerySecurityBase.java b/geode-dunit/src/main/java/org/apache/geode/security/query/QuerySecurityBase.java
deleted file mode 100644
index e7b694a..0000000
--- a/geode-dunit/src/main/java/org/apache/geode/security/query/QuerySecurityBase.java
+++ /dev/null
@@ -1,238 +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.security.query;
-
-import static org.apache.geode.distributed.ConfigurationProperties.SECURITY_MANAGER;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-
-import org.apache.geode.cache.Region;
-import org.apache.geode.cache.RegionShortcut;
-import org.apache.geode.cache.client.ClientCache;
-import org.apache.geode.cache.client.Pool;
-import org.apache.geode.cache.client.PoolManager;
-import org.apache.geode.cache.query.FunctionDomainException;
-import org.apache.geode.cache.query.NameResolutionException;
-import org.apache.geode.cache.query.Query;
-import org.apache.geode.cache.query.QueryInvocationTargetException;
-import org.apache.geode.cache.query.QueryService;
-import org.apache.geode.cache.query.TypeMismatchException;
-import org.apache.geode.internal.cache.GemFireCacheImpl;
-import org.apache.geode.security.SecurityTestUtil;
-import org.apache.geode.security.TestSecurityManager;
-import org.apache.geode.test.dunit.Host;
-import org.apache.geode.test.dunit.VM;
-import org.apache.geode.test.dunit.internal.JUnit4DistributedTestCase;
-import org.apache.geode.test.dunit.rules.ClusterStartupRule;
-import org.apache.geode.test.dunit.rules.DistributedRule;
-import org.apache.geode.test.junit.rules.ServerStarterRule;
-
-/**
- * @deprecated Please use {@link DistributedRule} and Geode User APIs or {@link ClusterStartupRule}
- *             instead.
- */
-public abstract class QuerySecurityBase extends JUnit4DistributedTestCase {
-
-  public RegionShortcut getRegionType() {
-    return RegionShortcut.REPLICATE;
-  }
-
-  protected String regionName = "region";
-  protected Object[] keys;
-  protected Object[] values;
-
-  @Rule
-  public ServerStarterRule server =
-      new ServerStarterRule().withProperty(SECURITY_MANAGER, TestSecurityManager.class.getName())
-          .withProperty(TestSecurityManager.SECURITY_JSON,
-              "org/apache/geode/management/internal/security/clientServer.json")
-          .withRegion(getRegionType(), regionName);
-
-  // Varibles used to store caches between invoke methods
-  private static ClientCache clientCache;
-
-  protected transient UserPermissions userPerms = new UserPermissions();
-
-  protected Host host;
-  protected VM superUserClient;
-  protected VM specificUserClient;
-
-  @Before
-  public void configureTest() {
-    host = Host.getHost(0);
-    superUserClient = host.getVM(1);
-    specificUserClient = host.getVM(2);
-    createClientCache(superUserClient, "super-user", userPerms.getUserPassword("super-user"));
-    createProxyRegion(superUserClient, regionName);
-  }
-
-  public void closeAnyPollutedCache() {
-    if (GemFireCacheImpl.getInstance() != null) {
-      GemFireCacheImpl.getInstance().close();
-    }
-  }
-
-  public void setClientCache(ClientCache cache) {
-    clientCache = cache;
-  }
-
-  public ClientCache getClientCache() {
-    return clientCache;
-  }
-
-  public void createClientCache(VM vm, String userName, String password) {
-    vm.invoke(() -> {
-      closeAnyPollutedCache();
-      ClientCache cache = SecurityTestUtil.createClientCache(userName, password, server.getPort());
-      setClientCache(cache);
-    });
-  }
-
-  public void createProxyRegion(VM vm, String regionName) {
-    vm.invoke(() -> {
-      SecurityTestUtil.createProxyRegion(getClientCache(), regionName);
-    });
-  }
-
-  @After
-  public void closeClientCaches() {
-    closeClientCache(superUserClient);
-    closeClientCache(specificUserClient);
-  }
-
-  public void closeClientCache(VM vm) {
-    vm.invoke(() -> {
-      if (getClientCache() != null) {
-        getClientCache().close();
-      }
-    });
-  }
-
-  protected void assertExceptionOccurred(QueryService qs, String query, String authErrorRegexp) {
-    try {
-      qs.newQuery(query).execute();
-      fail();
-    } catch (Exception e) {
-      e.printStackTrace();
-      if (!e.getMessage().matches(authErrorRegexp)) {
-        Throwable cause = e.getCause();
-        while (cause != null) {
-          if (cause.getMessage().matches(authErrorRegexp)) {
-            return;
-          }
-          cause = cause.getCause();
-        }
-        e.printStackTrace();
-        fail();
-      }
-    }
-  }
-
-  protected void assertExceptionOccurred(QueryService qs, String query, Object[] bindParams,
-      String authErrorRegexp) {
-    System.out.println("Execution exception should match:" + authErrorRegexp);
-    try {
-      qs.newQuery(query).execute(bindParams);
-      fail();
-    } catch (Exception e) {
-
-      if (!e.getMessage().matches(authErrorRegexp)) {
-        Throwable cause = e.getCause();
-        while (cause != null) {
-          if (cause.getMessage().matches(authErrorRegexp)) {
-            return;
-          }
-          cause = cause.getCause();
-        }
-        e.printStackTrace();
-        fail();
-      }
-    }
-  }
-
-  protected void assertQueryResults(ClientCache clientCache, String query,
-      List<Object> expectedResults) throws FunctionDomainException, TypeMismatchException,
-      NameResolutionException, QueryInvocationTargetException {
-    assertQueryResults(clientCache, query, null, expectedResults);
-  }
-
-  protected void assertQueryResults(ClientCache clientCache, String queryString,
-      Object[] bindParameters, List<Object> expectedResults) throws FunctionDomainException,
-      TypeMismatchException, NameResolutionException, QueryInvocationTargetException {
-    Query query = clientCache.getQueryService().newQuery(queryString);
-    Collection results;
-    if (bindParameters == null) {
-      results = (Collection) query.execute();
-    } else {
-      results = (Collection) query.execute(bindParameters);
-    }
-    assertNotNull(results);
-    assertEquals("Query results size did not match expected for " + query, expectedResults.size(),
-        results.size());
-
-    results.forEach((i) -> {
-      assertTrue("Result:" + i + " was not found in the expectedResults",
-          expectedResults.contains(i));
-    });
-  }
-
-  public void executeAndConfirmRegionMatches(VM vm, String regionName,
-      List<Object> expectedRegionResults) throws Exception {
-    vm.invoke(() -> {
-      assertQueryResults(getClientCache(), "select * from /" + regionName, expectedRegionResults);
-    });
-  }
-
-  protected void putIntoRegion(VM vm, Object[] keys, Object[] values, String regionName) {
-    vm.invoke(() -> {
-      Region region = getClientCache().getRegion(regionName);
-      assertEquals(
-          "Bad region put. The list of keys does not have the same length as the list of values.",
-          keys.length, values.length);
-      for (int i = 0; i < keys.length; i++) {
-        region.put(keys[i], values[i]);
-      }
-    });
-  }
-
-  protected void executeQueryWithCheckForAccessPermissions(VM vm, String query, String regionName,
-      List<Object> expectedSuccessfulQueryResults) {
-    vm.invoke(() -> {
-      Region region = getClientCache().getRegion(regionName);
-      assertQueryResults(getClientCache(), query, expectedSuccessfulQueryResults);
-    });
-  }
-
-
-  protected void executeQueryWithCheckForAccessPermissions(VM vm, String query, String regionName,
-      String regexForExpectedExceptions) {
-    vm.invoke(() -> {
-      Region region = getClientCache().getRegion(regionName);
-      assertExceptionOccurred(getClientCache().getQueryService(), query,
-          regexForExpectedExceptions);
-      Pool pool = PoolManager.find(region);
-      assertExceptionOccurred(pool.getQueryService(), query, regexForExpectedExceptions);
-    });
-  }
-}