You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "adelapena (via GitHub)" <gi...@apache.org> on 2023/02/01 10:17:41 UTC

[GitHub] [cassandra] adelapena opened a new pull request, #2125: CASSANDRA-18069 trunk: Add UNMASK permission

adelapena opened a new pull request, #2125:
URL: https://github.com/apache/cassandra/pull/2125

   Thanks for sending a pull request! Here are some tips if you're new here:
    
    * Ensure you have added or run the [appropriate tests](https://cassandra.apache.org/_/development/testing.html) for your PR.
    * Be sure to keep the PR description updated to reflect all changes.
    * Write your PR title to summarize what this PR proposes.
    * If possible, provide a concise example to reproduce the issue for a faster review.
    * Read our [contributor guidelines](https://cassandra.apache.org/_/development/index.html)
    * If you're making a documentation change, see our [guide to documentation contribution](https://cassandra.apache.org/_/development/documentation.html)
    
   Commit messages should follow the following format:
   
   ```
   <One sentence description, usually Jira title or CHANGES.txt summary>
   
   <Optional lengthier description (context on patch)>
   
   patch by <Authors>; reviewed by <Reviewers> for CASSANDRA-#####
   
   Co-authored-by: Name1 <email1>
   Co-authored-by: Name2 <email2>
   
   ```
   
   The [Cassandra Jira](https://issues.apache.org/jira/projects/CASSANDRA/issues/)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1117321986


##########
src/java/org/apache/cassandra/cql3/selection/Selector.java:
##########
@@ -443,14 +447,21 @@ private <V> ByteBuffer value(Cell<V> c)
         }
 
         /**
-         * Return the value of the column with the specified index.
+         * Return the value of the column with the specified index. If the column the value belongs to is masked with a
+         * {@link ColumnMask} and {@link #unmask} hasn't been specified, such mask will be applied to the value.
          *
          * @param index the column index
-         * @return the value of the column with the specified index
+         * @return the value of the column with the specified index, masked if its column is masked
          */
         public ByteBuffer getValue(int index)
         {
-            return values[index];
+            ByteBuffer value = values[index];

Review Comment:
   I have changed [the patch for CASSANDRA-18068](https://github.com/apache/cassandra/pull/2110) to store the masked values on `InputRow` and thus reverted the changes on `SelectStatement`. I'll rebase this PR if that looks good.
   
   Regarding ordering, DDM usually preserves the order of the unmasked columns, mostly because trying to order based on the masked value of the column would require a kind of replica-side ordering that we want to avoid for performance reasons. 
   
   However, if the user explicitly uses `ORDER BY` to order selected partitions in the coordinator the rows will be ordered by the masked value of the clustering key. That's so because when the sorting is done the columns are already masked. I'm not sure whether that is acceptable. If it's not, I guess we could try to either:
   
   - Collect both masked and unmasked values, which adds complexity and memory pressure.
   - Mask after ordering, although that will probably cause problems with chained selectors.
   - Disable `ORDER BY` if there are masked clustering columns
   
   What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blerer commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "blerer (via GitHub)" <gi...@apache.org>.
blerer commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1111903406


##########
src/java/org/apache/cassandra/auth/CassandraRoleManager.java:
##########
@@ -82,7 +82,7 @@ public class CassandraRoleManager implements IRoleManager
     private static final Logger logger = LoggerFactory.getLogger(CassandraRoleManager.class);
 
     public static final String DEFAULT_SUPERUSER_NAME = "cassandra";
-    static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra";
+    public static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra";

Review Comment:
   Do we know why the password was package private but not the username?



##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -121,6 +121,11 @@ public class SelectStatement implements CQLStatement.SingleKeyspaceCqlStatement
      */
     private final Comparator<List<ByteBuffer>> orderingComparator;
 
+    /**
+     * Whether masked columns should be unmasked.
+     */
+    private boolean unmask = true;

Review Comment:
   `SelectStatements` are multithreaded we cannot relies on instance variables.



##########
src/java/org/apache/cassandra/cql3/selection/Selector.java:
##########
@@ -443,14 +447,21 @@ private <V> ByteBuffer value(Cell<V> c)
         }
 
         /**
-         * Return the value of the column with the specified index.
+         * Return the value of the column with the specified index. If the column the value belongs to is masked with a
+         * {@link ColumnMask} and {@link #unmask} hasn't been specified, such mask will be applied to the value.
          *
          * @param index the column index
-         * @return the value of the column with the specified index
+         * @return the value of the column with the specified index, masked if its column is masked
          */
         public ByteBuffer getValue(int index)
         {
-            return values[index];
+            ByteBuffer value = values[index];

Review Comment:
   This approach is different from the one that was used in the previous patch and makes some of the changes useless.
   If we start relying on `InputRow` to do the masking there are no need to go through the `wildcardWithGroupByOrMaskedColumns` path as the wilcard path should now also work. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blerer commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "blerer (via GitHub)" <gi...@apache.org>.
blerer commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1127848226


##########
src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java:
##########
@@ -142,19 +142,24 @@ public void addInput(InputRow input) throws InvalidRequestException
         if (!isSet)
         {
             isSet = true;
-            current = input.getValue(idx);
             writetimes = input.getWritetimes(idx);
             ttls = input.getTtls(idx);
+
+            /*
+            We apply the column mask of the column unless:
+            - The column doesn't have a mask
+            - This selector is for a query with ORDER BY post-ordering, indicated by this.unmask
+            - The input row is for a user with UNMASK permission, indicated by input.umask()
+             */
+            ColumnMask mask = unmask || input.unmask() ? null : column.getMask();

Review Comment:
   It is becoming hard to understand the logic. Should we rename `unmask` by `useForPostOrdering`?



##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -256,6 +258,7 @@ public ResultMessage.Rows execute(QueryState state, QueryOptions options, long q
         int userLimit = getLimit(options);
         int userPerPartitionLimit = getPerPartitionLimit(options);
         int pageSize = options.getPageSize();
+        boolean unmask = state.getClientState().hasUnmaskPermission(table);

Review Comment:
   If the query do not hit a table with masked columns (which will probably be the greatest majority of the cases) does it make sense to waste time looking for the permissions?



##########
test/unit/org/apache/cassandra/cql3/CQLTester.java:
##########
@@ -1721,6 +1726,24 @@ protected void assertColumnNames(UntypedResultSet result, String... expectedColu
         }
     }
 
+    protected void assertColumnNames(ResultSet result, String... expectedColumnNames)

Review Comment:
   Same question for this one.



##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -1401,8 +1411,8 @@ private Comparator<List<ByteBuffer>> getOrderingComparator(Selection selection,
             if (!restrictions.keyIsInRelation())
                 return null;
 
-            List<Integer> idToSort = new ArrayList<>(orderingColumns.size());
-            List<Comparator<ByteBuffer>> sorters = new ArrayList<>(orderingColumns.size());
+            List<Integer> idToSort = new ArrayList<Integer>(orderingColumns.size());

Review Comment:
   Why did we need to add the type? In this case I imagine that the JVM should have inferred it.



##########
test/unit/org/apache/cassandra/cql3/CQLTester.java:
##########
@@ -1276,6 +1276,11 @@ protected com.datastax.driver.core.ResultSet executeNetWithPaging(String query,
         return sessionNet().execute(new SimpleStatement(formatQuery(query)).setFetchSize(pageSize));
     }
 
+    protected com.datastax.driver.core.ResultSet executeNetWithoutPaging(String query)
+    {
+        return executeNetWithPaging(query, Integer.MAX_VALUE);
+    }
+

Review Comment:
   Why did you introduce that method?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1130925908


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -1401,8 +1411,8 @@ private Comparator<List<ByteBuffer>> getOrderingComparator(Selection selection,
             if (!restrictions.keyIsInRelation())
                 return null;
 
-            List<Integer> idToSort = new ArrayList<>(orderingColumns.size());
-            List<Comparator<ByteBuffer>> sorters = new ArrayList<>(orderingColumns.size());
+            List<Integer> idToSort = new ArrayList<Integer>(orderingColumns.size());

Review Comment:
   Actually those pre-Java 7 generics are [on trunk|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java#L1400-L1401]. I updated them on CASSANDRA-18068, but I messed up during rebase. I've changed them again to use the diamond operator.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1118312840


##########
src/java/org/apache/cassandra/cql3/selection/Selector.java:
##########
@@ -443,14 +447,21 @@ private <V> ByteBuffer value(Cell<V> c)
         }
 
         /**
-         * Return the value of the column with the specified index.
+         * Return the value of the column with the specified index. If the column the value belongs to is masked with a
+         * {@link ColumnMask} and {@link #unmask} hasn't been specified, such mask will be applied to the value.
          *
          * @param index the column index
-         * @return the value of the column with the specified index
+         * @return the value of the column with the specified index, masked if its column is masked
          */
         public ByteBuffer getValue(int index)
         {
-            return values[index];
+            ByteBuffer value = values[index];

Review Comment:
   Forbidding `ORDER BY` can be a pain as it would require changing the CQL in your app to toggle behaviors. If they're concerned on ordering they can do a partial hash, use a bespoke masking function,... they're not perfect solutions but they sound better than the added cost to sort by masked values imo.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1105384822


##########
test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeColumnMaskingTest.java:
##########
@@ -104,10 +111,21 @@ private static void assertFails(ICoordinator coordinator, String schemaQuery, St
                   .hasMessageContaining(expectedMessage);
     }
 
-    private void assertColumnValue(UpgradeableCluster cluster, String table, String column, Object value)
+    private static void assertColumnIsMasked(UpgradeableCluster cluster, String table, String column)

Review Comment:
   I wonder if _not_ checking the actual value now is an inferior solution to the previous one... Here we're testing the wiring is ok but not that the light bulb actually switches on anymore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1127381294


##########
src/java/org/apache/cassandra/auth/CassandraRoleManager.java:
##########
@@ -82,7 +82,7 @@ public class CassandraRoleManager implements IRoleManager
     private static final Logger logger = LoggerFactory.getLogger(CassandraRoleManager.class);
 
     public static final String DEFAULT_SUPERUSER_NAME = "cassandra";
-    static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra";
+    public static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra";

Review Comment:
   Do we want to add a `VisibleForTesting` or similar here? is it important they're not accessed directly or we don't care?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1127387565


##########
doc/cql3/CQL.textile:
##########
@@ -1587,6 +1588,9 @@ Not all permissions are applicable to every type of resource. For instance, @EXE
 | @EXECUTE@    | @ALL MBEANS@                 |Execute operations on any mbean|
 | @EXECUTE@    | @MBEANS@                     |Execute operations on any mbean matching a wildcard pattern|
 | @EXECUTE@    | @MBEAN@                      |Execute operations on named mbean|
+| @UNMASK@     | @ALL KEYSPACES@              |See the clear contents of masked columns on any table|
+| @UNMASK@     | @KEYSPACE@                   |See the clear contents of masked columns on any table in keyspace|
+| @UNMASK@     | @TABLE@                      |See the clear contents of masked columns on specified table|

Review Comment:
   on _the_ specified table?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1128408960


##########
CHANGES.txt:
##########
@@ -1,4 +1,5 @@
 4.2
+ * Add UNMASK permission (CASSANDRA-18068)

Review Comment:
   Indeed it should, changed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1130840739


##########
test/unit/org/apache/cassandra/cql3/CQLTester.java:
##########
@@ -1276,6 +1276,11 @@ protected com.datastax.driver.core.ResultSet executeNetWithPaging(String query,
         return sessionNet().execute(new SimpleStatement(formatQuery(query)).setFetchSize(pageSize));
     }
 
+    protected com.datastax.driver.core.ResultSet executeNetWithoutPaging(String query)
+    {
+        return executeNetWithPaging(query, Integer.MAX_VALUE);
+    }
+

Review Comment:
   `executeNet` uses the default page size, whereas `executeNetWithPaging` allows to specify an ad-hoc page size for the query. The new method is a shortcut to run the query with paging disabled, which is required by post-ordering.
   
   Currently the only caller is `ColumnMaskTest`, but I think it can be useful for other tests to have that utility method on `CQLTester`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1127388450


##########
doc/modules/cassandra/pages/cql/cql_singlefile.adoc:
##########
@@ -2420,6 +2421,12 @@ use of function in `CREATE AGGREGATE` | | |
 wildcard pattern | | |
 
 |`EXECUTE` |`MBEAN` |Execute operations on named mbean | | |
+
+|`UNMASK` |`ALL KEYSPACES` |See the clear contents of masked columns on any table | | |
+
+|`UNMASK` |`KEYSPACE` |See the clear contents of masked columns on any table in keyspace | | |
+
+|`UNMASK` |`TABLE` |See the clear contents of masked columns on specified table | | |

Review Comment:
   same



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#issuecomment-1463471408

   Thx


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1130930564


##########
src/java/org/apache/cassandra/cql3/selection/SimpleSelector.java:
##########
@@ -142,19 +142,24 @@ public void addInput(InputRow input) throws InvalidRequestException
         if (!isSet)
         {
             isSet = true;
-            current = input.getValue(idx);
             writetimes = input.getWritetimes(idx);
             ttls = input.getTtls(idx);
+
+            /*
+            We apply the column mask of the column unless:
+            - The column doesn't have a mask
+            - This selector is for a query with ORDER BY post-ordering, indicated by this.unmask
+            - The input row is for a user with UNMASK permission, indicated by input.umask()
+             */
+            ColumnMask mask = unmask || input.unmask() ? null : column.getMask();

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1130915995


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -256,6 +258,7 @@ public ResultMessage.Rows execute(QueryState state, QueryOptions options, long q
         int userLimit = getLimit(options);
         int userPerPartitionLimit = getPerPartitionLimit(options);
         int pageSize = options.getPageSize();
+        boolean unmask = state.getClientState().hasUnmaskPermission(table);

Review Comment:
   Good idea, done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1127386760


##########
CHANGES.txt:
##########
@@ -1,4 +1,5 @@
 4.2
+ * Add UNMASK permission (CASSANDRA-18068)

Review Comment:
   This should be 18069?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1105413198


##########
test/unit/org/apache/cassandra/cql3/functions/masking/UnmaskPermissionTest.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.cassandra.cql3.functions.masking;
+
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.exceptions.AuthenticationException;
+import com.datastax.driver.core.exceptions.UnauthorizedException;
+import org.apache.cassandra.cql3.CQLTester;
+import org.awaitility.Awaitility;
+import org.awaitility.core.ThrowingRunnable;
+
+import static java.lang.String.format;
+
+/**
+ * Tests the {@link org.apache.cassandra.auth.Permission#UNMASK} permission.
+ * <p>
+ * The permission is tested for a regular user with the {@code UNMASK} permissions on different resources,
+ * while also verifying the absence of side effects on other ordinary users, superusers and internal queries.
+ */
+public class UnmaskPermissionTest extends CQLTester
+{
+    private static final String CREATE_KEYSPACE = "CREATE KEYSPACE IF NOT EXISTS %s WITH replication = " +
+                                                  "{'class': 'SimpleStrategy', 'replication_factor': '1'}";
+    private static final String CREATE_TABLE = "CREATE TABLE IF NOT EXISTS %s.%s " +
+                                               "(k int, c int, v text MASKED WITH mask_replace('redacted'), " +
+                                               "PRIMARY KEY (k, c))";
+    private static final String INSERT = "INSERT INTO %s.%s (k, c, v) VALUES (?, ?, ?)";
+    private static final String SELECT_WILDCARD = "SELECT * FROM %s.%s";
+    private static final String SELECT_COLUMNS = "SELECT k, c, v FROM %s.%s";
+
+    private static final Object[] CLEAR_ROW = row(0, 0, "sensitive");
+    private static final Object[] MASKED_ROW = row(0, 0, "redacted");
+
+    private static final String KEYSPACE_1 = "mask_keyspace_1";
+    private static final String KEYSPACE_2 = "mask_keyspace_2";
+    private static final String TABLE_1 = "mask_table_1";
+    private static final String TABLE_2 = "mask_table_2";
+
+    private static final String USER = "ddm_user"; // user that will have their permissions changed
+    private static final String OTHER_USER = "ddm_ordinary_user"; // user that won't have their permissions altered
+    private static final String PASSWORD = "ddm_password";
+
+    @BeforeClass
+    public static void beforeClass()
+    {
+        CQLTester.setUpClass();
+        requireAuthentication();
+        requireNetwork();
+    }
+
+    @Before
+    public void before() throws Throwable
+    {
+        useSuperUser();
+
+        schemaChange(format(CREATE_KEYSPACE, KEYSPACE_1));
+        schemaChange(format(CREATE_KEYSPACE, KEYSPACE_2));
+
+        createTable(format(CREATE_TABLE, KEYSPACE_1, TABLE_1));
+        createTable(format(CREATE_TABLE, KEYSPACE_1, TABLE_2));
+        createTable(format(CREATE_TABLE, KEYSPACE_2, TABLE_1));
+
+        execute(format(INSERT, KEYSPACE_1, TABLE_1), CLEAR_ROW);
+        execute(format(INSERT, KEYSPACE_1, TABLE_2), CLEAR_ROW);
+        execute(format(INSERT, KEYSPACE_2, TABLE_1), CLEAR_ROW);
+
+        for (String user : Arrays.asList(USER, OTHER_USER))

Review Comment:
   Couldn't all the above CQL be moved to 'BeforeClass' to avoid extra work? We never drop those in between tests



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1115714276


##########
src/java/org/apache/cassandra/cql3/selection/Selector.java:
##########
@@ -443,14 +447,21 @@ private <V> ByteBuffer value(Cell<V> c)
         }
 
         /**
-         * Return the value of the column with the specified index.
+         * Return the value of the column with the specified index. If the column the value belongs to is masked with a
+         * {@link ColumnMask} and {@link #unmask} hasn't been specified, such mask will be applied to the value.
          *
          * @param index the column index
-         * @return the value of the column with the specified index
+         * @return the value of the column with the specified index, masked if its column is masked
          */
         public ByteBuffer getValue(int index)
         {
-            return values[index];
+            ByteBuffer value = values[index];

Review Comment:
   Why would those changes be unnecessary? Note that `InputRow` collects the same as before, and it's only the `InputRow#getValue` getter method what is changed. The only caller of `InputRow#getValue` is `SimpleSelector`, which is where we did the masking before. So things shouldn't change from the previous state.
   
   Indeed the wildcard query tests fail if we remove the call to `TableMetadata#hasMaskedColumns` in `SelectStatement.RawStatement#prepareSelection` that makes it go through `wildcardWithGroupByOrMaskedColumns`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1127382281


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -121,6 +121,11 @@ public class SelectStatement implements CQLStatement.SingleKeyspaceCqlStatement
      */
     private final Comparator<List<ByteBuffer>> orderingComparator;
 
+    /**
+     * Whether masked columns should be unmasked.
+     */
+    private boolean unmask = true;

Review Comment:
   We don't want to add the `ThreadSafe` annotation to the class?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1128409702


##########
doc/cql3/CQL.textile:
##########
@@ -1587,6 +1588,9 @@ Not all permissions are applicable to every type of resource. For instance, @EXE
 | @EXECUTE@    | @ALL MBEANS@                 |Execute operations on any mbean|
 | @EXECUTE@    | @MBEANS@                     |Execute operations on any mbean matching a wildcard pattern|
 | @EXECUTE@    | @MBEAN@                      |Execute operations on named mbean|
+| @UNMASK@     | @ALL KEYSPACES@              |See the clear contents of masked columns on any table|
+| @UNMASK@     | @KEYSPACE@                   |See the clear contents of masked columns on any table in keyspace|
+| @UNMASK@     | @TABLE@                      |See the clear contents of masked columns on specified table|

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1127388900


##########
doc/modules/cassandra/pages/cql/dynamic_data_masking.adoc:
##########
@@ -67,3 +67,61 @@ In a similar way, a masking function can be dettached from a column with an `ALT
 ----
 include::example$CQL/ddm_drop_mask.cql[]
 ----
+
+== Permissions
+
+The `UNMASK` permission allows users to retrieve the unmasked values of masked columns.
+The masks will only be applied to the results of a `SELECT` query if the user doesn't have the `UNMASK` permission.
+Ordinary users are created without the `UNMASK` permission, whereas superusers do have it.
+
+As an example, suppose that we have table with masked columns:

Review Comment:
   hav _a_ table



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1105384822


##########
test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeColumnMaskingTest.java:
##########
@@ -104,10 +111,21 @@ private static void assertFails(ICoordinator coordinator, String schemaQuery, St
                   .hasMessageContaining(expectedMessage);
     }
 
-    private void assertColumnValue(UpgradeableCluster cluster, String table, String column, Object value)
+    private static void assertColumnIsMasked(UpgradeableCluster cluster, String table, String column)

Review Comment:
   I wonder if _not_ checking the actual value now is an inferior solution to the previous one... Here we're testing the wiring is ok but not that the lighbulb switches on anymore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1106692062


##########
test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeColumnMaskingTest.java:
##########
@@ -104,10 +111,21 @@ private static void assertFails(ICoordinator coordinator, String schemaQuery, St
                   .hasMessageContaining(expectedMessage);
     }
 
-    private void assertColumnValue(UpgradeableCluster cluster, String table, String column, Object value)
+    private static void assertColumnIsMasked(UpgradeableCluster cluster, String table, String column)

Review Comment:
   I would do it if there was an easy way to target the SELECT to a specific node, but missing that it's not worth it imo



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1115693311


##########
src/java/org/apache/cassandra/auth/CassandraRoleManager.java:
##########
@@ -82,7 +82,7 @@ public class CassandraRoleManager implements IRoleManager
     private static final Logger logger = LoggerFactory.getLogger(CassandraRoleManager.class);
 
     public static final String DEFAULT_SUPERUSER_NAME = "cassandra";
-    static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra";
+    public static final String DEFAULT_SUPERUSER_PASSWORD = "cassandra";

Review Comment:
   The username was made `public` for guardrails tests. Those tests use it in a call to `ClientState#authenticate` call that doesn't require the password. However, here we need both the username and the password to authenticate with the driver.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#issuecomment-1463337109

   +1 with a lingering question. Why the removal of the spin asserts?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1128410033


##########
doc/modules/cassandra/pages/cql/cql_singlefile.adoc:
##########
@@ -2420,6 +2421,12 @@ use of function in `CREATE AGGREGATE` | | |
 wildcard pattern | | |
 
 |`EXECUTE` |`MBEAN` |Execute operations on named mbean | | |
+
+|`UNMASK` |`ALL KEYSPACES` |See the clear contents of masked columns on any table | | |
+
+|`UNMASK` |`KEYSPACE` |See the clear contents of masked columns on any table in keyspace | | |
+
+|`UNMASK` |`TABLE` |See the clear contents of masked columns on specified table | | |

Review Comment:
   Changed



##########
doc/modules/cassandra/pages/cql/dynamic_data_masking.adoc:
##########
@@ -67,3 +67,61 @@ In a similar way, a masking function can be dettached from a column with an `ALT
 ----
 include::example$CQL/ddm_drop_mask.cql[]
 ----
+
+== Permissions
+
+The `UNMASK` permission allows users to retrieve the unmasked values of masked columns.
+The masks will only be applied to the results of a `SELECT` query if the user doesn't have the `UNMASK` permission.
+Ordinary users are created without the `UNMASK` permission, whereas superusers do have it.
+
+As an example, suppose that we have table with masked columns:

Review Comment:
   Fixed



##########
doc/modules/cassandra/pages/cql/security.adoc:
##########
@@ -477,6 +478,12 @@ and use of any function in keyspace in `CREATE AGGREGATE`
 | `EXECUTE` | `MBEANS` | Execute operations on any mbean matching a wildcard pattern
 
 | `EXECUTE` | `MBEAN` | Execute operations on named mbean
+
+|`UNMASK` |`ALL KEYSPACES` | See the clear contents of masked columns on any table
+
+|`UNMASK` |`KEYSPACE` | See the clear contents of masked columns on any table in keyspace
+
+|`UNMASK` |`TABLE` | See the clear contents of masked columns on specified table

Review Comment:
   Changed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1130925908


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -1401,8 +1411,8 @@ private Comparator<List<ByteBuffer>> getOrderingComparator(Selection selection,
             if (!restrictions.keyIsInRelation())
                 return null;
 
-            List<Integer> idToSort = new ArrayList<>(orderingColumns.size());
-            List<Comparator<ByteBuffer>> sorters = new ArrayList<>(orderingColumns.size());
+            List<Integer> idToSort = new ArrayList<Integer>(orderingColumns.size());

Review Comment:
   Actually, those pre-Java 7 generics are [on trunk](https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java#L1400-L1401). I updated them on CASSANDRA-18068, but I messed up during rebase. I've changed them again to use the diamond operator.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1127389706


##########
doc/modules/cassandra/pages/cql/security.adoc:
##########
@@ -477,6 +478,12 @@ and use of any function in keyspace in `CREATE AGGREGATE`
 | `EXECUTE` | `MBEANS` | Execute operations on any mbean matching a wildcard pattern
 
 | `EXECUTE` | `MBEAN` | Execute operations on named mbean
+
+|`UNMASK` |`ALL KEYSPACES` | See the clear contents of masked columns on any table
+
+|`UNMASK` |`KEYSPACE` | See the clear contents of masked columns on any table in keyspace
+
+|`UNMASK` |`TABLE` | See the clear contents of masked columns on specified table

Review Comment:
   same



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1105748371


##########
test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeColumnMaskingTest.java:
##########
@@ -104,10 +111,21 @@ private static void assertFails(ICoordinator coordinator, String schemaQuery, St
                   .hasMessageContaining(expectedMessage);
     }
 
-    private void assertColumnValue(UpgradeableCluster cluster, String table, String column, Object value)
+    private static void assertColumnIsMasked(UpgradeableCluster cluster, String table, String column)

Review Comment:
   The purpose of the test is only verifying that functions cannot be attached to table columns during a rolling upgrade involving nodes that don't include DDM. So I think the important part is checking that the schema-altering queries that add a mask are rejected on mixed clusters. 
   
   Verifying that the masks are actually created is an extra, and I tried to achieve it in the simpler possible way. In CASSANDRA-18069 it was easy enough to just run a query. However, this is trickier to do with the dtests machinery when we have permissions. We would need to create users and CQL driver sessions, and switch between sessions to use a superuser for the schema-altering queries and an unprivileged user for the `SELECT` queries. 
   
   Since we have two nodes in different versions and we want the driver to target specific nodes, we would need to write a custom `LoadBalancingPolicy` that only connects to one node. Also, since permission propagation takes a time we would need a spin assert for every `SELECT` query. 
   
   The combination of multiple sessions and multiple spin asserts would make the test significantly slower. Since upgrade dtests are quite prone to timeouts and OOMs, we would probably need to split the test with at least one class per upgrade path.
   
   These changes add quite a bit of complexity to what I think is the less important part of the test. So I decided to go for the simpler approach of just checking that the tested schema changes are applied, and assume that if the schema is good then the behaviour associated to it is also good. After all, that part should be covered by the regular unit tests.
   
   Do you think it makes sense, or should we go for an upgrade dtest testing also `SELECT` queries?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blerer commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "blerer (via GitHub)" <gi...@apache.org>.
blerer commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1117334235


##########
src/java/org/apache/cassandra/cql3/selection/Selector.java:
##########
@@ -443,14 +447,21 @@ private <V> ByteBuffer value(Cell<V> c)
         }
 
         /**
-         * Return the value of the column with the specified index.
+         * Return the value of the column with the specified index. If the column the value belongs to is masked with a
+         * {@link ColumnMask} and {@link #unmask} hasn't been specified, such mask will be applied to the value.
          *
          * @param index the column index
-         * @return the value of the column with the specified index
+         * @return the value of the column with the specified index, masked if its column is masked
          */
         public ByteBuffer getValue(int index)
         {
-            return values[index];
+            ByteBuffer value = values[index];

Review Comment:
   It is the question I was wondering. May be rejecting `ORDER BY` is the way to go. I am not sure to be honest. For once I would be in favor of asking on the dev list to get more input. Of course it might take a bit of time before we get an answer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1118763288


##########
src/java/org/apache/cassandra/cql3/selection/Selector.java:
##########
@@ -443,14 +447,21 @@ private <V> ByteBuffer value(Cell<V> c)
         }
 
         /**
-         * Return the value of the column with the specified index.
+         * Return the value of the column with the specified index. If the column the value belongs to is masked with a
+         * {@link ColumnMask} and {@link #unmask} hasn't been specified, such mask will be applied to the value.
          *
          * @param index the column index
-         * @return the value of the column with the specified index
+         * @return the value of the column with the specified index, masked if its column is masked
          */
         public ByteBuffer getValue(int index)
         {
-            return values[index];
+            ByteBuffer value = values[index];

Review Comment:
   After some thought I believe we can keep the unmasked ordering with `ORDER BY` in a quite reasonable way. 
   Currently, columns that are not selected but are needed by `ORDER BY` post-ordering are temporarily included into the result, and later discarded. In case the ordering column is also selected, then it's only included once in the result set. 
   
   What we could do is including into the selection a `SimpleSelector` for masked columns that are needed for ordering, but making that selector return clear data. That clear column would be used for post-ordering and then discarded. That means that if the ordering column is also selected there would be two `SimpleSelector`s for that column, one for masked data and other for clear data. 
   
   This approach should only collect the masked/clear data for the columns that really need it, so it shouldn't incur in any performance cost if there aren't masked columns of `ORDER BY` clauses.
   
   I gave it a go in [the PR for CASSANDRA-18068](https://github.com/apache/cassandra/pull/2110).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1117321986


##########
src/java/org/apache/cassandra/cql3/selection/Selector.java:
##########
@@ -443,14 +447,21 @@ private <V> ByteBuffer value(Cell<V> c)
         }
 
         /**
-         * Return the value of the column with the specified index.
+         * Return the value of the column with the specified index. If the column the value belongs to is masked with a
+         * {@link ColumnMask} and {@link #unmask} hasn't been specified, such mask will be applied to the value.
          *
          * @param index the column index
-         * @return the value of the column with the specified index
+         * @return the value of the column with the specified index, masked if its column is masked
          */
         public ByteBuffer getValue(int index)
         {
-            return values[index];
+            ByteBuffer value = values[index];

Review Comment:
   I have changed [the patch for CASSANDRA-18068](https://github.com/apache/cassandra/pull/2110) to store the masked values on `InputRow` and thus reverted the changes on `SelectStatement`. I'll rebase this PR if that looks good.
   
   Regarding ordering, DDM usually preserves the order of the unmasked columns, mostly because trying to order based on the masked value of the column would require a kind of replica-side ordering that we want to avoid for performance reasons. 
   
   However, if the user explicitly uses `ORDER BY` to order selected partitions in the coordinator the rows will be ordered by the masked value of the clustering key. That's so because when the sorting is done the columns are already masked. I'm not sure whether that is acceptable. If it's not, I guess we could try to either:
   
   - Collect both masked and unmasked values, which adds complexity and memory pressure.
   - Mask after ordering, although that will probably cause problems with chained selectors.
   - Forbid `ORDER BY` if there are masked clustering columns
   
   What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1117321986


##########
src/java/org/apache/cassandra/cql3/selection/Selector.java:
##########
@@ -443,14 +447,21 @@ private <V> ByteBuffer value(Cell<V> c)
         }
 
         /**
-         * Return the value of the column with the specified index.
+         * Return the value of the column with the specified index. If the column the value belongs to is masked with a
+         * {@link ColumnMask} and {@link #unmask} hasn't been specified, such mask will be applied to the value.
          *
          * @param index the column index
-         * @return the value of the column with the specified index
+         * @return the value of the column with the specified index, masked if its column is masked
          */
         public ByteBuffer getValue(int index)
         {
-            return values[index];
+            ByteBuffer value = values[index];

Review Comment:
   I have changed [the patch for CASSANDRA-18068](https://github.com/apache/cassandra/pull/2110) to store the masked values on `InputRow` and thus reverted the changes on `SelectStatement`. I'll rebase this PR if that looks good.
   
   Regarding ordering, DDM usually preserves the order of the unmasked columns, mostly because trying to order based on the masked value of the column would require a kind of replica-side ordering that we want to avoid for performance reasons. 
   
   However, if the user explicitly uses `ORDER BY` to order selected partitions in the coordinator the rows will be ordered by the masked value of the clustering key. That's so because when the sorting is done the columns have already been masked. I'm not sure whether that is correct. If it's not, I guess that we could try to either:
   
   - Collect both masked and unmasked values, which adds complexity and memory pressure.
   - Mask after ordering, although that will probably cause problems with chained selectors.
   - Disable `ORDER BY` if there are masked clustering columns
   
   What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1105716591


##########
test/unit/org/apache/cassandra/cql3/functions/masking/UnmaskPermissionTest.java:
##########
@@ -0,0 +1,240 @@
+/*
+ * 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.cassandra.cql3.functions.masking;
+
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.exceptions.AuthenticationException;
+import com.datastax.driver.core.exceptions.UnauthorizedException;
+import org.apache.cassandra.cql3.CQLTester;
+import org.awaitility.Awaitility;
+import org.awaitility.core.ThrowingRunnable;
+
+import static java.lang.String.format;
+
+/**
+ * Tests the {@link org.apache.cassandra.auth.Permission#UNMASK} permission.
+ * <p>
+ * The permission is tested for a regular user with the {@code UNMASK} permissions on different resources,
+ * while also verifying the absence of side effects on other ordinary users, superusers and internal queries.
+ */
+public class UnmaskPermissionTest extends CQLTester
+{
+    private static final String CREATE_KEYSPACE = "CREATE KEYSPACE IF NOT EXISTS %s WITH replication = " +
+                                                  "{'class': 'SimpleStrategy', 'replication_factor': '1'}";
+    private static final String CREATE_TABLE = "CREATE TABLE IF NOT EXISTS %s.%s " +
+                                               "(k int, c int, v text MASKED WITH mask_replace('redacted'), " +
+                                               "PRIMARY KEY (k, c))";
+    private static final String INSERT = "INSERT INTO %s.%s (k, c, v) VALUES (?, ?, ?)";
+    private static final String SELECT_WILDCARD = "SELECT * FROM %s.%s";
+    private static final String SELECT_COLUMNS = "SELECT k, c, v FROM %s.%s";
+
+    private static final Object[] CLEAR_ROW = row(0, 0, "sensitive");
+    private static final Object[] MASKED_ROW = row(0, 0, "redacted");
+
+    private static final String KEYSPACE_1 = "mask_keyspace_1";
+    private static final String KEYSPACE_2 = "mask_keyspace_2";
+    private static final String TABLE_1 = "mask_table_1";
+    private static final String TABLE_2 = "mask_table_2";
+
+    private static final String USER = "ddm_user"; // user that will have their permissions changed
+    private static final String OTHER_USER = "ddm_ordinary_user"; // user that won't have their permissions altered
+    private static final String PASSWORD = "ddm_password";
+
+    @BeforeClass
+    public static void beforeClass()
+    {
+        CQLTester.setUpClass();
+        requireAuthentication();
+        requireNetwork();
+    }
+
+    @Before
+    public void before() throws Throwable
+    {
+        useSuperUser();
+
+        schemaChange(format(CREATE_KEYSPACE, KEYSPACE_1));
+        schemaChange(format(CREATE_KEYSPACE, KEYSPACE_2));
+
+        createTable(format(CREATE_TABLE, KEYSPACE_1, TABLE_1));
+        createTable(format(CREATE_TABLE, KEYSPACE_1, TABLE_2));
+        createTable(format(CREATE_TABLE, KEYSPACE_2, TABLE_1));
+
+        execute(format(INSERT, KEYSPACE_1, TABLE_1), CLEAR_ROW);
+        execute(format(INSERT, KEYSPACE_1, TABLE_2), CLEAR_ROW);
+        execute(format(INSERT, KEYSPACE_2, TABLE_1), CLEAR_ROW);
+
+        for (String user : Arrays.asList(USER, OTHER_USER))

Review Comment:
   Indeed we only need to create `OTHER_USER` once at the beginning of the class. However, the `CQLTester#executeNet` calls that we use here are instance methods, so they can't be called by a static `@BeforeClass` class method. As for `USER`, we drop it and recreate it on every test to reset its permissions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1112825133


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -121,6 +121,11 @@ public class SelectStatement implements CQLStatement.SingleKeyspaceCqlStatement
      */
     private final Comparator<List<ByteBuffer>> orderingComparator;
 
+    /**
+     * Whether masked columns should be unmasked.
+     */
+    private boolean unmask = true;

Review Comment:
   Would we want to add a note or annotate somewhere this so next time a review happens on that class that is clearer?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blerer commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "blerer (via GitHub)" <gi...@apache.org>.
blerer commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1116686899


##########
src/java/org/apache/cassandra/cql3/selection/Selector.java:
##########
@@ -443,14 +447,21 @@ private <V> ByteBuffer value(Cell<V> c)
         }
 
         /**
-         * Return the value of the column with the specified index.
+         * Return the value of the column with the specified index. If the column the value belongs to is masked with a
+         * {@link ColumnMask} and {@link #unmask} hasn't been specified, such mask will be applied to the value.
          *
          * @param index the column index
-         * @return the value of the column with the specified index
+         * @return the value of the column with the specified index, masked if its column is masked
          */
         public ByteBuffer getValue(int index)
         {
-            return values[index];
+            ByteBuffer value = values[index];

Review Comment:
   I looked into the code and my confusion was coming from the fact that `SimpleSelection` relies on `InputRow.getValues()`  (https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/cql3/selection/Selection.java#L478) and not `getValue`. Nevertheless, it introduce some weird behavior has one method will return masked values and not the other one. I think that we should be consistent. We could either do those operations at the `InputRow` level and do it when the value is set (which would allow us to simplify the logic at the SelectStatement level) or keep the logic in `SimpleSelector `.
   What is also unclear to me is how do we plan to handle the post ordering (https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java#L1059)? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1130844691


##########
test/unit/org/apache/cassandra/cql3/CQLTester.java:
##########
@@ -1721,6 +1726,24 @@ protected void assertColumnNames(UntypedResultSet result, String... expectedColu
         }
     }
 
+    protected void assertColumnNames(ResultSet result, String... expectedColumnNames)

Review Comment:
   This is a net version of the already existent ` assertColumnNames(UntypedResultSet, String[])`. Maybe we should rename it to `assertColumnNamesNet`? Although the different arguments seem to make the `*net` suffix unnecessary.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] adelapena commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "adelapena (via GitHub)" <gi...@apache.org>.
adelapena commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1128430069


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -121,6 +121,11 @@ public class SelectStatement implements CQLStatement.SingleKeyspaceCqlStatement
      */
     private final Comparator<List<ByteBuffer>> orderingComparator;
 
+    /**
+     * Whether masked columns should be unmasked.
+     */
+    private boolean unmask = true;

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] bereng commented on a diff in pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "bereng (via GitHub)" <gi...@apache.org>.
bereng commented on code in PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#discussion_r1131992742


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -256,6 +258,7 @@ public ResultMessage.Rows execute(QueryState state, QueryOptions options, long q
         int userLimit = getLimit(options);
         int userPerPartitionLimit = getPerPartitionLimit(options);
         int pageSize = options.getPageSize();
+        boolean unmask = state.getClientState().hasUnmaskPermission(table);

Review Comment:
   Nice catch



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] blerer commented on pull request #2125: CASSANDRA-18069 trunk: Add UNMASK permission

Posted by "blerer (via GitHub)" <gi...@apache.org>.
blerer commented on PR #2125:
URL: https://github.com/apache/cassandra/pull/2125#issuecomment-1463455924

   The only reason why spin asserts were required was that the Auth framework use caching for permissions and roles. If you disable those caches by using `DatabaseDescriptor.setPermissionsValidity(0);` and  `DatabaseDescriptor.setRolesValidity(0);` then changes to permissions and roles are instantaneous.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org