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/13 11:43:33 UTC

[GitHub] [cassandra] adelapena opened a new pull request, #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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

   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] blerer commented on a diff in pull request #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/DropFunctionStatement.java:
##########
@@ -120,9 +121,28 @@ public Keyspaces apply(Keyspaces schema)
         if (!dependentAggregates.isEmpty())
             throw ire("Function '%s' is still referenced by aggregates %s", name, dependentAggregates);
 
+        String dependentMasks = keyspace.tables.stream()
+                                               .filter(table -> hasDependingMask(table, function))
+                                               .map(table -> table.name)
+                                               .collect(joining(", "));

Review Comment:
   The approach that is mostly use within the code base is to delegate that type of call to the classes themselves. See `dependentAggregates` bellow.



-- 
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 pull request #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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

   > Could you add a test that use an UDF from another keyspace?
   
   That's already tested [here](https://github.com/apache/cassandra/blob/c41e567fbfddff8660e07109a0c6c7ada547465b/test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskWithUDFTest.java#L210-L221).


-- 
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 #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskWithUDFTest.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * 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 org.junit.Test;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.assertj.core.api.Assertions;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertNull;
+
+/**
+ * {@link ColumnMaskTester} for column masks using a UDF.
+ */
+public class ColumnMaskWithUDFTest extends ColumnMaskTester
+{
+    @Test
+    @SuppressWarnings("resource")
+    public void testUDF() throws Throwable
+    {
+        // create a table masked with and UDF and with a materialized view

Review Comment:
   and -> an



-- 
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 #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/DropFunctionStatement.java:
##########
@@ -120,9 +121,28 @@ public Keyspaces apply(Keyspaces schema)
         if (!dependentAggregates.isEmpty())
             throw ire("Function '%s' is still referenced by aggregates %s", name, dependentAggregates);
 
+        String dependentMasks = keyspace.tables.stream()
+                                               .filter(table -> hasDependingMask(table, function))
+                                               .map(table -> table.name)
+                                               .collect(joining(", "));
+
+        if (!dependentMasks.isEmpty())
+            throw ire("Function '%s' is still referenced by column masks in tables %s", name, dependentMasks);

Review Comment:
   I would add the keyspace name here "ks.table". Countless times I've amended error messages just bc a user had the same exact ks but: ks_dev, ks_testWhatever, ks_preProd, etc
   
   Yes it's immediately obvious to the user. But on captured output, logs, etc... I would still add it if easy enough.



-- 
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 #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskWithUDFTest.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * 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 org.junit.Test;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.assertj.core.api.Assertions;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertNull;
+
+/**
+ * {@link ColumnMaskTester} for column masks using a UDF.
+ */
+public class ColumnMaskWithUDFTest extends ColumnMaskTester
+{
+    @Test
+    @SuppressWarnings("resource")
+    public void testUDF() throws Throwable
+    {
+        // create a table masked with and UDF and with a materialized view

Review Comment:
   and -> a



-- 
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 #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/DropFunctionStatement.java:
##########
@@ -120,9 +121,28 @@ public Keyspaces apply(Keyspaces schema)
         if (!dependentAggregates.isEmpty())
             throw ire("Function '%s' is still referenced by aggregates %s", name, dependentAggregates);
 
+        String dependentMasks = keyspace.tables.stream()
+                                               .filter(table -> hasDependingMask(table, function))
+                                               .map(table -> table.name)
+                                               .collect(joining(", "));

Review Comment:
   I have just tried to do that with two new `KeyspaceMetadata#tablesUsingFunction(Function)` and `TableMetadata#dependsOn(Function)` methods.



-- 
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 #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskWithUDFTest.java:
##########
@@ -0,0 +1,284 @@
+/*
+ * 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 org.junit.Test;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.assertj.core.api.Assertions;
+
+import static java.lang.String.format;
+import static org.junit.Assert.assertNull;
+
+/**
+ * {@link ColumnMaskTester} for column masks using a UDF.
+ */
+public class ColumnMaskWithUDFTest extends ColumnMaskTester
+{
+    @Test
+    @SuppressWarnings("resource")
+    public void testUDF() throws Throwable
+    {
+        // create a table masked with and UDF and with a materialized view

Review Comment:
   Fixed, thanks



-- 
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 #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
test/distributed/org/apache/cassandra/distributed/test/ColumnMaskTest.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.cassandra.distributed.test;
+
+import java.util.function.Consumer;
+
+import org.junit.Test;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.auth.CassandraRoleManager;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.impl.RowUtil;
+
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.auth.CassandraRoleManager.DEFAULT_SUPERUSER_NAME;
+import static org.apache.cassandra.auth.CassandraRoleManager.DEFAULT_SUPERUSER_PASSWORD;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.awaitility.Awaitility.await;
+
+/**
+ * Tests for dynamic data masking.
+ */
+public class ColumnMaskTest extends TestBaseImpl
+{
+    /**
+     * Tests that column masks using UDFs are correctly loaded on startup.
+     * The UDF should be loaded before it's referenced by the mask.
+     */
+    @Test
+    public void testUDFMaskedColumnsOnStartup() throws Throwable
+    {
+        try (Cluster cluster = init(Cluster.build()
+                                           .withNodes(1)
+                                           .withConfig(conf -> conf.with(GOSSIP, NATIVE_PROTOCOL)
+                                                                   .set("user_defined_functions_enabled", "true")
+                                                                   .set("authenticator", "PasswordAuthenticator")
+                                                                   .set("authorizer", "CassandraAuthorizer"))
+                                           .start()))
+        {
+            IInvokableInstance node = cluster.get(1);
+
+            // create a table with a column masked with a UDF
+            cluster.schemaChange(withKeyspace("CREATE FUNCTION %s.f(column text, replacement text) " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS text " +
+                                              "LANGUAGE java " +
+                                              "AS 'return replacement;'"));
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.t (k int PRIMARY KEY, v text MASKED WITH %<s.f('redacted'))"));
+            node.executeInternal(withKeyspace("INSERT INTO %s.t(k, v) VALUES (0, 'secret')"));
+
+            // create a user without UNMASK permission
+            withAuthenticatedSession(node, DEFAULT_SUPERUSER_NAME, DEFAULT_SUPERUSER_PASSWORD, session -> {
+                session.execute("CREATE USER test WITH PASSWORD 'test'");
+                session.execute(withKeyspace("GRANT ALL ON KEYSPACE %s TO test"));
+                session.execute(withKeyspace("REVOKE UNMASK ON KEYSPACE %s FROM test"));
+            });
+
+            // restart the node, so the schema elements (the UDF and the mask) have to be loaded
+            node.shutdown().get();
+            node.startup();
+
+            // verify that the user without UNMASK permission can't see the clear data
+            withAuthenticatedSession(node, "test", "test", session -> {
+                ResultSet resultSet = session.execute(withKeyspace("SELECT * FROM %s.t"));
+                assertRows(RowUtil.toObjects(resultSet), row(0, "redacted"));
+            });
+        }
+    }
+
+    private static void withAuthenticatedSession(IInvokableInstance instance, String username, String password, Consumer<Session> consumer)
+    {

Review Comment:
   Not only. We would need something similar for authentication, but we can't use `callOnInstance` on upgrade tests. Also, as it's mentioned on that conversation we would also need a custom load balancing policy since the upgrade test involves multiple nodes. And, since upgrade tests are more sensitive to OOMs and timeouts, we would need to split the test. I have given it a go on top of 18069, [on this commit](https://github.com/apache/cassandra/pull/2125/commits/58e4cb9f98c462baa80ca1b6be43ccc376b34f7f). Some repeated runs can be found [here](https://app.circleci.com/pipelines/github/adelapena/cassandra/2650/workflows/a75e1832-bd9e-4eba-947b-e7782d6457d2).



-- 
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 #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/DropFunctionStatement.java:
##########
@@ -120,9 +121,28 @@ public Keyspaces apply(Keyspaces schema)
         if (!dependentAggregates.isEmpty())
             throw ire("Function '%s' is still referenced by aggregates %s", name, dependentAggregates);
 
+        String dependentMasks = keyspace.tables.stream()
+                                               .filter(table -> hasDependingMask(table, function))
+                                               .map(table -> table.name)
+                                               .collect(joining(", "));
+
+        if (!dependentMasks.isEmpty())
+            throw ire("Function '%s' is still referenced by column masks in tables %s", name, dependentMasks);

Review Comment:
   The `name` var is the qualified name of the function, so it already contains the name of the keyspace. That `name` var is created a few lines above, at the beginning of the 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] bereng commented on a diff in pull request #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/DropFunctionStatement.java:
##########
@@ -120,9 +121,28 @@ public Keyspaces apply(Keyspaces schema)
         if (!dependentAggregates.isEmpty())
             throw ire("Function '%s' is still referenced by aggregates %s", name, dependentAggregates);
 
+        String dependentMasks = keyspace.tables.stream()
+                                               .filter(table -> hasDependingMask(table, function))
+                                               .map(table -> table.name)
+                                               .collect(joining(", "));
+
+        if (!dependentMasks.isEmpty())
+            throw ire("Function '%s' is still referenced by column masks in tables %s", name, dependentMasks);

Review Comment:
   I would add the keyspace name here "ks.table". Countless times I've amended error messages just bc a user had the same exact ks but: ks_dev, ks_testWhatever, ks_preProd, etc



-- 
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 #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
test/distributed/org/apache/cassandra/distributed/test/ColumnMaskTest.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.cassandra.distributed.test;
+
+import java.util.function.Consumer;
+
+import org.junit.Test;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.auth.CassandraRoleManager;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.impl.RowUtil;
+
+import static java.util.concurrent.TimeUnit.MINUTES;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.apache.cassandra.auth.CassandraRoleManager.DEFAULT_SUPERUSER_NAME;
+import static org.apache.cassandra.auth.CassandraRoleManager.DEFAULT_SUPERUSER_PASSWORD;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+import static org.awaitility.Awaitility.await;
+
+/**
+ * Tests for dynamic data masking.
+ */
+public class ColumnMaskTest extends TestBaseImpl
+{
+    /**
+     * Tests that column masks using UDFs are correctly loaded on startup.
+     * The UDF should be loaded before it's referenced by the mask.
+     */
+    @Test
+    public void testUDFMaskedColumnsOnStartup() throws Throwable
+    {
+        try (Cluster cluster = init(Cluster.build()
+                                           .withNodes(1)
+                                           .withConfig(conf -> conf.with(GOSSIP, NATIVE_PROTOCOL)
+                                                                   .set("user_defined_functions_enabled", "true")
+                                                                   .set("authenticator", "PasswordAuthenticator")
+                                                                   .set("authorizer", "CassandraAuthorizer"))
+                                           .start()))
+        {
+            IInvokableInstance node = cluster.get(1);
+
+            // create a table with a column masked with a UDF
+            cluster.schemaChange(withKeyspace("CREATE FUNCTION %s.f(column text, replacement text) " +
+                                              "RETURNS NULL ON NULL INPUT " +
+                                              "RETURNS text " +
+                                              "LANGUAGE java " +
+                                              "AS 'return replacement;'"));
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.t (k int PRIMARY KEY, v text MASKED WITH %<s.f('redacted'))"));
+            node.executeInternal(withKeyspace("INSERT INTO %s.t(k, v) VALUES (0, 'secret')"));
+
+            // create a user without UNMASK permission
+            withAuthenticatedSession(node, DEFAULT_SUPERUSER_NAME, DEFAULT_SUPERUSER_PASSWORD, session -> {
+                session.execute("CREATE USER test WITH PASSWORD 'test'");
+                session.execute(withKeyspace("GRANT ALL ON KEYSPACE %s TO test"));
+                session.execute(withKeyspace("REVOKE UNMASK ON KEYSPACE %s FROM test"));
+            });
+
+            // restart the node, so the schema elements (the UDF and the mask) have to be loaded
+            node.shutdown().get();
+            node.startup();
+
+            // verify that the user without UNMASK permission can't see the clear data
+            withAuthenticatedSession(node, "test", "test", session -> {
+                ResultSet resultSet = session.execute(withKeyspace("SELECT * FROM %s.t"));
+                assertRows(RowUtil.toObjects(resultSet), row(0, "redacted"));
+            });
+        }
+    }
+
+    private static void withAuthenticatedSession(IInvokableInstance instance, String username, String password, Consumer<Session> consumer)
+    {

Review Comment:
   Is this the code we would need [here](https://github.com/apache/cassandra/pull/2125#pullrequestreview-1296963803)? :thinking: 



-- 
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 #2156: CASSANDRA-18071 trunk: Allow to use user-defined functions (UDF) as masking functions

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/DropFunctionStatement.java:
##########
@@ -120,9 +121,28 @@ public Keyspaces apply(Keyspaces schema)
         if (!dependentAggregates.isEmpty())
             throw ire("Function '%s' is still referenced by aggregates %s", name, dependentAggregates);
 
+        String dependentMasks = keyspace.tables.stream()
+                                               .filter(table -> hasDependingMask(table, function))
+                                               .map(table -> table.name)
+                                               .collect(joining(", "));
+
+        if (!dependentMasks.isEmpty())
+            throw ire("Function '%s' is still referenced by column masks in tables %s", name, dependentMasks);

Review Comment:
   Wfm 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