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/01/24 17:40:54 UTC

[GitHub] [cassandra] adelapena opened a new pull request, #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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

   This should be merged to a feature branch. We shouldn't include DDM with attached columns into trunk until we have CASSANDRA-18069 and, possibly, CASSANDRA-18070


-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
doc/cql3/CQL.textile:
##########
@@ -2552,8 +2560,8 @@ CQL distinguishes between _reserved_ and _non-reserved_ keywords. Reserved keywo
 | @VIEW@         | yes |
 | @WHERE@        | yes |
 | @WITH@         | yes |
-| @WRITETIME@    | no  |
-| @MAXWRITETIME@    | no  |
+| @WRITETIME@    | yes |

Review Comment:
   Is this change unrelated to this PR/ticket? just for my understanding



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/functions/masking/ColumnMask.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Terms;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.FunctionResolver;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.CassandraVersion;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * Dynamic data mask that can be applied to a schema column.
+ * <p>
+ * It consists on a partial application of a certain {@link MaskingFunction} to the values of a column, with the
+ * precondition that the type of any masked column is compatible with the type of the first argument of the function.
+ * <p>
+ * This partial application is meant to be associated to specific columns in the schema, acting as a mask for the values
+ * of those columns. It's associated to queries such as:
+ * <pre>
+ *    CREATE TABLE %t (k int PRIMARY KEY, v int MASKED WITH mask_inner(1, 1);
+ *    ALTER TABLE t ALTER v MASKED WITH mask_inner(2, 1);
+ *    ALTER TABLE t ALTER v DROP MASKED;
+ * </pre>
+ * Note that in the example above we are referencing the {@code mask_inner} function with two arguments. However, that
+ * CQL function actually has three arguments. The first argument is always ommitted when attaching the function to a
+ * schema column. The value of that first argument is always the value of the masked column, in this case an int.
+ */
+public class ColumnMask
+{
+    /** The CQL function used for masking. */
+    public final ScalarFunction function;
+
+    /** The values of the arguments of the partially applied masking function. */
+    public final List<ByteBuffer> partialArgumentValues;
+
+    public ColumnMask(ScalarFunction function, List<ByteBuffer> partialArgumentValues)
+    {
+        assert function.argTypes().size() == partialArgumentValues.size() + 1;
+        this.function = function;
+        this.partialArgumentValues = partialArgumentValues;
+    }
+
+    /**
+     * @return The types of the arguments of the partially applied masking function.
+     */
+    public List<AbstractType<?>> partialArgumentTypes()
+    {
+        List<AbstractType<?>> argTypes = function.argTypes();
+        return argTypes.size() == 1
+               ? Collections.emptyList()
+               : argTypes.subList(1, argTypes.size());
+    }
+
+    /**
+     * @return A copy of this mask for a version of its masked column that has its type reversed.
+     */
+    public ColumnMask withReversedType()
+    {
+        AbstractType<?> reversed = ReversedType.getInstance(function.argTypes().get(0));
+        List<AbstractType<?>> args = ImmutableList.<AbstractType<?>>builder()
+                                                  .add(reversed)
+                                                  .addAll(partialArgumentTypes())
+                                                  .build();
+        Function newFunction = FunctionResolver.get(function.name().keyspace, function.name(), args, null, null, null);
+        assert newFunction != null;
+        return new ColumnMask((ScalarFunction) newFunction, partialArgumentValues);
+    }
+
+    /**
+     * @param protocolVersion the used version of the transport protocol
+     * @param value           a column value to be masked
+     * @return the specified value after having been masked by the masked function
+     */
+    public ByteBuffer mask(ProtocolVersion protocolVersion, ByteBuffer value)
+    {
+        List<ByteBuffer> args = new ArrayList<>(partialArgumentValues.size() + 1);
+        args.add(value);
+        args.addAll(partialArgumentValues);
+        return function.execute(protocolVersion, args);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        ColumnMask mask = (ColumnMask) o;
+        return function.name().equals(mask.function.name())
+               && partialArgumentValues.equals(mask.partialArgumentValues);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(function.name(), partialArgumentValues);
+    }
+
+    @Override
+    public String toString()
+    {
+        List<AbstractType<?>> types = partialArgumentTypes();
+        List<String> arguments = new ArrayList<>(types.size());
+        for (int i = 0; i < types.size(); i++)
+        {
+            CQL3Type type = types.get(i).asCQL3Type();
+            ByteBuffer value = partialArgumentValues.get(i);
+            arguments.add(type.toCQLLiteral(value, ProtocolVersion.CURRENT));
+        }
+        return String.format("%s(%s)", function.name(), StringUtils.join(arguments, ", "));
+    }
+
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(" MASKED WITH ").append(toString());
+    }
+
+    /**
+     * @return {@code true} if we know that the current cluster supports masked columns, or {@code false} if it either
+     * doesn't support it due to the presence of not-upgrades nodes, or we don't know if such old nodes exist.
+     */
+    public static boolean clusterSupportsMaskedColumns()
+    {
+        if (!Gossiper.instance.isEnabled())
+            return false;
+
+        long timeout = DatabaseDescriptor.getWriteRpcTimeout(TimeUnit.MILLISECONDS);
+        CassandraVersion minVersion = Gossiper.instance.getMinVersion(timeout, TimeUnit.MILLISECONDS);
+        return minVersion != null && minVersion.familyLowerBound.get().compareTo(CassandraVersion.CASSANDRA_4_1) > 0;
+    }
+
+    /**
+     * A parsed but not prepared column mask.
+     */
+    public final static class Raw
+    {
+        private static final Joiner JOINER = Joiner.on(',');
+
+        public final FunctionName name;
+        public final List<Term.Raw> rawPartialArguments;
+
+        public Raw(FunctionName name, List<Term.Raw> rawPartialArguments)
+        {
+            this.name = name;
+            this.rawPartialArguments = rawPartialArguments;
+        }
+
+        public ColumnMask prepare(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            ScalarFunction function = findMaskingFunction(keyspace, table, column, type);
+
+            List<ByteBuffer> partialArguments = preparePartialArguments(keyspace, function);
+
+            return new ColumnMask(function, partialArguments);
+        }
+
+        private ScalarFunction findMaskingFunction(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            List<AssignmentTestable> args = new ArrayList<>(rawPartialArguments.size() + 1);
+            args.add(type);
+            args.addAll(rawPartialArguments);
+
+            Function function = FunctionResolver.get(keyspace, name, args, keyspace, table, type);
+
+            if (function == null)
+                throw invalidRequest("Unable to find masking function for %s, " +
+                                     "no declared function matches the signature %s",
+                                     column, this);
+
+            if (function.isAggregate())
+                throw invalidRequest("Aggregate function %s cannot be used for masking table columns", this);
+
+            if (!function.isNative())
+                throw invalidRequest("User defined function %s cannot be used for masking table columns", this);
+
+            if (!(function instanceof MaskingFunction))
+                throw invalidRequest("Not-masking function %s cannot be used for masking table columns", this);
+
+            if (!function.returnType().equals(type))

Review Comment:
   Note to self: will this fail on reversed types?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeColumnMaskingTest.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.upgrade;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.assertj.core.api.Assertions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+/**
+ * Tests that dynamic data masking (DDM) functions can be attached to table columns during a rolling upgrade involving
+ * nodes that don't include DDM.

Review Comment:
   can/cannot



##########
src/java/org/apache/cassandra/cql3/functions/masking/ColumnMask.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Terms;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.FunctionResolver;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.CassandraVersion;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * Dynamic data mask that can be applied to a schema column.
+ * <p>
+ * It consists on a partial application of a certain {@link MaskingFunction} to the values of a column, with the
+ * precondition that the type of any masked column is compatible with the type of the first argument of the function.
+ * <p>
+ * This partial application is meant to be associated to specific columns in the schema, acting as a mask for the values
+ * of those columns. It's associated to queries such as:
+ * <pre>
+ *    CREATE TABLE %t (k int PRIMARY KEY, v int MASKED WITH mask_inner(1, 1));
+ *    ALTER TABLE t ALTER v MASKED WITH mask_inner(2, 1);
+ *    ALTER TABLE t ALTER v DROP MASKED;
+ * </pre>
+ * Note that in the example above we are referencing the {@code mask_inner} function with two arguments. However, that
+ * CQL function actually has three arguments. The first argument is always ommitted when attaching the function to a
+ * schema column. The value of that first argument is always the value of the masked column, in this case an int.
+ */
+public class ColumnMask
+{
+    /** The CQL function used for masking. */
+    public final ScalarFunction function;
+
+    /** The values of the arguments of the partially applied masking function. */
+    public final List<ByteBuffer> partialArgumentValues;
+
+    public ColumnMask(ScalarFunction function, List<ByteBuffer> partialArgumentValues)
+    {
+        assert function.argTypes().size() == partialArgumentValues.size() + 1;
+        this.function = function;
+        this.partialArgumentValues = partialArgumentValues;
+    }
+
+    /**
+     * @return The types of the arguments of the partially applied masking function.
+     */
+    public List<AbstractType<?>> partialArgumentTypes()
+    {
+        List<AbstractType<?>> argTypes = function.argTypes();
+        return argTypes.size() == 1
+               ? Collections.emptyList()
+               : argTypes.subList(1, argTypes.size());
+    }
+
+    /**
+     * @return A copy of this mask for a version of its masked column that has its type reversed.
+     */
+    public ColumnMask withReversedType()
+    {
+        AbstractType<?> reversed = ReversedType.getInstance(function.argTypes().get(0));
+        List<AbstractType<?>> args = ImmutableList.<AbstractType<?>>builder()
+                                                  .add(reversed)
+                                                  .addAll(partialArgumentTypes())
+                                                  .build();
+        Function newFunction = FunctionResolver.get(function.name().keyspace, function.name(), args, null, null, null);
+        assert newFunction != null;
+        return new ColumnMask((ScalarFunction) newFunction, partialArgumentValues);
+    }
+
+    /**
+     * @param protocolVersion the used version of the transport protocol
+     * @param value           a column value to be masked
+     * @return the specified value after having been masked by the masked function
+     */
+    public ByteBuffer mask(ProtocolVersion protocolVersion, ByteBuffer value)
+    {
+        List<ByteBuffer> args = new ArrayList<>(partialArgumentValues.size() + 1);
+        args.add(value);
+        args.addAll(partialArgumentValues);
+        return function.execute(protocolVersion, args);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        ColumnMask mask = (ColumnMask) o;
+        return function.name().equals(mask.function.name())
+               && partialArgumentValues.equals(mask.partialArgumentValues);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(function.name(), partialArgumentValues);
+    }
+
+    @Override
+    public String toString()
+    {
+        List<AbstractType<?>> types = partialArgumentTypes();
+        List<String> arguments = new ArrayList<>(types.size());
+        for (int i = 0; i < types.size(); i++)
+        {
+            CQL3Type type = types.get(i).asCQL3Type();
+            ByteBuffer value = partialArgumentValues.get(i);
+            arguments.add(type.toCQLLiteral(value, ProtocolVersion.CURRENT));
+        }
+        return String.format("%s(%s)", function.name(), StringUtils.join(arguments, ", "));
+    }
+
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(" MASKED WITH ").append(toString());
+    }
+
+    /**
+     * @return {@code true} if we know that the current cluster supports masked columns, or {@code false} if it either
+     * doesn't support it due to the presence of not-upgrades nodes, or we don't know if such old nodes exist.
+     */
+    public static boolean clusterSupportsMaskedColumns()
+    {
+        if (!Gossiper.instance.isEnabled())
+            return false;
+
+        long timeout = DatabaseDescriptor.getWriteRpcTimeout(TimeUnit.MILLISECONDS);
+        CassandraVersion minVersion = Gossiper.instance.getMinVersion(timeout, TimeUnit.MILLISECONDS);
+        return minVersion != null && minVersion.familyLowerBound.get().compareTo(CassandraVersion.CASSANDRA_4_1) > 0;
+    }
+
+    /**
+     * A parsed but not prepared column mask.
+     */
+    public final static class Raw
+    {
+        private static final Joiner JOINER = Joiner.on(',');
+
+        public final FunctionName name;
+        public final List<Term.Raw> rawPartialArguments;
+
+        public Raw(FunctionName name, List<Term.Raw> rawPartialArguments)
+        {
+            this.name = name;
+            this.rawPartialArguments = rawPartialArguments;
+        }
+
+        public ColumnMask prepare(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            ScalarFunction function = findMaskingFunction(keyspace, table, column, type);
+
+            List<ByteBuffer> partialArguments = preparePartialArguments(keyspace, function);
+
+            return new ColumnMask(function, partialArguments);
+        }
+
+        private ScalarFunction findMaskingFunction(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            List<AssignmentTestable> args = new ArrayList<>(rawPartialArguments.size() + 1);
+            args.add(type);
+            args.addAll(rawPartialArguments);
+
+            Function function = FunctionResolver.get(keyspace, name, args, keyspace, table, type);
+
+            if (function == null)
+                throw invalidRequest("Unable to find masking function for %s, " +
+                                     "no declared function matches the signature %s",
+                                     column, this);
+
+            if (function.isAggregate())
+                throw invalidRequest("Aggregate function %s cannot be used for masking table columns", this);
+
+            if (!function.isNative())
+                throw invalidRequest("User defined function %s cannot be used for masking table columns", this);
+
+            if (!(function instanceof MaskingFunction))
+                throw invalidRequest("Not-masking function %s cannot be used for masking table columns", this);
+
+            if (!function.returnType().equals(type))
+                throw invalidRequest("Masking function %s return type is %s. " +
+                                     "This is different to the type of the masked column %s of type %s. " +
+                                     "Masking functions can only be attached to table columns " +
+                                     "if they return the same data type as the masked column.",
+                                     this, function.returnType().asCQL3Type(), column, type.asCQL3Type());
+
+            return (ScalarFunction) function;
+        }
+
+        private List<ByteBuffer> preparePartialArguments(String keyspace, ScalarFunction function)
+        {
+            // Note that there could be null arguments
+            List<ByteBuffer> arguments = new ArrayList<>(rawPartialArguments.size());
+
+            for (int i = 0; i < rawPartialArguments.size(); i++)
+            {
+                String term = rawPartialArguments.get(i).toString();
+                AbstractType<?> type = function.argTypes().get(i + 1);
+                arguments.add(Terms.asBytes(keyspace, term, type));
+            }
+
+            return arguments;
+        }
+
+        @Override
+        public String toString()
+        {
+            return name.toString() + '(' + JOINER.join(rawPartialArguments) + ')';

Review Comment:
   nit: Do we really need to optimize by having a static JOINER here? It seems more confusing that anything else the method is not called on production and is not really optimized either.



##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -187,7 +187,9 @@ public static Selection fromSelectors(TableMetadata table,
                                                                             factories,
                                                                             isJson);
 
-        return (processesSelection(selectables) || selectables.size() != selectedColumns.size() || hasGroupBy)
+        boolean hasMaskedColumns = selectedColumns.stream().anyMatch(ColumnMetadata::isMasked);

Review Comment:
   For masked column. `ColumnMetadata.processSelection` should return `true`. That will ensure that  `processesSelection(selectables)` returns `true` for masked columns and avoid the need for `hasMaskedColumns`.
   
   Regarding performance my suggestion would be to use a profiler and see if the use of streams is visible or not. Otherwise it will end up as a long and inefficient discussion.
   
   



##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -1153,10 +1153,14 @@ private Selection prepareSelection(TableMetadata table,
             if (hasGroupBy)
                 Guardrails.groupByEnabled.ensureEnabled(state);
 
+            boolean isJson = parameters.isJson;
+            boolean returnStaticContentOnPartitionWithNoRows = restrictions.returnStaticContentOnPartitionWithNoRows();
+
             if (selectables.isEmpty()) // wildcard query
             {
-                return hasGroupBy ? Selection.wildcardWithGroupBy(table, boundNames, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows())
-                                  : Selection.wildcard(table, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows());
+                return hasGroupBy || table.columns().stream().anyMatch(ColumnMetadata::isMasked)

Review Comment:
   I would have expected to have a `table.hasMaskedColumn() ` method to avoid code duplication each time people need that information and to make the code more readable.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -137,6 +139,11 @@ private SchemaKeyspace()
               + "kind text,"
               + "position int,"
               + "type text,"
+              + "mask_keyspace text,"
+              + "mask_name text,"
+              + "mask_argument_types frozen<list<text>>,"
+              + "mask_argument_values frozen<list<blob>>,"
+              + "mask_argument_nulls frozen<list<boolean>>," // arguments that are null

Review Comment:
   Using two tables has disadvantages in terms of complexity, space used and performance (we duplicate reads and writes when loading and storing the schema). However, the table is small and we don't access it very often, so the benefit of not breaking backwards compatibility could very well outweigh the costs, especially since we want to put this on current trunk, which is still 4.2.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java:
##########
@@ -158,6 +161,73 @@ public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
         }
     }
 
+    /**
+     * ALTER TABLE [IF EXISTS] <table> ALTER [IF EXISTS] <column> ( MASKED WITH <newMask> | DROP MASKED )
+     */
+    public static class MaskColumn extends AlterTableStatement
+    {
+        private final ColumnIdentifier columnName;
+        @Nullable
+        private final ColumnMask.Raw mask;
+        private final boolean ifColumnExists;
+
+        MaskColumn(String keyspaceName,
+                   String tableName,
+                   ColumnIdentifier columnName,
+                   @Nullable ColumnMask.Raw mask,
+                   boolean ifTableExists,
+                   boolean ifColumnExists)
+        {
+            super(keyspaceName, tableName, ifTableExists);
+            this.columnName = columnName;
+            this.mask = mask;
+            this.ifColumnExists = ifColumnExists;
+        }
+
+        @Override
+        public void validate(ClientState state)
+        {
+            super.validate(state);
+            validateMasking(mask == null
+                            ? "remove masking function from column"
+                            : "add masking function to column");
+        }
+
+        @Override
+        public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
+        {
+            ColumnMetadata column = table.getColumn(columnName);
+
+            if (column == null)
+            {
+                if (!ifColumnExists)
+                    throw ire("Column with name '%s' doesn't exist on table '%s'", columnName, tableName);
+
+                return keyspace;
+            }
+
+            ColumnMask columnMask = mask == null ? null : mask.prepare(keyspace.name, table.name, columnName, column.type);

Review Comment:
   Do we need to proceed if `columnMask` is null? The later `tableBuilder.alterColumnMask()` would throw anyway iiuc



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeColumnMaskingTest.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.upgrade;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.assertj.core.api.Assertions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+/**
+ * Tests that dynamic data masking (DDM) functions can be attached to table columns during a rolling upgrade involving
+ * nodes that don't include DDM.
+ */
+public class MixedModeColumnMaskingTest extends UpgradeTestBase
+{
+    @Test
+    public void testColumnMasking() throws Throwable
+    {
+        new TestCase()
+        .nodes(2)
+        .nodesToUpgrade(1, 2)
+        .upgradesToCurrentFrom(v30)
+        .withConfig(config -> config.with(GOSSIP))
+        .setup(cluster -> {
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.t (k int PRIMARY KEY, v int)"));
+            cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.t(k, v) VALUES (0, 7)"), ALL);
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {
+            if (node == 1)
+            {
+                ICoordinator coordinator = cluster.coordinator(1);
+
+                // create table with masked column
+                assertFails(coordinator,
+                            "CREATE TABLE %s.t1 (k int PRIMARY KEY, v int MASKED WITH DEFAULT)",
+                            "Cannot create a table with data masking functions during rolling upgrade");
+
+                // mask existing column
+                assertFails(coordinator,
+                            "ALTER TABLE %s.t ALTER v MASKED WITH DEFAULT",
+                            "Cannot add masking function to column during rolling upgrade");
+                assertColumnValue(cluster, "t", "v", 7);
+
+                // unmask existing column
+                assertFails(coordinator,
+                            "ALTER TABLE %s.t ALTER v DROP MASKED",
+                            "Cannot remove masking function from column during rolling upgrade");
+                assertColumnValue(cluster, "t", "v", 7);
+
+                // add new masked column
+                assertFails(coordinator,
+                            "ALTER TABLE %s.t ADD v2 int MASKED WITH DEFAULT",
+                            "Cannot add column with masking function during rolling upgrade");
+                assertColumnValue(cluster, "t", "v", 7);
+            }
+        }).runAfterClusterUpgrade(cluster -> {
+
+            // create table with masked column
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.t2 (k int PRIMARY KEY, v int MASKED WITH DEFAULT)"));
+            cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.t2(k, v) VALUES (0, 7)"), ALL);
+            assertColumnValue(cluster, "t2", "v", 0);
+
+            // mask existing column

Review Comment:
   It is. Or, more exactly, the block of code immediately below it is swapped with the next block of code :) 
   
   The intention is first masking an unmasked column, then unmasking it. 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] adelapena commented on a diff in pull request #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskInAnyPositionTester.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+import static java.lang.String.format;
+
+/**
+ * {@link ColumnMaskTester} verifying that masks can be applied to columns in any position (partition key columns,
+ * clustering key columns, static columns and regular columns). The columns of any depending materialized views should

Review Comment:
   This test class is focused on positions in the table, regardless of the column type. Counters as a data type are already tested in `ColumnMaskNativeTypesTest`.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/functions/masking/ColumnMask.java:
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Terms;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.FunctionResolver;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static java.lang.String.format;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * Dynamic data mask that can be applied to a schema column.
+ * <p>
+ * It consists on a partial application of a certain {@link MaskingFunction} to the values of a column, with the
+ * precondition that the type of any masked column is compatible with the type of the first argument of the function.
+ * <p>
+ * This partial application is meant to be associated to specific columns in the schema, acting as a mask for the values
+ * of those columns. It's associated to queries such as:
+ * <pre>
+ *    CREATE TABLE %t (k int PRIMARY KEY, v int MASKED WITH mask_inner(1, 1));
+ *    ALTER TABLE t ALTER v MASKED WITH mask_inner(2, 1);
+ *    ALTER TABLE t ALTER v DROP MASKED;
+ * </pre>
+ * Note that in the example above we are referencing the {@code mask_inner} function with two arguments. However, that
+ * CQL function actually has three arguments. The first argument is always ommitted when attaching the function to a
+ * schema column. The value of that first argument is always the value of the masked column, in this case an int.
+ */
+public class ColumnMask
+{
+    /** The CQL function used for masking. */
+    public final ScalarFunction function;
+
+    /** The values of the arguments of the partially applied masking function. */
+    public final List<ByteBuffer> partialArgumentValues;
+
+    public ColumnMask(ScalarFunction function, List<ByteBuffer> partialArgumentValues)
+    {
+        assert function.argTypes().size() == partialArgumentValues.size() + 1;
+        this.function = function;
+        this.partialArgumentValues = partialArgumentValues;
+    }
+
+    /**
+     * @return The types of the arguments of the partially applied masking function.
+     */
+    public List<AbstractType<?>> partialArgumentTypes()
+    {
+        List<AbstractType<?>> argTypes = function.argTypes();
+        return argTypes.size() == 1
+               ? Collections.emptyList()
+               : argTypes.subList(1, argTypes.size());
+    }
+
+    /**
+     * @return A copy of this mask for a version of its masked column that has its type reversed.
+     */
+    public ColumnMask withReversedType()
+    {
+        AbstractType<?> reversed = ReversedType.getInstance(function.argTypes().get(0));
+        List<AbstractType<?>> args = ImmutableList.<AbstractType<?>>builder()
+                                                  .add(reversed)
+                                                  .addAll(partialArgumentTypes())
+                                                  .build();
+        Function newFunction = FunctionResolver.get(function.name().keyspace, function.name(), args, null, null, null);
+        assert newFunction != null;
+        return new ColumnMask((ScalarFunction) newFunction, partialArgumentValues);
+    }
+
+    /**
+     * @param protocolVersion the used version of the transport protocol
+     * @param value           a column value to be masked
+     * @return the specified value after having been masked by the masked function
+     */
+    public ByteBuffer mask(ProtocolVersion protocolVersion, ByteBuffer value)
+    {
+        List<ByteBuffer> args = new ArrayList<>(partialArgumentValues.size() + 1);
+        args.add(value);
+        args.addAll(partialArgumentValues);
+        return function.execute(protocolVersion, args);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        ColumnMask mask = (ColumnMask) o;
+        return function.name().equals(mask.function.name())
+               && partialArgumentValues.equals(mask.partialArgumentValues);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(function.name(), partialArgumentValues);
+    }
+
+    @Override
+    public String toString()
+    {
+        List<AbstractType<?>> types = partialArgumentTypes();
+        List<String> arguments = new ArrayList<>(types.size());
+        for (int i = 0; i < types.size(); i++)
+        {
+            CQL3Type type = types.get(i).asCQL3Type();
+            ByteBuffer value = partialArgumentValues.get(i);
+            arguments.add(type.toCQLLiteral(value, ProtocolVersion.CURRENT));
+        }
+        return format("%s(%s)", function.name(), StringUtils.join(arguments, ", "));
+    }
+
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(" MASKED WITH ").append(toString());
+    }
+
+    /**
+     * A parsed but not prepared column mask.
+     */
+    public final static class Raw
+    {
+        public final FunctionName name;
+        public final List<Term.Raw> rawPartialArguments;
+
+        public Raw(FunctionName name, List<Term.Raw> rawPartialArguments)
+        {
+            this.name = name;
+            this.rawPartialArguments = rawPartialArguments;
+        }
+
+        public ColumnMask prepare(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            ScalarFunction function = findMaskingFunction(keyspace, table, column, type);
+

Review Comment:
   I'll remove them on commit.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -137,6 +139,11 @@ private SchemaKeyspace()
               + "kind text,"
               + "position int,"
               + "type text,"
+              + "mask_keyspace text,"
+              + "mask_name text,"
+              + "mask_argument_types frozen<list<text>>,"
+              + "mask_argument_values frozen<list<blob>>,"
+              + "mask_argument_nulls frozen<list<boolean>>," // arguments that are null

Review Comment:
   The separate table solution seems superior right? It has disadvantage and a clear advantage.



##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -137,6 +139,11 @@ private SchemaKeyspace()
               + "kind text,"
               + "position int,"
               + "type text,"
+              + "mask_keyspace text,"
+              + "mask_name text,"
+              + "mask_argument_types frozen<list<text>>,"
+              + "mask_argument_values frozen<list<blob>>,"
+              + "mask_argument_nulls frozen<list<boolean>>," // arguments that are null

Review Comment:
   The separate table solution seems superior right? It has no disadvantage and a clear advantage.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
doc/cql3/CQL.textile:
##########
@@ -2552,8 +2560,8 @@ CQL distinguishes between _reserved_ and _non-reserved_ keywords. Reserved keywo
 | @VIEW@         | yes |
 | @WHERE@        | yes |
 | @WITH@         | yes |
-| @WRITETIME@    | no  |
-| @MAXWRITETIME@    | no  |
+| @WRITETIME@    | yes |

Review Comment:
   It's unrelated, I fixed the alignement because the `WITHOUT` keyword is next to it without the additional, optional change mentioned [here](https://issues.apache.org/jira/browse/CASSANDRA-18068?focusedCommentId=17682967&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17682967). 



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -187,7 +187,9 @@ public static Selection fromSelectors(TableMetadata table,
                                                                             factories,
                                                                             isJson);
 
-        return (processesSelection(selectables) || selectables.size() != selectedColumns.size() || hasGroupBy)
+        boolean hasMaskedColumns = selectedColumns.stream().anyMatch(ColumnMetadata::isMasked);

Review Comment:
   As I try to argue in CASSANDRA-18248 I think it boils down to deciding if we see streams avoidance as a free lunch or not. Profiling would mean getting the actual change in iiuc.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -137,6 +139,11 @@ private SchemaKeyspace()
               + "kind text,"
               + "position int,"
               + "type text,"
+              + "mask_keyspace text,"
+              + "mask_name text,"
+              + "mask_argument_types frozen<list<text>>,"
+              + "mask_argument_values frozen<list<blob>>,"
+              + "mask_argument_nulls frozen<list<boolean>>," // arguments that are null

Review Comment:
   In light of the current discussions about downgradability it might be prudent to use a separate table for the masks, so the schema can be opened after an aborted upgrade. Otherwise, the new columns on the table column would fail at startup. wdyt?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/functions/masking/ColumnMask.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Terms;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.FunctionResolver;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.CassandraVersion;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * Dynamic data mask that can be applied to a schema column.
+ * <p>
+ * It consists on a partial application of a certain {@link MaskingFunction} to the values of a column, with the
+ * precondition that the type of any masked column is compatible with the type of the first argument of the function.
+ * <p>
+ * This partial application is meant to be associated to specific columns in the schema, acting as a mask for the values
+ * of those columns. It's associated to queries such as:
+ * <pre>
+ *    CREATE TABLE %t (k int PRIMARY KEY, v int MASKED WITH mask_inner(1, 1);

Review Comment:
   missing closing `) `?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -1153,10 +1153,14 @@ private Selection prepareSelection(TableMetadata table,
             if (hasGroupBy)
                 Guardrails.groupByEnabled.ensureEnabled(state);
 
+            boolean isJson = parameters.isJson;
+            boolean returnStaticContentOnPartitionWithNoRows = restrictions.returnStaticContentOnPartitionWithNoRows();
+
             if (selectables.isEmpty()) // wildcard query
             {
-                return hasGroupBy ? Selection.wildcardWithGroupBy(table, boundNames, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows())
-                                  : Selection.wildcard(table, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows());
+                return hasGroupBy || table.columns().stream().anyMatch(ColumnMetadata::isMasked)

Review Comment:
   Avoid streams?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -137,6 +139,11 @@ private SchemaKeyspace()
               + "kind text,"
               + "position int,"
               + "type text,"
+              + "mask_keyspace text,"
+              + "mask_name text,"
+              + "mask_argument_types frozen<list<text>>,"
+              + "mask_argument_values frozen<list<blob>>,"
+              + "mask_argument_nulls frozen<list<boolean>>," // arguments that are null

Review Comment:
   @blerer what do you think of this? Should we go with the separate 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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java:
##########
@@ -158,6 +162,77 @@ public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
         }
     }
 
+    /**
+     * ALTER TABLE [IF EXISTS] <table> ALTER [IF EXISTS] <column> ( MASKED WITH <newMask> | DROP MASKED )
+     */
+    public static class MaskColumn extends AlterTableStatement
+    {
+        private final ColumnIdentifier columnName;
+        @Nullable
+        private final ColumnMask.Raw mask;

Review Comment:
   Makes sense, renamed to `rawMask`.



##########
src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java:
##########
@@ -595,6 +684,10 @@ public static final class Raw extends CQLStatement.Raw
         // ADD
         private final List<AddColumns.Column> addedColumns = new ArrayList<>();
 
+        // ALTER MASK
+        private ColumnIdentifier maskedColumn = null;
+        private ColumnMask.Raw maskedColumnMask = null;

Review Comment:
   Renamed to `rawMask`.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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

   Nice one on splitting long 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] bereng commented on a diff in pull request #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -688,15 +695,58 @@ private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadat
     {
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
-            type = ((ReversedType) type).baseType;
+            type = ((ReversedType<?>) type).baseType;
 
-        builder.update(Columns)
+        Row.SimpleBuilder rowBuilder = builder.update(Columns)
                .row(table.name, column.name.toString())
                .add("column_name_bytes", column.name.bytes)
                .add("kind", column.kind.toString().toLowerCase())
                .add("position", column.position())
                .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
                .add("type", type.asCQL3Type().toString());
+
+        // Dynamic data masking functions shouldn't be attached to columns during rolling upgrades
+        // to avoid sending mutations with columns that are unknown to the old nodes.
+        ColumnMask mask = column.getMask();
+        if (ColumnMask.clusterSupportsMaskedColumns())
+        {
+            if (mask == null)
+            {
+                rowBuilder.delete("mask_keyspace")

Review Comment:
   "mask_keyspace" & friends could be constants



##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -688,15 +695,58 @@ private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadat
     {
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
-            type = ((ReversedType) type).baseType;
+            type = ((ReversedType<?>) type).baseType;
 
-        builder.update(Columns)
+        Row.SimpleBuilder rowBuilder = builder.update(Columns)
                .row(table.name, column.name.toString())
                .add("column_name_bytes", column.name.bytes)
                .add("kind", column.kind.toString().toLowerCase())
                .add("position", column.position())
                .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
                .add("type", type.asCQL3Type().toString());
+
+        // Dynamic data masking functions shouldn't be attached to columns during rolling upgrades
+        // to avoid sending mutations with columns that are unknown to the old nodes.
+        ColumnMask mask = column.getMask();
+        if (ColumnMask.clusterSupportsMaskedColumns())
+        {
+            if (mask == null)
+            {
+                rowBuilder.delete("mask_keyspace")

Review Comment:
   `mask_keyspace` & friends could be constants



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/functions/masking/ColumnMask.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Terms;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.FunctionResolver;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.CassandraVersion;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * Dynamic data mask that can be applied to a schema column.
+ * <p>
+ * It consists on a partial application of a certain {@link MaskingFunction} to the values of a column, with the
+ * precondition that the type of any masked column is compatible with the type of the first argument of the function.
+ * <p>
+ * This partial application is meant to be associated to specific columns in the schema, acting as a mask for the values
+ * of those columns. It's associated to queries such as:
+ * <pre>
+ *    CREATE TABLE %t (k int PRIMARY KEY, v int MASKED WITH mask_inner(1, 1);
+ *    ALTER TABLE t ALTER v MASKED WITH mask_inner(2, 1);
+ *    ALTER TABLE t ALTER v DROP MASKED;
+ * </pre>
+ * Note that in the example above we are referencing the {@code mask_inner} function with two arguments. However, that
+ * CQL function actually has three arguments. The first argument is always ommitted when attaching the function to a
+ * schema column. The value of that first argument is always the value of the masked column, in this case an int.
+ */
+public class ColumnMask
+{
+    /** The CQL function used for masking. */
+    public final ScalarFunction function;
+
+    /** The values of the arguments of the partially applied masking function. */
+    public final List<ByteBuffer> partialArgumentValues;
+
+    public ColumnMask(ScalarFunction function, List<ByteBuffer> partialArgumentValues)
+    {
+        assert function.argTypes().size() == partialArgumentValues.size() + 1;
+        this.function = function;
+        this.partialArgumentValues = partialArgumentValues;
+    }
+
+    /**
+     * @return The types of the arguments of the partially applied masking function.
+     */
+    public List<AbstractType<?>> partialArgumentTypes()
+    {
+        List<AbstractType<?>> argTypes = function.argTypes();
+        return argTypes.size() == 1
+               ? Collections.emptyList()
+               : argTypes.subList(1, argTypes.size());
+    }
+
+    /**
+     * @return A copy of this mask for a version of its masked column that has its type reversed.
+     */
+    public ColumnMask withReversedType()
+    {
+        AbstractType<?> reversed = ReversedType.getInstance(function.argTypes().get(0));
+        List<AbstractType<?>> args = ImmutableList.<AbstractType<?>>builder()
+                                                  .add(reversed)
+                                                  .addAll(partialArgumentTypes())
+                                                  .build();
+        Function newFunction = FunctionResolver.get(function.name().keyspace, function.name(), args, null, null, null);
+        assert newFunction != null;
+        return new ColumnMask((ScalarFunction) newFunction, partialArgumentValues);
+    }
+
+    /**
+     * @param protocolVersion the used version of the transport protocol
+     * @param value           a column value to be masked
+     * @return the specified value after having been masked by the masked function
+     */
+    public ByteBuffer mask(ProtocolVersion protocolVersion, ByteBuffer value)
+    {
+        List<ByteBuffer> args = new ArrayList<>(partialArgumentValues.size() + 1);
+        args.add(value);
+        args.addAll(partialArgumentValues);
+        return function.execute(protocolVersion, args);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        ColumnMask mask = (ColumnMask) o;
+        return function.name().equals(mask.function.name())
+               && partialArgumentValues.equals(mask.partialArgumentValues);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(function.name(), partialArgumentValues);
+    }
+
+    @Override
+    public String toString()
+    {
+        List<AbstractType<?>> types = partialArgumentTypes();
+        List<String> arguments = new ArrayList<>(types.size());
+        for (int i = 0; i < types.size(); i++)
+        {
+            CQL3Type type = types.get(i).asCQL3Type();
+            ByteBuffer value = partialArgumentValues.get(i);
+            arguments.add(type.toCQLLiteral(value, ProtocolVersion.CURRENT));
+        }
+        return String.format("%s(%s)", function.name(), StringUtils.join(arguments, ", "));
+    }
+
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(" MASKED WITH ").append(toString());
+    }
+
+    /**
+     * @return {@code true} if we know that the current cluster supports masked columns, or {@code false} if it either
+     * doesn't support it due to the presence of not-upgrades nodes, or we don't know if such old nodes exist.
+     */
+    public static boolean clusterSupportsMaskedColumns()
+    {
+        if (!Gossiper.instance.isEnabled())
+            return false;
+
+        long timeout = DatabaseDescriptor.getWriteRpcTimeout(TimeUnit.MILLISECONDS);
+        CassandraVersion minVersion = Gossiper.instance.getMinVersion(timeout, TimeUnit.MILLISECONDS);
+        return minVersion != null && minVersion.familyLowerBound.get().compareTo(CassandraVersion.CASSANDRA_4_1) > 0;
+    }
+
+    /**
+     * A parsed but not prepared column mask.
+     */
+    public final static class Raw
+    {
+        private static final Joiner JOINER = Joiner.on(',');
+
+        public final FunctionName name;
+        public final List<Term.Raw> rawPartialArguments;
+
+        public Raw(FunctionName name, List<Term.Raw> rawPartialArguments)
+        {
+            this.name = name;
+            this.rawPartialArguments = rawPartialArguments;
+        }
+
+        public ColumnMask prepare(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            ScalarFunction function = findMaskingFunction(keyspace, table, column, type);
+
+            List<ByteBuffer> partialArguments = preparePartialArguments(keyspace, function);
+
+            return new ColumnMask(function, partialArguments);
+        }
+
+        private ScalarFunction findMaskingFunction(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            List<AssignmentTestable> args = new ArrayList<>(rawPartialArguments.size() + 1);
+            args.add(type);
+            args.addAll(rawPartialArguments);
+
+            Function function = FunctionResolver.get(keyspace, name, args, keyspace, table, type);
+
+            if (function == null)
+                throw invalidRequest("Unable to find masking function for %s, " +
+                                     "no declared function matches the signature %s",
+                                     column, this);
+
+            if (function.isAggregate())
+                throw invalidRequest("Aggregate function %s cannot be used for masking table columns", this);
+
+            if (!function.isNative())
+                throw invalidRequest("User defined function %s cannot be used for masking table columns", this);
+
+            if (!(function instanceof MaskingFunction))
+                throw invalidRequest("Not-masking function %s cannot be used for masking table columns", this);
+
+            if (!function.returnType().equals(type))

Review Comment:
   I think it won't fail. We have several tests that use masking with reversed types, but I'm adding some more explicit tests.
   
   Nevertheless, this works because the masking functions added by CASSANDRA-17941 take care of returning a reversed type if their input is reversed. Those functions are the only ones that we allow here, so probably we are fine. However, we might need to revisit this when we add support for using UDFs as masking functions in CASSANDRA-18071. 



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java:
##########
@@ -158,6 +161,73 @@ public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
         }
     }
 
+    /**
+     * ALTER TABLE [IF EXISTS] <table> ALTER [IF EXISTS] <column> ( MASKED WITH <newMask> | DROP MASKED )
+     */
+    public static class MaskColumn extends AlterTableStatement
+    {
+        private final ColumnIdentifier columnName;
+        @Nullable
+        private final ColumnMask.Raw mask;
+        private final boolean ifColumnExists;
+
+        MaskColumn(String keyspaceName,
+                   String tableName,
+                   ColumnIdentifier columnName,
+                   @Nullable ColumnMask.Raw mask,
+                   boolean ifTableExists,
+                   boolean ifColumnExists)
+        {
+            super(keyspaceName, tableName, ifTableExists);
+            this.columnName = columnName;
+            this.mask = mask;
+            this.ifColumnExists = ifColumnExists;
+        }
+
+        @Override
+        public void validate(ClientState state)
+        {
+            super.validate(state);
+            validateMasking(mask == null
+                            ? "remove masking function from column"
+                            : "add masking function to column");
+        }
+
+        @Override
+        public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
+        {
+            ColumnMetadata column = table.getColumn(columnName);
+
+            if (column == null)
+            {
+                if (!ifColumnExists)
+                    throw ire("Column with name '%s' doesn't exist on table '%s'", columnName, tableName);
+
+                return keyspace;
+            }
+
+            ColumnMask columnMask = mask == null ? null : mask.prepare(keyspace.name, table.name, columnName, column.type);

Review Comment:
   We do need to proceed is `columnMask` is `null`, that's used for `DROP MASKED`. However, that makes me think that we can take a shortcut if the new mask (null or not) is equals to the old mask. Just added it.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/functions/masking/ColumnMask.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Terms;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.FunctionResolver;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.CassandraVersion;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * Dynamic data mask that can be applied to a schema column.
+ * <p>
+ * It consists on a partial application of a certain {@link MaskingFunction} to the values of a column, with the
+ * precondition that the type of any masked column is compatible with the type of the first argument of the function.
+ * <p>
+ * This partial application is meant to be associated to specific columns in the schema, acting as a mask for the values
+ * of those columns. It's associated to queries such as:
+ * <pre>
+ *    CREATE TABLE %t (k int PRIMARY KEY, v int MASKED WITH mask_inner(1, 1));
+ *    ALTER TABLE t ALTER v MASKED WITH mask_inner(2, 1);
+ *    ALTER TABLE t ALTER v DROP MASKED;
+ * </pre>
+ * Note that in the example above we are referencing the {@code mask_inner} function with two arguments. However, that
+ * CQL function actually has three arguments. The first argument is always ommitted when attaching the function to a
+ * schema column. The value of that first argument is always the value of the masked column, in this case an int.
+ */
+public class ColumnMask
+{
+    /** The CQL function used for masking. */
+    public final ScalarFunction function;
+
+    /** The values of the arguments of the partially applied masking function. */
+    public final List<ByteBuffer> partialArgumentValues;
+
+    public ColumnMask(ScalarFunction function, List<ByteBuffer> partialArgumentValues)
+    {
+        assert function.argTypes().size() == partialArgumentValues.size() + 1;
+        this.function = function;
+        this.partialArgumentValues = partialArgumentValues;
+    }
+
+    /**
+     * @return The types of the arguments of the partially applied masking function.
+     */
+    public List<AbstractType<?>> partialArgumentTypes()
+    {
+        List<AbstractType<?>> argTypes = function.argTypes();
+        return argTypes.size() == 1
+               ? Collections.emptyList()
+               : argTypes.subList(1, argTypes.size());
+    }
+
+    /**
+     * @return A copy of this mask for a version of its masked column that has its type reversed.
+     */
+    public ColumnMask withReversedType()
+    {
+        AbstractType<?> reversed = ReversedType.getInstance(function.argTypes().get(0));
+        List<AbstractType<?>> args = ImmutableList.<AbstractType<?>>builder()
+                                                  .add(reversed)
+                                                  .addAll(partialArgumentTypes())
+                                                  .build();
+        Function newFunction = FunctionResolver.get(function.name().keyspace, function.name(), args, null, null, null);
+        assert newFunction != null;
+        return new ColumnMask((ScalarFunction) newFunction, partialArgumentValues);
+    }
+
+    /**
+     * @param protocolVersion the used version of the transport protocol
+     * @param value           a column value to be masked
+     * @return the specified value after having been masked by the masked function
+     */
+    public ByteBuffer mask(ProtocolVersion protocolVersion, ByteBuffer value)
+    {
+        List<ByteBuffer> args = new ArrayList<>(partialArgumentValues.size() + 1);
+        args.add(value);
+        args.addAll(partialArgumentValues);
+        return function.execute(protocolVersion, args);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        ColumnMask mask = (ColumnMask) o;
+        return function.name().equals(mask.function.name())
+               && partialArgumentValues.equals(mask.partialArgumentValues);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(function.name(), partialArgumentValues);
+    }
+
+    @Override
+    public String toString()
+    {
+        List<AbstractType<?>> types = partialArgumentTypes();
+        List<String> arguments = new ArrayList<>(types.size());
+        for (int i = 0; i < types.size(); i++)
+        {
+            CQL3Type type = types.get(i).asCQL3Type();
+            ByteBuffer value = partialArgumentValues.get(i);
+            arguments.add(type.toCQLLiteral(value, ProtocolVersion.CURRENT));
+        }
+        return String.format("%s(%s)", function.name(), StringUtils.join(arguments, ", "));
+    }
+
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(" MASKED WITH ").append(toString());
+    }
+
+    /**
+     * @return {@code true} if we know that the current cluster supports masked columns, or {@code false} if it either
+     * doesn't support it due to the presence of not-upgrades nodes, or we don't know if such old nodes exist.
+     */
+    public static boolean clusterSupportsMaskedColumns()
+    {
+        if (!Gossiper.instance.isEnabled())
+            return false;
+
+        long timeout = DatabaseDescriptor.getWriteRpcTimeout(TimeUnit.MILLISECONDS);
+        CassandraVersion minVersion = Gossiper.instance.getMinVersion(timeout, TimeUnit.MILLISECONDS);
+        return minVersion != null && minVersion.familyLowerBound.get().compareTo(CassandraVersion.CASSANDRA_4_1) > 0;
+    }
+
+    /**
+     * A parsed but not prepared column mask.
+     */
+    public final static class Raw
+    {
+        private static final Joiner JOINER = Joiner.on(',');
+
+        public final FunctionName name;
+        public final List<Term.Raw> rawPartialArguments;
+
+        public Raw(FunctionName name, List<Term.Raw> rawPartialArguments)
+        {
+            this.name = name;
+            this.rawPartialArguments = rawPartialArguments;
+        }
+
+        public ColumnMask prepare(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            ScalarFunction function = findMaskingFunction(keyspace, table, column, type);
+
+            List<ByteBuffer> partialArguments = preparePartialArguments(keyspace, function);
+
+            return new ColumnMask(function, partialArguments);
+        }
+
+        private ScalarFunction findMaskingFunction(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            List<AssignmentTestable> args = new ArrayList<>(rawPartialArguments.size() + 1);
+            args.add(type);
+            args.addAll(rawPartialArguments);
+
+            Function function = FunctionResolver.get(keyspace, name, args, keyspace, table, type);
+
+            if (function == null)
+                throw invalidRequest("Unable to find masking function for %s, " +
+                                     "no declared function matches the signature %s",
+                                     column, this);
+
+            if (function.isAggregate())
+                throw invalidRequest("Aggregate function %s cannot be used for masking table columns", this);
+
+            if (!function.isNative())
+                throw invalidRequest("User defined function %s cannot be used for masking table columns", this);
+
+            if (!(function instanceof MaskingFunction))
+                throw invalidRequest("Not-masking function %s cannot be used for masking table columns", this);
+
+            if (!function.returnType().equals(type))
+                throw invalidRequest("Masking function %s return type is %s. " +
+                                     "This is different to the type of the masked column %s of type %s. " +
+                                     "Masking functions can only be attached to table columns " +
+                                     "if they return the same data type as the masked column.",
+                                     this, function.returnType().asCQL3Type(), column, type.asCQL3Type());
+
+            return (ScalarFunction) function;
+        }
+
+        private List<ByteBuffer> preparePartialArguments(String keyspace, ScalarFunction function)
+        {
+            // Note that there could be null arguments
+            List<ByteBuffer> arguments = new ArrayList<>(rawPartialArguments.size());
+
+            for (int i = 0; i < rawPartialArguments.size(); i++)
+            {
+                String term = rawPartialArguments.get(i).toString();
+                AbstractType<?> type = function.argTypes().get(i + 1);
+                arguments.add(Terms.asBytes(keyspace, term, type));
+            }
+
+            return arguments;
+        }
+
+        @Override
+        public String toString()
+        {
+            return name.toString() + '(' + JOINER.join(rawPartialArguments) + ')';

Review Comment:
   We don't, changed to `StringUtils.join`.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -688,15 +695,58 @@ private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadat
     {
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
-            type = ((ReversedType) type).baseType;
+            type = ((ReversedType<?>) type).baseType;
 
-        builder.update(Columns)
+        Row.SimpleBuilder rowBuilder = builder.update(Columns)
                .row(table.name, column.name.toString())
                .add("column_name_bytes", column.name.bytes)
                .add("kind", column.kind.toString().toLowerCase())
                .add("position", column.position())
                .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
                .add("type", type.asCQL3Type().toString());
+
+        // Dynamic data masking functions shouldn't be attached to columns during rolling upgrades
+        // to avoid sending mutations with columns that are unknown to the old nodes.
+        ColumnMask mask = column.getMask();
+        if (ColumnMask.clusterSupportsMaskedColumns())
+        {
+            if (mask == null)
+            {
+                rowBuilder.delete("mask_keyspace")
+                          .delete("mask_name")
+                          .delete("mask_argument_types")
+                          .delete("mask_argument_values")
+                          .delete("mask_argument_nulls");
+            }
+            else
+            {
+                FunctionName maskFunctionName = mask.function.name();
+
+                // Some arguments of the masking function can be null, but the CQL's list type that stores them doesn't

Review Comment:
   That's a great idea, it didn't occur to me. Changed to CQL strings.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -137,6 +139,11 @@ private SchemaKeyspace()
               + "kind text,"
               + "position int,"
               + "type text,"
+              + "mask_keyspace text,"
+              + "mask_name text,"
+              + "mask_argument_types frozen<list<text>>,"
+              + "mask_argument_values frozen<list<blob>>,"
+              + "mask_argument_nulls frozen<list<boolean>>," // arguments that are null

Review Comment:
   Using two tables has disadvantages in terms of complexity, space used and performance (we duplicate reads and writes when loading and storing the schema). However, the table is small and we don't write it very often, so the benefit of not breaking backwards compatibility could very well outweigh the costs, especially since we want to put this on current trunk, which is still 4.2.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java:
##########
@@ -158,6 +161,73 @@ public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
         }
     }
 
+    /**
+     * ALTER TABLE [IF EXISTS] <table> ALTER [IF EXISTS] <column> ( MASKED WITH <newMask> | DROP MASKED )
+     */
+    public static class MaskColumn extends AlterTableStatement
+    {
+        private final ColumnIdentifier columnName;
+        @Nullable
+        private final ColumnMask.Raw mask;
+        private final boolean ifColumnExists;
+
+        MaskColumn(String keyspaceName,
+                   String tableName,
+                   ColumnIdentifier columnName,
+                   @Nullable ColumnMask.Raw mask,
+                   boolean ifTableExists,
+                   boolean ifColumnExists)
+        {
+            super(keyspaceName, tableName, ifTableExists);
+            this.columnName = columnName;
+            this.mask = mask;
+            this.ifColumnExists = ifColumnExists;
+        }
+
+        @Override
+        public void validate(ClientState state)
+        {
+            super.validate(state);
+            validateMasking(mask == null
+                            ? "remove masking function from column"
+                            : "add masking function to column");
+        }
+
+        @Override
+        public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
+        {
+            ColumnMetadata column = table.getColumn(columnName);
+
+            if (column == null)
+            {
+                if (!ifColumnExists)
+                    throw ire("Column with name '%s' doesn't exist on table '%s'", columnName, tableName);
+
+                return keyspace;
+            }
+
+            ColumnMask columnMask = mask == null ? null : mask.prepare(keyspace.name, table.name, columnName, column.type);

Review Comment:
   Do we need to proceed if `columnMask` is null? `tableBuilder.alterColumnMask` would throw anyway iiuc



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskInAnyPositionTester.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+import static java.lang.String.format;
+
+/**
+ * {@link ColumnMaskTester} verifying that masks can be applied to columns in any position (partition key columns,
+ * clustering key columns, static columns and regular columns). The columns of any depending materialized views should
+ * be udpated accordingly.
+ */
+@RunWith(Parameterized.class)
+public abstract class ColumnMaskInAnyPositionTester extends ColumnMaskTester
+{
+    /** The column mask as expressed in CQL statements right after the {@code MASKED WITH} keywords. */
+    @Parameterized.Parameter
+    public String mask;
+
+    /** The type of the masked column */
+    @Parameterized.Parameter(1)
+    public String type;
+
+    /** The types of the tested masking function partial arguments. */
+    @Parameterized.Parameter(2)
+    public List<AbstractType<?>> argumentTypes;
+
+    /** The serialized values of the tested masking function partial arguments. */
+    @Parameterized.Parameter(3)
+    public List<ByteBuffer> argumentValues;
+
+    @Test
+    public void testCreateTableWithMaskedColumns() throws Throwable
+    {
+        // Nothing is masked
+        createTable("CREATE TABLE %s (k int, c int, r int, s int static, PRIMARY KEY(k, c))");
+        assertTableColumnsAreNotMasked("k", "c", "r", "s");
+
+        // Masked partition key
+        createTable(format("CREATE TABLE %%s (k %s MASKED WITH %s PRIMARY KEY, r int)", type, mask));
+        assertTableColumnsAreMasked("k");
+        assertTableColumnsAreNotMasked("r");
+
+        // Masked partition key component
+        createTable(format("CREATE TABLE %%s (k1 int, k2 %s MASKED WITH %s, r int, PRIMARY KEY(k1, k2))", type, mask));
+        assertTableColumnsAreMasked("k2");
+        assertTableColumnsAreNotMasked("k1", "r");
+
+        // Masked clustering key
+        createTable(format("CREATE TABLE %%s (k int, c %s MASKED WITH %s, r int, PRIMARY KEY (k, c))", type, mask));
+        assertTableColumnsAreMasked("c");
+        assertTableColumnsAreNotMasked("k", "r");
+
+        // Masked clustering key with reverse order
+        createTable(format("CREATE TABLE %%s (k int, c %s MASKED WITH %s, r int, PRIMARY KEY (k, c)) " +
+                           "WITH CLUSTERING ORDER BY (c DESC)", type, mask));
+        assertTableColumnsAreMasked("c");
+        assertTableColumnsAreNotMasked("k", "r");
+
+        // Masked clustering key component
+        createTable(format("CREATE TABLE %%s (k int, c1 int, c2 %s MASKED WITH %s, r int, PRIMARY KEY (k, c1, c2))", type, mask));
+        assertTableColumnsAreMasked("c2");
+        assertTableColumnsAreNotMasked("k", "c1", "r");
+
+        // Masked regular column
+        createTable(format("CREATE TABLE %%s (k int PRIMARY KEY, r1 %s MASKED WITH %s, r2 int)", type, mask));
+        assertTableColumnsAreMasked("r1");
+        assertTableColumnsAreNotMasked("k", "r2");
+
+        // Masked static column
+        createTable(format("CREATE TABLE %%s (k int, c int, r int, s %s STATIC MASKED WITH %s, PRIMARY KEY (k, c))", type, mask));
+        assertTableColumnsAreMasked("s");
+        assertTableColumnsAreNotMasked("k", "c", "r");
+
+        // Multiple masked columns
+        createTable(format("CREATE TABLE %%s (" +
+                           "k1 int, k2 %s MASKED WITH %s, " +
+                           "c1 int, c2 %s MASKED WITH %s, " +
+                           "r1 int, r2 %s MASKED WITH %s, " +
+                           "s1 int static, s2 %s static MASKED WITH %s, " +
+                           "PRIMARY KEY((k1, k2), c1, c2))",
+                           type, mask, type, mask, type, mask, type, mask));
+        assertTableColumnsAreMasked("k2", "c2", "r2", "s2");
+        assertTableColumnsAreNotMasked("k1", "c1", "r1", "s1");
+    }
+
+    @Test
+    public void testCreateTableWithMaskedColumnsAndMaterializedView() throws Throwable
+    {
+        createTable(format("CREATE TABLE %%s (" +
+                           "k1 int, k2 %s MASKED WITH %s, " +
+                           "c1 int, c2 %s MASKED WITH %s, " +
+                           "r1 int, r2 %s MASKED WITH %s, " +
+                           "s1 int static, s2 %s static MASKED WITH %s, " +
+                           "PRIMARY KEY((k1, k2), c1, c2))",
+                           type, mask, type, mask, type, mask, type, mask));
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT k1, k2, c1, c2, r1, r2 FROM %s " +
+                   "WHERE k1 IS NOT NULL AND k2 IS NOT NULL " +
+                   "AND c1 IS NOT NULL AND c2 IS NOT NULL " +
+                   "AND r1 IS NOT NULL AND r2 IS NOT NULL " +
+                   "PRIMARY KEY (r2, c2, c1, k2, k1)");
+
+        assertTableColumnsAreMasked("k2", "c2", "r2", "s2");
+        assertTableColumnsAreNotMasked("k1", "c1", "r1", "s1");
+
+        assertViewColumnsAreMasked("k2", "c2", "r2");
+        assertViewColumnsAreNotMasked("k1", "c1", "r1");
+    }
+
+    @Test
+    public void testAlterTableWithMaskedColumns() throws Throwable
+    {
+        // Create the table to be altered
+        createTable(format("CREATE TABLE %%s (k %s, c %<s, r1 %<s, r2 %<s MASKED WITH %s, r3 %s, s %<s static, " +
+                           "PRIMARY KEY (k, c))", type, mask, type));
+        assertTableColumnsAreMasked("r2");
+

Review Comment:
   Right, just added them, here and in the block immediately below.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -688,15 +695,58 @@ private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadat
     {
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
-            type = ((ReversedType) type).baseType;
+            type = ((ReversedType<?>) type).baseType;
 
-        builder.update(Columns)
+        Row.SimpleBuilder rowBuilder = builder.update(Columns)
                .row(table.name, column.name.toString())
                .add("column_name_bytes", column.name.bytes)
                .add("kind", column.kind.toString().toLowerCase())
                .add("position", column.position())
                .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
                .add("type", type.asCQL3Type().toString());
+
+        // Dynamic data masking functions shouldn't be attached to columns during rolling upgrades
+        // to avoid sending mutations with columns that are unknown to the old nodes.
+        ColumnMask mask = column.getMask();
+        if (ColumnMask.clusterSupportsMaskedColumns())

Review Comment:
   I don't know what that class would be. `QueryProcessor` is more CQL related and more related to the coordinator than to the cluster, so I don't see it there. Maybe in `Gossiper`, since the method calls it twice? But that would couple `Gossiper` with masking, which is not ideal.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -137,6 +139,11 @@ private SchemaKeyspace()
               + "kind text,"
               + "position int,"
               + "type text,"
+              + "mask_keyspace text,"
+              + "mask_name text,"
+              + "mask_argument_types frozen<list<text>>,"
+              + "mask_argument_values frozen<list<blob>>,"
+              + "mask_argument_nulls frozen<list<boolean>>," // arguments that are null

Review Comment:
   Just for the record it was commented yesterday 5.0 will be the choice granted by other changes such as removal of scripted UFFs.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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

   Latest changes LGTM +1


-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -187,7 +187,9 @@ public static Selection fromSelectors(TableMetadata table,
                                                                             factories,
                                                                             isJson);
 
-        return (processesSelection(selectables) || selectables.size() != selectedColumns.size() || hasGroupBy)
+        boolean hasMaskedColumns = selectedColumns.stream().anyMatch(ColumnMetadata::isMasked);

Review Comment:
   I disagree it's not in the hot path. It's not for a _specific query_ as yes, it gets prepared only once. But it's in the hot path in terms of how many total prepare requests we can handle. So in low prepared query scenarios _reusage_ it will have an impact.
   
   IOW prepare once and query 1K times we're ok. Prepare once and reuse a handful of times we're not ok. High env traffic env where the prepared stmnts cache rotates often we're not ok. Nodes handling multi-tenant traffic may be handling a high number of prepares given the many different loads/apps they may support and we're not ok.
   
   Maybe food for another ticket? I would still avoid it here if possible.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -1153,10 +1153,14 @@ private Selection prepareSelection(TableMetadata table,
             if (hasGroupBy)
                 Guardrails.groupByEnabled.ensureEnabled(state);
 
+            boolean isJson = parameters.isJson;
+            boolean returnStaticContentOnPartitionWithNoRows = restrictions.returnStaticContentOnPartitionWithNoRows();
+
             if (selectables.isEmpty()) // wildcard query
             {
-                return hasGroupBy ? Selection.wildcardWithGroupBy(table, boundNames, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows())
-                                  : Selection.wildcard(table, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows());
+                return hasGroupBy || table.columns().stream().anyMatch(ColumnMetadata::isMasked)

Review Comment:
   Same as before, it's not in the hot path.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -187,7 +187,9 @@ public static Selection fromSelectors(TableMetadata table,
                                                                             factories,
                                                                             isJson);
 
-        return (processesSelection(selectables) || selectables.size() != selectedColumns.size() || hasGroupBy)
+        boolean hasMaskedColumns = selectedColumns.stream().anyMatch(ColumnMetadata::isMasked);

Review Comment:
   This is run only once per query, so not in the hot path. It's relatively common to use streams while preparing the query.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
doc/cql3/CQL.textile:
##########
@@ -2552,8 +2560,8 @@ CQL distinguishes between _reserved_ and _non-reserved_ keywords. Reserved keywo
 | @VIEW@         | yes |
 | @WHERE@        | yes |
 | @WITH@         | yes |
-| @WRITETIME@    | no  |
-| @MAXWRITETIME@    | no  |
+| @WRITETIME@    | yes |

Review Comment:
   But why writetime changes from being non reserved to being reserved? Sorry if I am being thick.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskInAnyPositionTester.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+import static java.lang.String.format;
+
+/**
+ * {@link ColumnMaskTester} verifying that masks can be applied to columns in any position (partition key columns,
+ * clustering key columns, static columns and regular columns). The columns of any depending materialized views should
+ * be udpated accordingly.
+ */
+@RunWith(Parameterized.class)
+public abstract class ColumnMaskInAnyPositionTester extends ColumnMaskTester
+{
+    /** The column mask as expressed in CQL statements right after the {@code MASKED WITH} keywords. */
+    @Parameterized.Parameter
+    public String mask;
+
+    /** The type of the masked column */
+    @Parameterized.Parameter(1)
+    public String type;
+
+    /** The types of the tested masking function partial arguments. */
+    @Parameterized.Parameter(2)
+    public List<AbstractType<?>> argumentTypes;
+
+    /** The serialized values of the tested masking function partial arguments. */
+    @Parameterized.Parameter(3)
+    public List<ByteBuffer> argumentValues;
+
+    @Test
+    public void testCreateTableWithMaskedColumns() throws Throwable
+    {
+        // Nothing is masked
+        createTable("CREATE TABLE %s (k int, c int, r int, s int static, PRIMARY KEY(k, c))");
+        assertTableColumnsAreNotMasked("k", "c", "r", "s");
+
+        // Masked partition key
+        createTable(format("CREATE TABLE %%s (k %s MASKED WITH %s PRIMARY KEY, r int)", type, mask));
+        assertTableColumnsAreMasked("k");
+        assertTableColumnsAreNotMasked("r");
+
+        // Masked partition key component
+        createTable(format("CREATE TABLE %%s (k1 int, k2 %s MASKED WITH %s, r int, PRIMARY KEY(k1, k2))", type, mask));
+        assertTableColumnsAreMasked("k2");
+        assertTableColumnsAreNotMasked("k1", "r");
+
+        // Masked clustering key
+        createTable(format("CREATE TABLE %%s (k int, c %s MASKED WITH %s, r int, PRIMARY KEY (k, c))", type, mask));
+        assertTableColumnsAreMasked("c");
+        assertTableColumnsAreNotMasked("k", "r");
+
+        // Masked clustering key with reverse order
+        createTable(format("CREATE TABLE %%s (k int, c %s MASKED WITH %s, r int, PRIMARY KEY (k, c)) " +
+                           "WITH CLUSTERING ORDER BY (c DESC)", type, mask));
+        assertTableColumnsAreMasked("c");
+        assertTableColumnsAreNotMasked("k", "r");
+
+        // Masked clustering key component
+        createTable(format("CREATE TABLE %%s (k int, c1 int, c2 %s MASKED WITH %s, r int, PRIMARY KEY (k, c1, c2))", type, mask));
+        assertTableColumnsAreMasked("c2");
+        assertTableColumnsAreNotMasked("k", "c1", "r");
+
+        // Masked regular column
+        createTable(format("CREATE TABLE %%s (k int PRIMARY KEY, r1 %s MASKED WITH %s, r2 int)", type, mask));
+        assertTableColumnsAreMasked("r1");
+        assertTableColumnsAreNotMasked("k", "r2");
+
+        // Masked static column
+        createTable(format("CREATE TABLE %%s (k int, c int, r int, s %s STATIC MASKED WITH %s, PRIMARY KEY (k, c))", type, mask));
+        assertTableColumnsAreMasked("s");
+        assertTableColumnsAreNotMasked("k", "c", "r");
+
+        // Multiple masked columns
+        createTable(format("CREATE TABLE %%s (" +
+                           "k1 int, k2 %s MASKED WITH %s, " +
+                           "c1 int, c2 %s MASKED WITH %s, " +
+                           "r1 int, r2 %s MASKED WITH %s, " +
+                           "s1 int static, s2 %s static MASKED WITH %s, " +
+                           "PRIMARY KEY((k1, k2), c1, c2))",
+                           type, mask, type, mask, type, mask, type, mask));
+        assertTableColumnsAreMasked("k2", "c2", "r2", "s2");
+        assertTableColumnsAreNotMasked("k1", "c1", "r1", "s1");
+    }
+
+    @Test
+    public void testCreateTableWithMaskedColumnsAndMaterializedView() throws Throwable
+    {
+        createTable(format("CREATE TABLE %%s (" +
+                           "k1 int, k2 %s MASKED WITH %s, " +
+                           "c1 int, c2 %s MASKED WITH %s, " +
+                           "r1 int, r2 %s MASKED WITH %s, " +
+                           "s1 int static, s2 %s static MASKED WITH %s, " +
+                           "PRIMARY KEY((k1, k2), c1, c2))",
+                           type, mask, type, mask, type, mask, type, mask));
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT k1, k2, c1, c2, r1, r2 FROM %s " +
+                   "WHERE k1 IS NOT NULL AND k2 IS NOT NULL " +
+                   "AND c1 IS NOT NULL AND c2 IS NOT NULL " +
+                   "AND r1 IS NOT NULL AND r2 IS NOT NULL " +
+                   "PRIMARY KEY (r2, c2, c1, k2, k1)");
+
+        assertTableColumnsAreMasked("k2", "c2", "r2", "s2");
+        assertTableColumnsAreNotMasked("k1", "c1", "r1", "s1");
+
+        assertViewColumnsAreMasked("k2", "c2", "r2");
+        assertViewColumnsAreNotMasked("k1", "c1", "r1");
+    }
+
+    @Test
+    public void testAlterTableWithMaskedColumns() throws Throwable
+    {
+        // Create the table to be altered
+        createTable(format("CREATE TABLE %%s (k %s, c %<s, r1 %<s, r2 %<s MASKED WITH %s, r3 %s, s %<s static, " +
+                           "PRIMARY KEY (k, c))", type, mask, type));
+        assertTableColumnsAreMasked("r2");
+

Review Comment:
   No `assertTableColumnsAreNotMasked()`



##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskInAnyPositionTester.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+import static java.lang.String.format;
+
+/**
+ * {@link ColumnMaskTester} verifying that masks can be applied to columns in any position (partition key columns,
+ * clustering key columns, static columns and regular columns). The columns of any depending materialized views should
+ * be udpated accordingly.
+ */
+@RunWith(Parameterized.class)
+public abstract class ColumnMaskInAnyPositionTester extends ColumnMaskTester
+{
+    /** The column mask as expressed in CQL statements right after the {@code MASKED WITH} keywords. */
+    @Parameterized.Parameter
+    public String mask;
+
+    /** The type of the masked column */
+    @Parameterized.Parameter(1)
+    public String type;
+
+    /** The types of the tested masking function partial arguments. */
+    @Parameterized.Parameter(2)
+    public List<AbstractType<?>> argumentTypes;
+
+    /** The serialized values of the tested masking function partial arguments. */
+    @Parameterized.Parameter(3)
+    public List<ByteBuffer> argumentValues;
+
+    @Test
+    public void testCreateTableWithMaskedColumns() throws Throwable
+    {
+        // Nothing is masked
+        createTable("CREATE TABLE %s (k int, c int, r int, s int static, PRIMARY KEY(k, c))");
+        assertTableColumnsAreNotMasked("k", "c", "r", "s");
+
+        // Masked partition key
+        createTable(format("CREATE TABLE %%s (k %s MASKED WITH %s PRIMARY KEY, r int)", type, mask));
+        assertTableColumnsAreMasked("k");
+        assertTableColumnsAreNotMasked("r");
+
+        // Masked partition key component
+        createTable(format("CREATE TABLE %%s (k1 int, k2 %s MASKED WITH %s, r int, PRIMARY KEY(k1, k2))", type, mask));
+        assertTableColumnsAreMasked("k2");
+        assertTableColumnsAreNotMasked("k1", "r");
+
+        // Masked clustering key
+        createTable(format("CREATE TABLE %%s (k int, c %s MASKED WITH %s, r int, PRIMARY KEY (k, c))", type, mask));
+        assertTableColumnsAreMasked("c");
+        assertTableColumnsAreNotMasked("k", "r");
+
+        // Masked clustering key with reverse order
+        createTable(format("CREATE TABLE %%s (k int, c %s MASKED WITH %s, r int, PRIMARY KEY (k, c)) " +
+                           "WITH CLUSTERING ORDER BY (c DESC)", type, mask));
+        assertTableColumnsAreMasked("c");
+        assertTableColumnsAreNotMasked("k", "r");
+
+        // Masked clustering key component
+        createTable(format("CREATE TABLE %%s (k int, c1 int, c2 %s MASKED WITH %s, r int, PRIMARY KEY (k, c1, c2))", type, mask));
+        assertTableColumnsAreMasked("c2");
+        assertTableColumnsAreNotMasked("k", "c1", "r");
+
+        // Masked regular column
+        createTable(format("CREATE TABLE %%s (k int PRIMARY KEY, r1 %s MASKED WITH %s, r2 int)", type, mask));
+        assertTableColumnsAreMasked("r1");
+        assertTableColumnsAreNotMasked("k", "r2");
+
+        // Masked static column
+        createTable(format("CREATE TABLE %%s (k int, c int, r int, s %s STATIC MASKED WITH %s, PRIMARY KEY (k, c))", type, mask));
+        assertTableColumnsAreMasked("s");
+        assertTableColumnsAreNotMasked("k", "c", "r");
+
+        // Multiple masked columns
+        createTable(format("CREATE TABLE %%s (" +
+                           "k1 int, k2 %s MASKED WITH %s, " +
+                           "c1 int, c2 %s MASKED WITH %s, " +
+                           "r1 int, r2 %s MASKED WITH %s, " +
+                           "s1 int static, s2 %s static MASKED WITH %s, " +
+                           "PRIMARY KEY((k1, k2), c1, c2))",
+                           type, mask, type, mask, type, mask, type, mask));
+        assertTableColumnsAreMasked("k2", "c2", "r2", "s2");
+        assertTableColumnsAreNotMasked("k1", "c1", "r1", "s1");
+    }
+
+    @Test
+    public void testCreateTableWithMaskedColumnsAndMaterializedView() throws Throwable
+    {
+        createTable(format("CREATE TABLE %%s (" +
+                           "k1 int, k2 %s MASKED WITH %s, " +
+                           "c1 int, c2 %s MASKED WITH %s, " +
+                           "r1 int, r2 %s MASKED WITH %s, " +
+                           "s1 int static, s2 %s static MASKED WITH %s, " +
+                           "PRIMARY KEY((k1, k2), c1, c2))",
+                           type, mask, type, mask, type, mask, type, mask));
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT k1, k2, c1, c2, r1, r2 FROM %s " +
+                   "WHERE k1 IS NOT NULL AND k2 IS NOT NULL " +
+                   "AND c1 IS NOT NULL AND c2 IS NOT NULL " +
+                   "AND r1 IS NOT NULL AND r2 IS NOT NULL " +
+                   "PRIMARY KEY (r2, c2, c1, k2, k1)");
+
+        assertTableColumnsAreMasked("k2", "c2", "r2", "s2");
+        assertTableColumnsAreNotMasked("k1", "c1", "r1", "s1");
+
+        assertViewColumnsAreMasked("k2", "c2", "r2");
+        assertViewColumnsAreNotMasked("k1", "c1", "r1");
+    }
+
+    @Test
+    public void testAlterTableWithMaskedColumns() throws Throwable
+    {
+        // Create the table to be altered
+        createTable(format("CREATE TABLE %%s (k %s, c %<s, r1 %<s, r2 %<s MASKED WITH %s, r3 %s, s %<s static, " +
+                           "PRIMARY KEY (k, c))", type, mask, type));
+        assertTableColumnsAreMasked("r2");
+

Review Comment:
   No `assertTableColumnsAreNotMasked()`?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -688,15 +695,58 @@ private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadat
     {
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
-            type = ((ReversedType) type).baseType;
+            type = ((ReversedType<?>) type).baseType;
 
-        builder.update(Columns)
+        Row.SimpleBuilder rowBuilder = builder.update(Columns)
                .row(table.name, column.name.toString())
                .add("column_name_bytes", column.name.bytes)
                .add("kind", column.kind.toString().toLowerCase())
                .add("position", column.position())
                .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
                .add("type", type.asCQL3Type().toString());
+
+        // Dynamic data masking functions shouldn't be attached to columns during rolling upgrades
+        // to avoid sending mutations with columns that are unknown to the old nodes.
+        ColumnMask mask = column.getMask();
+        if (ColumnMask.clusterSupportsMaskedColumns())

Review Comment:
   As I am reading the PR I wonder if `clusterSupportsMaskedColumns()` should be moved to `QueryProcessor` or some other 'higher'/clusterish class as being buried in `ColumnMask` sounds weird to me.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -137,6 +139,11 @@ private SchemaKeyspace()
               + "kind text,"
               + "position int,"
               + "type text,"
+              + "mask_keyspace text,"
+              + "mask_name text,"
+              + "mask_argument_types frozen<list<text>>,"
+              + "mask_argument_values frozen<list<blob>>,"
+              + "mask_argument_nulls frozen<list<boolean>>," // arguments that are null

Review Comment:
   Yep I agree



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/functions/masking/ColumnMask.java:
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Terms;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.FunctionResolver;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+import static java.lang.String.format;
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * Dynamic data mask that can be applied to a schema column.
+ * <p>
+ * It consists on a partial application of a certain {@link MaskingFunction} to the values of a column, with the
+ * precondition that the type of any masked column is compatible with the type of the first argument of the function.
+ * <p>
+ * This partial application is meant to be associated to specific columns in the schema, acting as a mask for the values
+ * of those columns. It's associated to queries such as:
+ * <pre>
+ *    CREATE TABLE %t (k int PRIMARY KEY, v int MASKED WITH mask_inner(1, 1));
+ *    ALTER TABLE t ALTER v MASKED WITH mask_inner(2, 1);
+ *    ALTER TABLE t ALTER v DROP MASKED;
+ * </pre>
+ * Note that in the example above we are referencing the {@code mask_inner} function with two arguments. However, that
+ * CQL function actually has three arguments. The first argument is always ommitted when attaching the function to a
+ * schema column. The value of that first argument is always the value of the masked column, in this case an int.
+ */
+public class ColumnMask
+{
+    /** The CQL function used for masking. */
+    public final ScalarFunction function;
+
+    /** The values of the arguments of the partially applied masking function. */
+    public final List<ByteBuffer> partialArgumentValues;
+
+    public ColumnMask(ScalarFunction function, List<ByteBuffer> partialArgumentValues)
+    {
+        assert function.argTypes().size() == partialArgumentValues.size() + 1;
+        this.function = function;
+        this.partialArgumentValues = partialArgumentValues;
+    }
+
+    /**
+     * @return The types of the arguments of the partially applied masking function.
+     */
+    public List<AbstractType<?>> partialArgumentTypes()
+    {
+        List<AbstractType<?>> argTypes = function.argTypes();
+        return argTypes.size() == 1
+               ? Collections.emptyList()
+               : argTypes.subList(1, argTypes.size());
+    }
+
+    /**
+     * @return A copy of this mask for a version of its masked column that has its type reversed.
+     */
+    public ColumnMask withReversedType()
+    {
+        AbstractType<?> reversed = ReversedType.getInstance(function.argTypes().get(0));
+        List<AbstractType<?>> args = ImmutableList.<AbstractType<?>>builder()
+                                                  .add(reversed)
+                                                  .addAll(partialArgumentTypes())
+                                                  .build();
+        Function newFunction = FunctionResolver.get(function.name().keyspace, function.name(), args, null, null, null);
+        assert newFunction != null;
+        return new ColumnMask((ScalarFunction) newFunction, partialArgumentValues);
+    }
+
+    /**
+     * @param protocolVersion the used version of the transport protocol
+     * @param value           a column value to be masked
+     * @return the specified value after having been masked by the masked function
+     */
+    public ByteBuffer mask(ProtocolVersion protocolVersion, ByteBuffer value)
+    {
+        List<ByteBuffer> args = new ArrayList<>(partialArgumentValues.size() + 1);
+        args.add(value);
+        args.addAll(partialArgumentValues);
+        return function.execute(protocolVersion, args);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        ColumnMask mask = (ColumnMask) o;
+        return function.name().equals(mask.function.name())
+               && partialArgumentValues.equals(mask.partialArgumentValues);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(function.name(), partialArgumentValues);
+    }
+
+    @Override
+    public String toString()
+    {
+        List<AbstractType<?>> types = partialArgumentTypes();
+        List<String> arguments = new ArrayList<>(types.size());
+        for (int i = 0; i < types.size(); i++)
+        {
+            CQL3Type type = types.get(i).asCQL3Type();
+            ByteBuffer value = partialArgumentValues.get(i);
+            arguments.add(type.toCQLLiteral(value, ProtocolVersion.CURRENT));
+        }
+        return format("%s(%s)", function.name(), StringUtils.join(arguments, ", "));
+    }
+
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(" MASKED WITH ").append(toString());
+    }
+
+    /**
+     * A parsed but not prepared column mask.
+     */
+    public final static class Raw
+    {
+        public final FunctionName name;
+        public final List<Term.Raw> rawPartialArguments;
+
+        public Raw(FunctionName name, List<Term.Raw> rawPartialArguments)
+        {
+            this.name = name;
+            this.rawPartialArguments = rawPartialArguments;
+        }
+
+        public ColumnMask prepare(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            ScalarFunction function = findMaskingFunction(keyspace, table, column, type);
+

Review Comment:
   nit: why the extra blank lines?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/statements/SelectStatement.java:
##########
@@ -1153,10 +1153,14 @@ private Selection prepareSelection(TableMetadata table,
             if (hasGroupBy)
                 Guardrails.groupByEnabled.ensureEnabled(state);
 
+            boolean isJson = parameters.isJson;
+            boolean returnStaticContentOnPartitionWithNoRows = restrictions.returnStaticContentOnPartitionWithNoRows();
+
             if (selectables.isEmpty()) // wildcard query
             {
-                return hasGroupBy ? Selection.wildcardWithGroupBy(table, boundNames, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows())
-                                  : Selection.wildcard(table, parameters.isJson, restrictions.returnStaticContentOnPartitionWithNoRows());
+                return hasGroupBy || table.columns().stream().anyMatch(ColumnMetadata::isMasked)

Review Comment:
   Added that convenience `TableMetadata#hasMaskedColumns()` method, which is way cleaner. Since that's now a general purpose method that might called in any path I'm changing it to a loop.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
doc/cql3/CQL.textile:
##########
@@ -2552,8 +2560,8 @@ CQL distinguishes between _reserved_ and _non-reserved_ keywords. Reserved keywo
 | @VIEW@         | yes |
 | @WHERE@        | yes |
 | @WITH@         | yes |
-| @WRITETIME@    | no  |
-| @MAXWRITETIME@    | no  |
+| @WRITETIME@    | yes |

Review Comment:
   Oh, right, I was thinking on `MAXWRITETIME ` instead of `WRITETIME `. 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 pull request #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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

   LGTM +1


-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskTest.java:
##########
@@ -207,22 +207,22 @@ public void testColumnMaskingWithNotNativeFunction() throws Throwable
     }
 
     @Test
+    @SuppressWarnings("resource")

Review Comment:
   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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskTest.java:
##########
@@ -207,22 +207,22 @@ public void testColumnMaskingWithNotNativeFunction() throws Throwable
     }
 
     @Test
+    @SuppressWarnings("resource")

Review Comment:
   The sessions provided by `CQLTester#sessionNet()` are cached in `CQLTester.sessions`. If we close the session (manually or with try-with-resources) then the next call to `CQLTester#sessionNet()` will receive a closed session. 
   
   Note that the multiple direct uses of `CQLTester#sessionNet()` on other tests never try to close the session, and all the flavours of `CQLTester#sessionNet` don't close it either. That seems to indicate that not closing the reusable session is the intended use, even if the warning suppression is not commonly used.
   
   The sessions are later closed in `CQLTester.tearDownClass`.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -187,7 +187,9 @@ public static Selection fromSelectors(TableMetadata table,
                                                                             factories,
                                                                             isJson);
 
-        return (processesSelection(selectables) || selectables.size() != selectedColumns.size() || hasGroupBy)
+        boolean hasMaskedColumns = selectedColumns.stream().anyMatch(ColumnMetadata::isMasked);

Review Comment:
   Shall we avoid streams for perf reasons and move to a classic loop?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -187,7 +187,9 @@ public static Selection fromSelectors(TableMetadata table,
                                                                             factories,
                                                                             isJson);
 
-        return (processesSelection(selectables) || selectables.size() != selectedColumns.size() || hasGroupBy)
+        boolean hasMaskedColumns = selectedColumns.stream().anyMatch(ColumnMetadata::isMasked);

Review Comment:
   I don't think readability is impacted if streams are dropped, they would become quite simple loops. Given it's a cheap and immediate optimization I would still avoid streams here if it were my PR but I'll defer to your preference being the author. It's something that in multi tenancy env might be a thing and will make a nice LHF ticket. I'll be happy to create it at least for discussion [CASSANDRA-18248](https://issues.apache.org/jira/browse/CASSANDRA-18248)



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskNativeTypesTest.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.cql3.CQL3Type;
+
+import static java.lang.String.format;
+import static java.util.Collections.emptyList;
+
+/**
+ * {@link ColumnMaskTester} verifying that we can attach column masks to table columns with any native data type.
+ */
+@RunWith(Parameterized.class)
+public class ColumnMaskNativeTypesTest extends ColumnMaskTester
+{
+    /** The type of the column. */
+    @Parameterized.Parameter
+    public CQL3Type.Native type;
+
+    @Parameterized.Parameters(name = "type={0}")
+    public static Collection<Object[]> options()
+    {
+        List<Object[]> parameters = new ArrayList<>();
+        for (CQL3Type.Native type : CQL3Type.Native.values())
+        {
+            if (type != CQL3Type.Native.EMPTY)
+                parameters.add(new Object[]{ type });
+        }
+        return parameters;
+    }
+
+    @Test
+    public void testNativeDataTypes() throws Throwable
+    {
+        String def = format("%s MASKED WITH DEFAULT", type);
+        String keyDef = type == CQL3Type.Native.COUNTER || type == CQL3Type.Native.DURATION
+                        ? "int MASKED WITH DEFAULT" : def;
+
+        // Create table with masks
+        String table = createTable(format("CREATE TABLE %%s (k %s, c %<s, r %s, s %<s, PRIMARY KEY(k, c))", keyDef, def));

Review Comment:
   Is `s` supposed to be static?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -187,7 +187,9 @@ public static Selection fromSelectors(TableMetadata table,
                                                                             factories,
                                                                             isJson);
 
-        return (processesSelection(selectables) || selectables.size() != selectedColumns.size() || hasGroupBy)
+        boolean hasMaskedColumns = selectedColumns.stream().anyMatch(ColumnMetadata::isMasked);

Review Comment:
   I understand that the hot path where we avoid streams consists on that areas of the code that are repeated for every row or, more probably, column. The number of times those are hit is several orders of magnitude greater than in things that are used only to prepare the query, thereof the preference for performance over readability.
   
   It seems to me that at the moment we are not avoiding streams in query building. Just taking a quick look at the CQL package there are multiple usages of streams, for example:
   * [o.a.c.cql3/CQL3Type.java#L840](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/CQL3Type.java#L840)
   * [o.a.c.cql3/CQL3Type.java#L876](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/CQL3Type.java#L876)
   * [o.a.c.cql3/Lists.java#L133](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/Lists.java#L133)
   * [o.a.c.cql3/Sets.java#L123](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/Sets.java#L123)
   * [o.a.c.cql3/MultiColumnRelation.java#L225](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/MultiColumnRelation.java#L225)
   * [o.a.c.cql3/QueryProcessor.java#L481](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/QueryProcessor.java#L481)
   * [o.a.c.cql3/TokenRelation.java#L138](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/TokenRelation.java#L138)
   * [o.a.c.cql3/conditions/ColumnConditions.java#L75](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/conditions/ColumnConditions.java#L75)
   * [o.a.c.cql3/functions/AbstractFunction.java#L69](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/functions/AbstractFunction.java#L69)
   * [o.a.c.cql3/functions/FunctionResolver.java#L203](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/functions/FunctionResolver.java#L203)
   * [o.a.c.cql3/functions/UDAggregate.java#L97](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/functions/UDAggregate.java#L97)
   * [o.a.c.cql3/restrictions/StatementRestrictions.java#L380](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java#L380)
   * [o.a.c.cql3/selection/AbstractFunctionSelector.java#L72](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java#L72)
   * [o.a.c.cql3/selection/MapSelector.java#L113](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/selection/MapSelector.java#L113)
   * [o.a.c.cql3/selection/Selectable.java#L708](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/selection/Selectable.java#L708)
   * [o.a.c.cql3/selection/Selectable.java#L793](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/selection/Selectable.java#L793)
   * [o.a.c.cql3/selection/Selectable.java#L886](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/selection/Selectable.java#L886)
   * [o.a.c.cql3/selection/Selectable.java#L954](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/selection/Selectable.java#L954)
   * [o.a.c.cql3/selection/Selectable.java#L1024](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/selection/Selectable.java#L1024)
   * [o.a.c.cql3/statements/SelectStatement.java#L1278](https://github.com/apache/cassandra/blob/cassandra-4.1.0/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java#L1278)
   
   We should also check if queries are calling packages other than `o.a.c.cql3` that also use streams.
   
   I'd say that getting rid of those streams used on query building leans towards premature optimization. But, if we are concerned about the performance of streams on query building we should probably open a followup ticket to audit where streams are used.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java:
##########
@@ -158,6 +161,73 @@ public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
         }
     }
 
+    /**
+     * ALTER TABLE [IF EXISTS] <table> ALTER [IF EXISTS] <column> ( MASKED WITH <newMask> | DROP MASKED )
+     */
+    public static class MaskColumn extends AlterTableStatement
+    {
+        private final ColumnIdentifier columnName;
+        @Nullable
+        private final ColumnMask.Raw mask;
+        private final boolean ifColumnExists;
+
+        MaskColumn(String keyspaceName,
+                   String tableName,
+                   ColumnIdentifier columnName,
+                   @Nullable ColumnMask.Raw mask,
+                   boolean ifTableExists,
+                   boolean ifColumnExists)
+        {
+            super(keyspaceName, tableName, ifTableExists);
+            this.columnName = columnName;
+            this.mask = mask;
+            this.ifColumnExists = ifColumnExists;
+        }
+
+        @Override
+        public void validate(ClientState state)
+        {
+            super.validate(state);
+            validateMasking(mask == null
+                            ? "remove masking function from column"
+                            : "add masking function to column");
+        }
+
+        @Override
+        public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
+        {
+            ColumnMetadata column = table.getColumn(columnName);
+
+            if (column == null)
+            {
+                if (!ifColumnExists)
+                    throw ire("Column with name '%s' doesn't exist on table '%s'", columnName, tableName);
+
+                return keyspace;
+            }
+
+            ColumnMask columnMask = mask == null ? null : mask.prepare(keyspace.name, table.name, columnName, column.type);

Review Comment:
   Do we need to proceed if `columnMask` is null?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeColumnMaskingTest.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.upgrade;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.UpgradeableCluster;
+import org.apache.cassandra.distributed.api.ICoordinator;
+import org.assertj.core.api.Assertions;
+
+import static org.apache.cassandra.distributed.api.ConsistencyLevel.ALL;
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.shared.AssertUtils.assertRows;
+import static org.apache.cassandra.distributed.shared.AssertUtils.row;
+
+/**
+ * Tests that dynamic data masking (DDM) functions can be attached to table columns during a rolling upgrade involving
+ * nodes that don't include DDM.
+ */
+public class MixedModeColumnMaskingTest extends UpgradeTestBase
+{
+    @Test
+    public void testColumnMasking() throws Throwable
+    {
+        new TestCase()
+        .nodes(2)
+        .nodesToUpgrade(1, 2)
+        .upgradesToCurrentFrom(v30)
+        .withConfig(config -> config.with(GOSSIP))
+        .setup(cluster -> {
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.t (k int PRIMARY KEY, v int)"));
+            cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.t(k, v) VALUES (0, 7)"), ALL);
+        })
+        .runAfterNodeUpgrade((cluster, node) -> {
+            if (node == 1)
+            {
+                ICoordinator coordinator = cluster.coordinator(1);
+
+                // create table with masked column
+                assertFails(coordinator,
+                            "CREATE TABLE %s.t1 (k int PRIMARY KEY, v int MASKED WITH DEFAULT)",
+                            "Cannot create a table with data masking functions during rolling upgrade");
+
+                // mask existing column
+                assertFails(coordinator,
+                            "ALTER TABLE %s.t ALTER v MASKED WITH DEFAULT",
+                            "Cannot add masking function to column during rolling upgrade");
+                assertColumnValue(cluster, "t", "v", 7);
+
+                // unmask existing column
+                assertFails(coordinator,
+                            "ALTER TABLE %s.t ALTER v DROP MASKED",
+                            "Cannot remove masking function from column during rolling upgrade");
+                assertColumnValue(cluster, "t", "v", 7);
+
+                // add new masked column
+                assertFails(coordinator,
+                            "ALTER TABLE %s.t ADD v2 int MASKED WITH DEFAULT",
+                            "Cannot add column with masking function during rolling upgrade");
+                assertColumnValue(cluster, "t", "v", 7);
+            }
+        }).runAfterClusterUpgrade(cluster -> {
+
+            // create table with masked column
+            cluster.schemaChange(withKeyspace("CREATE TABLE %s.t2 (k int PRIMARY KEY, v int MASKED WITH DEFAULT)"));
+            cluster.coordinator(1).execute(withKeyspace("INSERT INTO %s.t2(k, v) VALUES (0, 7)"), ALL);
+            assertColumnValue(cluster, "t2", "v", 0);
+
+            // mask existing column

Review Comment:
   Comment is wrong?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskInAnyPositionTester.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.db.marshal.AbstractType;
+
+import static java.lang.String.format;
+
+/**
+ * {@link ColumnMaskTester} verifying that masks can be applied to columns in any position (partition key columns,
+ * clustering key columns, static columns and regular columns). The columns of any depending materialized views should

Review Comment:
   Should we consider counters as well?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java:
##########
@@ -158,6 +162,77 @@ public KeyspaceMetadata apply(KeyspaceMetadata keyspace, TableMetadata table)
         }
     }
 
+    /**
+     * ALTER TABLE [IF EXISTS] <table> ALTER [IF EXISTS] <column> ( MASKED WITH <newMask> | DROP MASKED )
+     */
+    public static class MaskColumn extends AlterTableStatement
+    {
+        private final ColumnIdentifier columnName;
+        @Nullable
+        private final ColumnMask.Raw mask;

Review Comment:
   Nit: We should may be name it `rawMask`. T prepare call  on the `mask` variable is a bit confusing otherwise.



##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -688,15 +695,58 @@ private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadat
     {
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
-            type = ((ReversedType) type).baseType;
+            type = ((ReversedType<?>) type).baseType;
 
-        builder.update(Columns)
+        Row.SimpleBuilder rowBuilder = builder.update(Columns)
                .row(table.name, column.name.toString())
                .add("column_name_bytes", column.name.bytes)
                .add("kind", column.kind.toString().toLowerCase())
                .add("position", column.position())
                .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
                .add("type", type.asCQL3Type().toString());
+
+        // Dynamic data masking functions shouldn't be attached to columns during rolling upgrades
+        // to avoid sending mutations with columns that are unknown to the old nodes.
+        ColumnMask mask = column.getMask();
+        if (ColumnMask.clusterSupportsMaskedColumns())
+        {
+            if (mask == null)
+            {
+                rowBuilder.delete("mask_keyspace")
+                          .delete("mask_name")
+                          .delete("mask_argument_types")
+                          .delete("mask_argument_values")
+                          .delete("mask_argument_nulls");
+            }
+            else
+            {
+                FunctionName maskFunctionName = mask.function.name();
+
+                // Some arguments of the masking function can be null, but the CQL's list type that stores them doesn't

Review Comment:
   Why not storing the elements as their CQL string? It will make the table more readable and allow us to store null within the same collection if I am not missing anything.



##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -688,15 +695,58 @@ private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadat
     {
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
-            type = ((ReversedType) type).baseType;
+            type = ((ReversedType<?>) type).baseType;
 
-        builder.update(Columns)
+        Row.SimpleBuilder rowBuilder = builder.update(Columns)
                .row(table.name, column.name.toString())
                .add("column_name_bytes", column.name.bytes)
                .add("kind", column.kind.toString().toLowerCase())
                .add("position", column.position())
                .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
                .add("type", type.asCQL3Type().toString());
+
+        // Dynamic data masking functions shouldn't be attached to columns during rolling upgrades
+        // to avoid sending mutations with columns that are unknown to the old nodes.
+        ColumnMask mask = column.getMask();
+        if (ColumnMask.clusterSupportsMaskedColumns())
+        {
+            if (mask == null)
+            {
+                rowBuilder.delete("mask_keyspace")

Review Comment:
   I agree.



##########
src/java/org/apache/cassandra/cql3/statements/schema/AlterTableStatement.java:
##########
@@ -595,6 +684,10 @@ public static final class Raw extends CQLStatement.Raw
         // ADD
         private final List<AddColumns.Column> addedColumns = new ArrayList<>();
 
+        // ALTER MASK
+        private ColumnIdentifier maskedColumn = null;
+        private ColumnMask.Raw maskedColumnMask = null;

Review Comment:
   `rawMask `instead of `maskedColumnMask `?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskTest.java:
##########
@@ -207,22 +207,22 @@ public void testColumnMaskingWithNotNativeFunction() throws Throwable
     }
 
     @Test
+    @SuppressWarnings("resource")

Review Comment:
   What's the point of of replacing the auto-closable by this?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/selection/Selection.java:
##########
@@ -187,7 +187,9 @@ public static Selection fromSelectors(TableMetadata table,
                                                                             factories,
                                                                             isJson);
 
-        return (processesSelection(selectables) || selectables.size() != selectedColumns.size() || hasGroupBy)
+        boolean hasMaskedColumns = selectedColumns.stream().anyMatch(ColumnMetadata::isMasked);

Review Comment:
   Makes sense, fixed `ColumnMetadata#processSelection` so the stream/loop is not needed.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/cql3/functions/masking/ColumnMask.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Terms;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.FunctionResolver;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.CassandraVersion;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * Dynamic data mask that can be applied to a schema column.
+ * <p>
+ * It consists on a partial application of a certain {@link MaskingFunction} to the values of a column, with the
+ * precondition that the type of any masked column is compatible with the type of the first argument of the function.
+ * <p>
+ * This partial application is meant to be associated to specific columns in the schema, acting as a mask for the values
+ * of those columns. It's associated to queries such as:
+ * <pre>
+ *    CREATE TABLE %t (k int PRIMARY KEY, v int MASKED WITH mask_inner(1, 1);
+ *    ALTER TABLE t ALTER v MASKED WITH mask_inner(2, 1);
+ *    ALTER TABLE t ALTER v DROP MASKED;
+ * </pre>
+ * Note that in the example above we are referencing the {@code mask_inner} function with two arguments. However, that
+ * CQL function actually has three arguments. The first argument is always ommitted when attaching the function to a
+ * schema column. The value of that first argument is always the value of the masked column, in this case an int.
+ */
+public class ColumnMask
+{
+    /** The CQL function used for masking. */
+    public final ScalarFunction function;
+
+    /** The values of the arguments of the partially applied masking function. */
+    public final List<ByteBuffer> partialArgumentValues;
+
+    public ColumnMask(ScalarFunction function, List<ByteBuffer> partialArgumentValues)
+    {
+        assert function.argTypes().size() == partialArgumentValues.size() + 1;
+        this.function = function;
+        this.partialArgumentValues = partialArgumentValues;
+    }
+
+    /**
+     * @return The types of the arguments of the partially applied masking function.
+     */
+    public List<AbstractType<?>> partialArgumentTypes()
+    {
+        List<AbstractType<?>> argTypes = function.argTypes();
+        return argTypes.size() == 1
+               ? Collections.emptyList()
+               : argTypes.subList(1, argTypes.size());
+    }
+
+    /**
+     * @return A copy of this mask for a version of its masked column that has its type reversed.
+     */
+    public ColumnMask withReversedType()
+    {
+        AbstractType<?> reversed = ReversedType.getInstance(function.argTypes().get(0));
+        List<AbstractType<?>> args = ImmutableList.<AbstractType<?>>builder()
+                                                  .add(reversed)
+                                                  .addAll(partialArgumentTypes())
+                                                  .build();
+        Function newFunction = FunctionResolver.get(function.name().keyspace, function.name(), args, null, null, null);
+        assert newFunction != null;
+        return new ColumnMask((ScalarFunction) newFunction, partialArgumentValues);
+    }
+
+    /**
+     * @param protocolVersion the used version of the transport protocol
+     * @param value           a column value to be masked
+     * @return the specified value after having been masked by the masked function
+     */
+    public ByteBuffer mask(ProtocolVersion protocolVersion, ByteBuffer value)
+    {
+        List<ByteBuffer> args = new ArrayList<>(partialArgumentValues.size() + 1);
+        args.add(value);
+        args.addAll(partialArgumentValues);
+        return function.execute(protocolVersion, args);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        ColumnMask mask = (ColumnMask) o;
+        return function.name().equals(mask.function.name())
+               && partialArgumentValues.equals(mask.partialArgumentValues);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(function.name(), partialArgumentValues);
+    }
+
+    @Override
+    public String toString()
+    {
+        List<AbstractType<?>> types = partialArgumentTypes();
+        List<String> arguments = new ArrayList<>(types.size());
+        for (int i = 0; i < types.size(); i++)
+        {
+            CQL3Type type = types.get(i).asCQL3Type();
+            ByteBuffer value = partialArgumentValues.get(i);
+            arguments.add(type.toCQLLiteral(value, ProtocolVersion.CURRENT));
+        }
+        return String.format("%s(%s)", function.name(), StringUtils.join(arguments, ", "));
+    }
+
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(" MASKED WITH ").append(toString());
+    }
+
+    /**
+     * @return {@code true} if we know that the current cluster supports masked columns, or {@code false} if it either
+     * doesn't support it due to the presence of not-upgrades nodes, or we don't know if such old nodes exist.
+     */
+    public static boolean clusterSupportsMaskedColumns()
+    {
+        if (!Gossiper.instance.isEnabled())
+            return false;
+
+        long timeout = DatabaseDescriptor.getWriteRpcTimeout(TimeUnit.MILLISECONDS);
+        CassandraVersion minVersion = Gossiper.instance.getMinVersion(timeout, TimeUnit.MILLISECONDS);
+        return minVersion != null && minVersion.familyLowerBound.get().compareTo(CassandraVersion.CASSANDRA_4_1) > 0;
+    }
+
+    /**
+     * A parsed but not prepared column mask.
+     */
+    public final static class Raw
+    {
+        private static final Joiner JOINER = Joiner.on(',');
+
+        public final FunctionName name;
+        public final List<Term.Raw> rawPartialArguments;
+
+        public Raw(FunctionName name, List<Term.Raw> rawPartialArguments)
+        {
+            this.name = name;
+            this.rawPartialArguments = rawPartialArguments;
+        }
+
+        public ColumnMask prepare(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            ScalarFunction function = findMaskingFunction(keyspace, table, column, type);
+
+            List<ByteBuffer> partialArguments = preparePartialArguments(keyspace, function);
+
+            return new ColumnMask(function, partialArguments);
+        }
+
+        private ScalarFunction findMaskingFunction(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            List<AssignmentTestable> args = new ArrayList<>(rawPartialArguments.size() + 1);
+            args.add(type);
+            args.addAll(rawPartialArguments);
+
+            Function function = FunctionResolver.get(keyspace, name, args, keyspace, table, type);
+
+            if (function == null)
+                throw invalidRequest("Unable to find masking function for %s, " +
+                                     "no declared function matches the signature %s",
+                                     column, this);
+
+            if (function.isAggregate())
+                throw invalidRequest("Aggregate function %s cannot be used for masking table columns", this);
+
+            if (!function.isNative())
+                throw invalidRequest("User defined function %s cannot be used for masking table columns", this);
+
+            if (!(function instanceof MaskingFunction))
+                throw invalidRequest("Not-masking function %s cannot be used for masking table columns", this);
+
+            if (!function.returnType().equals(type))

Review Comment:
   > take care of returning a reversed type if their input is reversed
   ^ That is what I was wondering thx



##########
src/java/org/apache/cassandra/cql3/functions/masking/ColumnMask.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.commons.lang3.StringUtils;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.CqlBuilder;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.Terms;
+import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.FunctionResolver;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.CassandraVersion;
+
+import static org.apache.cassandra.cql3.statements.RequestValidations.invalidRequest;
+
+/**
+ * Dynamic data mask that can be applied to a schema column.
+ * <p>
+ * It consists on a partial application of a certain {@link MaskingFunction} to the values of a column, with the
+ * precondition that the type of any masked column is compatible with the type of the first argument of the function.
+ * <p>
+ * This partial application is meant to be associated to specific columns in the schema, acting as a mask for the values
+ * of those columns. It's associated to queries such as:
+ * <pre>
+ *    CREATE TABLE %t (k int PRIMARY KEY, v int MASKED WITH mask_inner(1, 1);
+ *    ALTER TABLE t ALTER v MASKED WITH mask_inner(2, 1);
+ *    ALTER TABLE t ALTER v DROP MASKED;
+ * </pre>
+ * Note that in the example above we are referencing the {@code mask_inner} function with two arguments. However, that
+ * CQL function actually has three arguments. The first argument is always ommitted when attaching the function to a
+ * schema column. The value of that first argument is always the value of the masked column, in this case an int.
+ */
+public class ColumnMask
+{
+    /** The CQL function used for masking. */
+    public final ScalarFunction function;
+
+    /** The values of the arguments of the partially applied masking function. */
+    public final List<ByteBuffer> partialArgumentValues;
+
+    public ColumnMask(ScalarFunction function, List<ByteBuffer> partialArgumentValues)
+    {
+        assert function.argTypes().size() == partialArgumentValues.size() + 1;
+        this.function = function;
+        this.partialArgumentValues = partialArgumentValues;
+    }
+
+    /**
+     * @return The types of the arguments of the partially applied masking function.
+     */
+    public List<AbstractType<?>> partialArgumentTypes()
+    {
+        List<AbstractType<?>> argTypes = function.argTypes();
+        return argTypes.size() == 1
+               ? Collections.emptyList()
+               : argTypes.subList(1, argTypes.size());
+    }
+
+    /**
+     * @return A copy of this mask for a version of its masked column that has its type reversed.
+     */
+    public ColumnMask withReversedType()
+    {
+        AbstractType<?> reversed = ReversedType.getInstance(function.argTypes().get(0));
+        List<AbstractType<?>> args = ImmutableList.<AbstractType<?>>builder()
+                                                  .add(reversed)
+                                                  .addAll(partialArgumentTypes())
+                                                  .build();
+        Function newFunction = FunctionResolver.get(function.name().keyspace, function.name(), args, null, null, null);
+        assert newFunction != null;
+        return new ColumnMask((ScalarFunction) newFunction, partialArgumentValues);
+    }
+
+    /**
+     * @param protocolVersion the used version of the transport protocol
+     * @param value           a column value to be masked
+     * @return the specified value after having been masked by the masked function
+     */
+    public ByteBuffer mask(ProtocolVersion protocolVersion, ByteBuffer value)
+    {
+        List<ByteBuffer> args = new ArrayList<>(partialArgumentValues.size() + 1);
+        args.add(value);
+        args.addAll(partialArgumentValues);
+        return function.execute(protocolVersion, args);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        ColumnMask mask = (ColumnMask) o;
+        return function.name().equals(mask.function.name())
+               && partialArgumentValues.equals(mask.partialArgumentValues);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(function.name(), partialArgumentValues);
+    }
+
+    @Override
+    public String toString()
+    {
+        List<AbstractType<?>> types = partialArgumentTypes();
+        List<String> arguments = new ArrayList<>(types.size());
+        for (int i = 0; i < types.size(); i++)
+        {
+            CQL3Type type = types.get(i).asCQL3Type();
+            ByteBuffer value = partialArgumentValues.get(i);
+            arguments.add(type.toCQLLiteral(value, ProtocolVersion.CURRENT));
+        }
+        return String.format("%s(%s)", function.name(), StringUtils.join(arguments, ", "));
+    }
+
+    public void appendCqlTo(CqlBuilder builder)
+    {
+        builder.append(" MASKED WITH ").append(toString());
+    }
+
+    /**
+     * @return {@code true} if we know that the current cluster supports masked columns, or {@code false} if it either
+     * doesn't support it due to the presence of not-upgrades nodes, or we don't know if such old nodes exist.
+     */
+    public static boolean clusterSupportsMaskedColumns()
+    {
+        if (!Gossiper.instance.isEnabled())
+            return false;
+
+        long timeout = DatabaseDescriptor.getWriteRpcTimeout(TimeUnit.MILLISECONDS);
+        CassandraVersion minVersion = Gossiper.instance.getMinVersion(timeout, TimeUnit.MILLISECONDS);
+        return minVersion != null && minVersion.familyLowerBound.get().compareTo(CassandraVersion.CASSANDRA_4_1) > 0;
+    }
+
+    /**
+     * A parsed but not prepared column mask.
+     */
+    public final static class Raw
+    {
+        private static final Joiner JOINER = Joiner.on(',');
+
+        public final FunctionName name;
+        public final List<Term.Raw> rawPartialArguments;
+
+        public Raw(FunctionName name, List<Term.Raw> rawPartialArguments)
+        {
+            this.name = name;
+            this.rawPartialArguments = rawPartialArguments;
+        }
+
+        public ColumnMask prepare(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            ScalarFunction function = findMaskingFunction(keyspace, table, column, type);
+
+            List<ByteBuffer> partialArguments = preparePartialArguments(keyspace, function);
+
+            return new ColumnMask(function, partialArguments);
+        }
+
+        private ScalarFunction findMaskingFunction(String keyspace, String table, ColumnIdentifier column, AbstractType<?> type)
+        {
+            List<AssignmentTestable> args = new ArrayList<>(rawPartialArguments.size() + 1);
+            args.add(type);
+            args.addAll(rawPartialArguments);
+
+            Function function = FunctionResolver.get(keyspace, name, args, keyspace, table, type);
+
+            if (function == null)
+                throw invalidRequest("Unable to find masking function for %s, " +
+                                     "no declared function matches the signature %s",
+                                     column, this);
+
+            if (function.isAggregate())
+                throw invalidRequest("Aggregate function %s cannot be used for masking table columns", this);
+
+            if (!function.isNative())
+                throw invalidRequest("User defined function %s cannot be used for masking table columns", this);
+
+            if (!(function instanceof MaskingFunction))
+                throw invalidRequest("Not-masking function %s cannot be used for masking table columns", this);
+
+            if (!function.returnType().equals(type))

Review Comment:
   > take care of returning a reversed type if their input is reversed
   
   ^ That is what I was wondering 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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
test/unit/org/apache/cassandra/cql3/functions/masking/ColumnMaskNativeTypesTest.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.cql3.CQL3Type;
+
+import static java.lang.String.format;
+import static java.util.Collections.emptyList;
+
+/**
+ * {@link ColumnMaskTester} verifying that we can attach column masks to table columns with any native data type.
+ */
+@RunWith(Parameterized.class)
+public class ColumnMaskNativeTypesTest extends ColumnMaskTester
+{
+    /** The type of the column. */
+    @Parameterized.Parameter
+    public CQL3Type.Native type;
+
+    @Parameterized.Parameters(name = "type={0}")
+    public static Collection<Object[]> options()
+    {
+        List<Object[]> parameters = new ArrayList<>();
+        for (CQL3Type.Native type : CQL3Type.Native.values())
+        {
+            if (type != CQL3Type.Native.EMPTY)
+                parameters.add(new Object[]{ type });
+        }
+        return parameters;
+    }
+
+    @Test
+    public void testNativeDataTypes() throws Throwable
+    {
+        String def = format("%s MASKED WITH DEFAULT", type);
+        String keyDef = type == CQL3Type.Native.COUNTER || type == CQL3Type.Native.DURATION
+                        ? "int MASKED WITH DEFAULT" : def;
+
+        // Create table with masks
+        String table = createTable(format("CREATE TABLE %%s (k %s, c %<s, r %s, s %<s, PRIMARY KEY(k, c))", keyDef, def));

Review Comment:
   Indeed, good catch! Just fixed it.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -688,15 +695,58 @@ private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadat
     {
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
-            type = ((ReversedType) type).baseType;
+            type = ((ReversedType<?>) type).baseType;
 
-        builder.update(Columns)
+        Row.SimpleBuilder rowBuilder = builder.update(Columns)
                .row(table.name, column.name.toString())
                .add("column_name_bytes", column.name.bytes)
                .add("kind", column.kind.toString().toLowerCase())
                .add("position", column.position())
                .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
                .add("type", type.asCQL3Type().toString());
+
+        // Dynamic data masking functions shouldn't be attached to columns during rolling upgrades
+        // to avoid sending mutations with columns that are unknown to the old nodes.
+        ColumnMask mask = column.getMask();
+        if (ColumnMask.clusterSupportsMaskedColumns())
+        {
+            if (mask == null)
+            {
+                rowBuilder.delete("mask_keyspace")

Review Comment:
   In other cases yes, but here I would prefer to keep the style of `SchemaKeyspace`, where column names are always referred to with direct strings instead of constants. There are so many of these that probably the class would lose readability if we made all constants.



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -688,15 +695,58 @@ private static void addColumnToSchemaMutation(TableMetadata table, ColumnMetadat
     {
         AbstractType<?> type = column.type;
         if (type instanceof ReversedType)
-            type = ((ReversedType) type).baseType;
+            type = ((ReversedType<?>) type).baseType;
 
-        builder.update(Columns)
+        Row.SimpleBuilder rowBuilder = builder.update(Columns)
                .row(table.name, column.name.toString())
                .add("column_name_bytes", column.name.bytes)
                .add("kind", column.kind.toString().toLowerCase())
                .add("position", column.position())
                .add("clustering_order", column.clusteringOrder().toString().toLowerCase())
                .add("type", type.asCQL3Type().toString());
+
+        // Dynamic data masking functions shouldn't be attached to columns during rolling upgrades
+        // to avoid sending mutations with columns that are unknown to the old nodes.
+        ColumnMask mask = column.getMask();
+        if (ColumnMask.clusterSupportsMaskedColumns())
+        {
+            if (mask == null)
+            {
+                rowBuilder.delete("mask_keyspace")
+                          .delete("mask_name")
+                          .delete("mask_argument_types")
+                          .delete("mask_argument_values")
+                          .delete("mask_argument_nulls");
+            }
+            else
+            {
+                FunctionName maskFunctionName = mask.function.name();
+
+                // Some arguments of the masking function can be null, but the CQL's list type that stores them doesn't

Review Comment:
   Just realized that we have methods to make the conversion from byte buffer column values to CQL literals (`AbstractType#toCQLString`, `CQL3Type#toCQLLiteral`), but we don't have methods for the opposite conversion. 
   
   If we want `mask_argument_values` to be able to represent nulls we would need to add the reverse parsing, so the argument values are stored with quotes, escaping, etc. That would allow us to distinguish between, for example, `'null'` and `'''null'''`. That parsing is currently done in antlr and `Constants.Literal`. Bringing that parsing to `AbstractType` would probably require some not entirely trivial refactoring, given how the conversion functions are spread across abstract types, CQL types, serializers and antlr.
   
   Not sure if that would be worth compared to the current approach, which is more focused on overcoming the limitations of lists than on how we represent values. I think that if we had support for nulls on lists we wouldn't store quoted CQL values but just nulls.
   
   However, storing the values as strings seems a win for readability, so I'm changing that part to store the argument values as strings, but keeping the boolean list of nulls. wdyt?
   



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -137,6 +139,11 @@ private SchemaKeyspace()
               + "kind text,"
               + "position int,"
               + "type text,"
+              + "mask_keyspace text,"
+              + "mask_name text,"
+              + "mask_argument_types frozen<list<text>>,"
+              + "mask_argument_values frozen<list<blob>>,"
+              + "mask_argument_nulls frozen<list<boolean>>," // arguments that are null

Review Comment:
   The schema produced by this will be rejected by Cassandra versions without this patch. So this will break downgrades. This is ok if this lands in a major (5.0), but not if it goes to a minor (4.2).
   
   As an alternative, we can just put the masks on a separate table, this way: https://github.com/adelapena/cassandra/commit/cc9983911c6a99954a1e6ef0aeffa8f439f9b888
   
   That separate table for masks will just be ignored by older versions during a downgrade. It replicates the primary key of `system_schema.columns`, which is not ideal unless that's done for supporting downgrades. @blerer wdyt?



-- 
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 #2110: CASSANDRA-18068 trunk: Allow to attach native masking functions to table columns

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


##########
src/java/org/apache/cassandra/schema/SchemaKeyspace.java:
##########
@@ -137,6 +139,11 @@ private SchemaKeyspace()
               + "kind text,"
               + "position int,"
               + "type text,"
+              + "mask_keyspace text,"
+              + "mask_name text,"
+              + "mask_argument_types frozen<list<text>>,"
+              + "mask_argument_values frozen<list<blob>>,"
+              + "mask_argument_nulls frozen<list<boolean>>," // arguments that are null

Review Comment:
   Just moved the masks to a separate table, so the new columns don't prevent downgrades.



-- 
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