You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2021/06/02 12:40:42 UTC

[GitHub] [cassandra] adelapena commented on a change in pull request #1031: Fix queries on empty partitions with static data (CASSANDRA-16686)

adelapena commented on a change in pull request #1031:
URL: https://github.com/apache/cassandra/pull/1031#discussion_r643893258



##########
File path: src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
##########
@@ -835,6 +835,32 @@ public boolean hasRegularColumnsRestrictions()
     {
         return hasRegularColumnsRestrictions;
     }
+
+    /**
+     * Checks if the query is a full partitions selection.
+     * @return {@code true} if the query is a full partitions selection, {@code false} otherwise.
+     */
+    private boolean queriesFullPartitions()
+    {
+        return !hasClusteringColumnsRestrictions() && !hasRegularColumnsRestrictions();
+    }
+
+    /**
+     * Determines if the query should return the static content when a partition without rows is returned (as a
+     * result set row with null for all other regular columns.)
+     *
+     * @return {@code true} if the query should return the static content when a partition without rows is returned, {@code false} otherwise.

Review comment:
       Nit: we can split this line:
   ```suggestion
        * @return {@code true} if the query should return the static content when a partition without rows is returned,
        * {@code false} otherwise.
   ```

##########
File path: src/java/org/apache/cassandra/cql3/restrictions/StatementRestrictions.java
##########
@@ -835,6 +835,32 @@ public boolean hasRegularColumnsRestrictions()
     {
         return hasRegularColumnsRestrictions;
     }
+
+    /**
+     * Checks if the query is a full partitions selection.
+     * @return {@code true} if the query is a full partitions selection, {@code false} otherwise.
+     */
+    private boolean queriesFullPartitions()
+    {
+        return !hasClusteringColumnsRestrictions() && !hasRegularColumnsRestrictions();
+    }
+
+    /**
+     * Determines if the query should return the static content when a partition without rows is returned (as a
+     * result set row with null for all other regular columns.)
+     *
+     * @return {@code true} if the query should return the static content when a partition without rows is returned, {@code false} otherwise.
+     */
+    public boolean returnStaticContentOnPartitionWithNoRows()
+    {
+        if (table.isStaticCompactTable())
+            return true;
+
+        // The general rational is that if some rows are specifically selected by the query (have clustering or
+        // regular columns restrictions), we ignore partitions that are empty outside of static content, but if it's a full partition
+        // query, then we include that content.

Review comment:
       ```suggestion
           // The general rationale is that if some rows are specifically selected by the query (have clustering or
           // regular columns restrictions), we ignore partitions that are empty outside of static content, but if it's
           // a full partition query, then we include that content.
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -49,111 +48,166 @@
  * in its request.
  *
  * The reason for distinguishing those 2 sets is that due to the CQL semantic (see #6588 for more details), we
- * often need to internally fetch all regular columns for the queried table, but can still do some optimizations for
- * those columns that are not directly queried by the user (see #10657 for more details).
+ * often need to internally fetch all regular columns or all columns for the queried table, but can still do some
+ * optimizations for those columns that are not directly queried by the user (see #10657 for more details).
  *
  * Note that in practice:
  *   - the _queried_ columns set is always included in the _fetched_ one.
- *   - whenever those sets are different, we know 1) the _fetched_ set contains all regular columns for the table and 2)
- *     _fetched_ == _queried_ for static columns, so we don't have to record this set, we just keep a pointer to the
- *     table metadata. The only set we concretely store is thus the _queried_ one.
+ *   - whenever those sets are different, the _fetched_ columns can contains either all the regular columns and
+ *     the static columns queried by the user or all the regular and static queries. If the query is a partition level

Review comment:
       ```suggestion
    *     the static columns queried by the user or all the regular and static columns. If the query is a partition level
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -49,111 +48,166 @@
  * in its request.
  *
  * The reason for distinguishing those 2 sets is that due to the CQL semantic (see #6588 for more details), we
- * often need to internally fetch all regular columns for the queried table, but can still do some optimizations for
- * those columns that are not directly queried by the user (see #10657 for more details).
+ * often need to internally fetch all regular columns or all columns for the queried table, but can still do some
+ * optimizations for those columns that are not directly queried by the user (see #10657 for more details).
  *
  * Note that in practice:
  *   - the _queried_ columns set is always included in the _fetched_ one.
- *   - whenever those sets are different, we know 1) the _fetched_ set contains all regular columns for the table and 2)
- *     _fetched_ == _queried_ for static columns, so we don't have to record this set, we just keep a pointer to the
- *     table metadata. The only set we concretely store is thus the _queried_ one.
+ *   - whenever those sets are different, the _fetched_ columns can contains either all the regular columns and
+ *     the static columns queried by the user or all the regular and static queries. If the query is a partition level
+ *     query (no restrictions on clustering or regular columns) all the static columns will need to be fetched as
+ *     some data will need to be returned to the user if the partition has no row but some static data. For all the
+ *     other scenarios only the all the regular columns are required.
  *   - in the special case of a {@code SELECT *} query, we want to query all columns, and _fetched_ == _queried.
- *     As this is a common case, we special case it by keeping the _queried_ set {@code null} (and we retrieve
- *     the columns through the metadata pointer).
+ *     As this is a common case, we special case it by using a specific subclass for it.
  *
  * For complex columns, this class optionally allows to specify a subset of the cells to query for each column.
  * We can either select individual cells by path name, or a slice of them. Note that this is a sub-selection of
  * _queried_ cells, so if _fetched_ != _queried_, then the cell selected by this sub-selection are considered
  * queried and the other ones are considered fetched (and if a column has some sub-selection, it must be a queried
  * column, which is actually enforced by the Builder below).
  */
-public class ColumnFilter
+public abstract class ColumnFilter
 {
     private final static Logger logger = LoggerFactory.getLogger(ColumnFilter.class);
 
     public static final ColumnFilter NONE = selection(RegularAndStaticColumns.NONE);
 
     public static final Serializer serializer = new Serializer();
 
-    // True if _fetched_ includes all regular columns (and any static in _queried_), in which case metadata must not be
-    // null. If false, then _fetched_ == _queried_ and we only store _queried_.
-    @VisibleForTesting
-    final boolean fetchAllRegulars;
+    /**
+     * The fetching strategy for the different queries.
+     */
+    private enum FetchingStrategy
+    {
+        /**
+         * This strategy will fetch all the regular and static columns.
+         *
+         * <p>According to the CQL semantic a partition exist if has at least one row or one of its static columns not null.
+         * For queries that have no restrictions on the clustering or regular columns, C* returns will return some data for
+         * the partition even if it does not contains any row as long as one of the static columns contains data.
+         * To be able to ensure those queries all columns need to be fetched.</p>
+         *
+         * <p>This strategy is also used, instead of the ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS one, in mixed version clusters
+         * where some nodes have a version lower than 4.0. To ensure backward compatibility with those version that interpret the
+         * _fetchAll_ serialization flag as a true fetch all request.</p>
+         */
+        ALL_COLUMNS
+        {
+            @Override
+            boolean fetchesAllColumns(boolean isStatic)
+            {
+                return true;
+            }
 
-    // This flag can be only set when fetchAllRegulars is set. When fetchAllRegulars is set and queried==null then
-    // it is implied to be true. The flag when set allows for interpreting the column filter in the same way as it was
-    // interpreted by pre 4.0 Cassandra versions (3.4 ~ 4.0), that is, we fetch all columns (both regulars and static)
-    // but we query only some of them. This allows for proper behaviour during upgrades.
-    private final boolean fetchAllStatics;
+            @Override
+            RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried)
+            {
+                return metadata.regularAndStaticColumns();
+            }
+        },
 
-    @VisibleForTesting
-    final RegularAndStaticColumns fetched;
+        /**
+         * This strategy will fetch all the regular and selected static columns.
+         *
+         * <p>According to the CQL semantic a row exist if has at least one of its columns is not null.

Review comment:
       ```suggestion
            * <p>According to the CQL semantic a row exists if at least one of its columns is not null.
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -49,111 +48,166 @@
  * in its request.
  *
  * The reason for distinguishing those 2 sets is that due to the CQL semantic (see #6588 for more details), we
- * often need to internally fetch all regular columns for the queried table, but can still do some optimizations for
- * those columns that are not directly queried by the user (see #10657 for more details).
+ * often need to internally fetch all regular columns or all columns for the queried table, but can still do some
+ * optimizations for those columns that are not directly queried by the user (see #10657 for more details).
  *
  * Note that in practice:
  *   - the _queried_ columns set is always included in the _fetched_ one.
- *   - whenever those sets are different, we know 1) the _fetched_ set contains all regular columns for the table and 2)
- *     _fetched_ == _queried_ for static columns, so we don't have to record this set, we just keep a pointer to the
- *     table metadata. The only set we concretely store is thus the _queried_ one.
+ *   - whenever those sets are different, the _fetched_ columns can contains either all the regular columns and
+ *     the static columns queried by the user or all the regular and static queries. If the query is a partition level
+ *     query (no restrictions on clustering or regular columns) all the static columns will need to be fetched as
+ *     some data will need to be returned to the user if the partition has no row but some static data. For all the
+ *     other scenarios only the all the regular columns are required.
  *   - in the special case of a {@code SELECT *} query, we want to query all columns, and _fetched_ == _queried.
- *     As this is a common case, we special case it by keeping the _queried_ set {@code null} (and we retrieve
- *     the columns through the metadata pointer).
+ *     As this is a common case, we special case it by using a specific subclass for it.
  *
  * For complex columns, this class optionally allows to specify a subset of the cells to query for each column.
  * We can either select individual cells by path name, or a slice of them. Note that this is a sub-selection of
  * _queried_ cells, so if _fetched_ != _queried_, then the cell selected by this sub-selection are considered
  * queried and the other ones are considered fetched (and if a column has some sub-selection, it must be a queried
  * column, which is actually enforced by the Builder below).
  */
-public class ColumnFilter
+public abstract class ColumnFilter
 {
     private final static Logger logger = LoggerFactory.getLogger(ColumnFilter.class);
 
     public static final ColumnFilter NONE = selection(RegularAndStaticColumns.NONE);
 
     public static final Serializer serializer = new Serializer();
 
-    // True if _fetched_ includes all regular columns (and any static in _queried_), in which case metadata must not be
-    // null. If false, then _fetched_ == _queried_ and we only store _queried_.
-    @VisibleForTesting
-    final boolean fetchAllRegulars;
+    /**
+     * The fetching strategy for the different queries.
+     */
+    private enum FetchingStrategy
+    {
+        /**
+         * This strategy will fetch all the regular and static columns.
+         *
+         * <p>According to the CQL semantic a partition exist if has at least one row or one of its static columns not null.

Review comment:
       ```suggestion
            * <p>According to the CQL semantic a partition exists if it has at least one row or one of its static columns is not null.
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -201,40 +260,59 @@ public static ColumnFilter all(TableMetadata metadata)
      */
     public static ColumnFilter selection(RegularAndStaticColumns columns)
     {
-        return new ColumnFilter(false, false, (TableMetadata) null, columns, null);
+        return SelectionColumnFilter.newInstance(FetchingStrategy.ONLY_QUERIED_COLUMNS,
+                                                 (TableMetadata) null,
+                                                 columns,
+                                                 null);

Review comment:
       Nit: I think we don't need the cast of null, and without it this can be a one-liner:
   ```suggestion
           return SelectionColumnFilter.newInstance(FetchingStrategy.ONLY_QUERIED_COLUMNS, null, columns, null);
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -49,111 +48,166 @@
  * in its request.
  *
  * The reason for distinguishing those 2 sets is that due to the CQL semantic (see #6588 for more details), we
- * often need to internally fetch all regular columns for the queried table, but can still do some optimizations for
- * those columns that are not directly queried by the user (see #10657 for more details).
+ * often need to internally fetch all regular columns or all columns for the queried table, but can still do some
+ * optimizations for those columns that are not directly queried by the user (see #10657 for more details).
  *
  * Note that in practice:
  *   - the _queried_ columns set is always included in the _fetched_ one.
- *   - whenever those sets are different, we know 1) the _fetched_ set contains all regular columns for the table and 2)
- *     _fetched_ == _queried_ for static columns, so we don't have to record this set, we just keep a pointer to the
- *     table metadata. The only set we concretely store is thus the _queried_ one.
+ *   - whenever those sets are different, the _fetched_ columns can contains either all the regular columns and
+ *     the static columns queried by the user or all the regular and static queries. If the query is a partition level
+ *     query (no restrictions on clustering or regular columns) all the static columns will need to be fetched as
+ *     some data will need to be returned to the user if the partition has no row but some static data. For all the
+ *     other scenarios only the all the regular columns are required.
  *   - in the special case of a {@code SELECT *} query, we want to query all columns, and _fetched_ == _queried.
- *     As this is a common case, we special case it by keeping the _queried_ set {@code null} (and we retrieve
- *     the columns through the metadata pointer).
+ *     As this is a common case, we special case it by using a specific subclass for it.
  *
  * For complex columns, this class optionally allows to specify a subset of the cells to query for each column.
  * We can either select individual cells by path name, or a slice of them. Note that this is a sub-selection of
  * _queried_ cells, so if _fetched_ != _queried_, then the cell selected by this sub-selection are considered
  * queried and the other ones are considered fetched (and if a column has some sub-selection, it must be a queried
  * column, which is actually enforced by the Builder below).
  */
-public class ColumnFilter
+public abstract class ColumnFilter
 {
     private final static Logger logger = LoggerFactory.getLogger(ColumnFilter.class);
 
     public static final ColumnFilter NONE = selection(RegularAndStaticColumns.NONE);
 
     public static final Serializer serializer = new Serializer();
 
-    // True if _fetched_ includes all regular columns (and any static in _queried_), in which case metadata must not be
-    // null. If false, then _fetched_ == _queried_ and we only store _queried_.
-    @VisibleForTesting
-    final boolean fetchAllRegulars;
+    /**
+     * The fetching strategy for the different queries.
+     */
+    private enum FetchingStrategy
+    {
+        /**
+         * This strategy will fetch all the regular and static columns.
+         *
+         * <p>According to the CQL semantic a partition exist if has at least one row or one of its static columns not null.
+         * For queries that have no restrictions on the clustering or regular columns, C* returns will return some data for
+         * the partition even if it does not contains any row as long as one of the static columns contains data.
+         * To be able to ensure those queries all columns need to be fetched.</p>
+         *
+         * <p>This strategy is also used, instead of the ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS one, in mixed version clusters
+         * where some nodes have a version lower than 4.0. To ensure backward compatibility with those version that interpret the
+         * _fetchAll_ serialization flag as a true fetch all request.</p>
+         */
+        ALL_COLUMNS
+        {
+            @Override
+            boolean fetchesAllColumns(boolean isStatic)
+            {
+                return true;
+            }
 
-    // This flag can be only set when fetchAllRegulars is set. When fetchAllRegulars is set and queried==null then
-    // it is implied to be true. The flag when set allows for interpreting the column filter in the same way as it was
-    // interpreted by pre 4.0 Cassandra versions (3.4 ~ 4.0), that is, we fetch all columns (both regulars and static)
-    // but we query only some of them. This allows for proper behaviour during upgrades.
-    private final boolean fetchAllStatics;
+            @Override
+            RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried)
+            {
+                return metadata.regularAndStaticColumns();
+            }
+        },
 
-    @VisibleForTesting
-    final RegularAndStaticColumns fetched;
+        /**
+         * This strategy will fetch all the regular and selected static columns.
+         *
+         * <p>According to the CQL semantic a row exist if has at least one of its columns is not null.
+         * To ensure that we need to fetch all regular columns.</p>
+         */
+        ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS
+        {
+            @Override
+            boolean fetchesAllColumns(boolean isStatic)
+            {
+                return !isStatic;
+            }
 
-    private final RegularAndStaticColumns queried; // can be null if fetchAllRegulars, to represent a wildcard query (all
+            @Override
+            RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried)
+            {
+                return new RegularAndStaticColumns(queried.statics, metadata.regularColumns());
+            }
+        },
 
-    // static and regular columns are both _fetched_ and _queried_).
-    private final SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections; // can be null
+        /**
+         * Fetch only the columns that have been selected.
+         *
+         * <p>With this strategy _queried_ == _fetched_. This strategy is only used for internal queries.</p>
+         */
+        ONLY_QUERIED_COLUMNS
+        {
+            @Override
+            boolean fetchesAllColumns(boolean isStatic)
+            {
+                return false;
+            }
 
-    private ColumnFilter(boolean fetchAllRegulars,
-                         boolean fetchAllStatics,
-                         TableMetadata metadata,
-                         RegularAndStaticColumns queried,
-                         SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections)
-    {
-        assert !fetchAllRegulars || metadata != null;
-        assert fetchAllRegulars || queried != null;
-        assert !fetchAllStatics || fetchAllRegulars;
-        this.fetchAllRegulars = fetchAllRegulars;
-        this.fetchAllStatics = fetchAllStatics || fetchAllRegulars && queried == null;
+            @Override
+            boolean areAllFetchedColumnsQueried()
+            {
+                return true;
+            }
 
-        if (fetchAllRegulars)
-        {
-            RegularAndStaticColumns all = metadata.regularAndStaticColumns();
+            @Override
+            RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried)
+            {
+                return queried;
+            }
+        };
 
-            this.fetched = (all.statics.isEmpty() || queried == null || fetchAllStatics)
-                           ? all
-                           : new RegularAndStaticColumns(queried.statics, all.regulars);
-        }
-        else
+        /**
+         * Checks if the strategy fetch all the specified columns
+         *
+         * @param isStatic {@code true} is the check is for static columns, {@code false} otherwise
+         * @return {@code true} if the strategy fetch all the static columns, {@code false} otherwise.
+         */
+        abstract boolean fetchesAllColumns(boolean isStatic);
+
+        /**
+         * Checks if all the fetched columns are guaranteed to be queried
+         *
+         * @return {@code true} if all the fetched columns are guaranteed to be queried, {@code false} otherwise.
+         */
+        boolean areAllFetchedColumnsQueried()
         {
-            this.fetched = queried;
+            return false;
         }
 
-        this.queried = queried;
-        this.subSelections = subSelections;
+        /**
+         * Returns the columns that must be fetched to answer the query.
+         *
+         * @param metadata the table metadata
+         * @param queried the queried columns
+         * @return the columns that must be fetched
+         */
+        abstract RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried);
     }
 
     /**
-     * Used on replica for deserialisation
+     * Returns {@code true} if there are 4.0 pre-release nodes in the cluster (e.g. 4.0-rc1), {@code false} otherwise.
+     *
+     * <p>ColumnFilters from 4.0 release before RC2 wrongly assumed that fetching all regular columns and not the all
+     * the static columns was enough. That was not the case for queries that needed to return rows for empty partitions.
+     * See CASSANDRA-16686 for more details.</p>
      */
-    private ColumnFilter(boolean fetchAllRegulars,
-                         boolean fetchAllStatics,
-                         RegularAndStaticColumns fetched,
-                         RegularAndStaticColumns queried,
-                         SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections)
+    private static boolean isUpgradingFromVersionLowerThan40RC2()

Review comment:
       I think that both the description and the trace message of this method suggest that it returns true for releases between 4.0-alpha1 and 4.0-rc1, whereas it returns true for any version below 4.0-rc2, even if it isn't in the 4.0 group.

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -49,111 +48,166 @@
  * in its request.
  *
  * The reason for distinguishing those 2 sets is that due to the CQL semantic (see #6588 for more details), we
- * often need to internally fetch all regular columns for the queried table, but can still do some optimizations for
- * those columns that are not directly queried by the user (see #10657 for more details).
+ * often need to internally fetch all regular columns or all columns for the queried table, but can still do some
+ * optimizations for those columns that are not directly queried by the user (see #10657 for more details).
  *
  * Note that in practice:
  *   - the _queried_ columns set is always included in the _fetched_ one.
- *   - whenever those sets are different, we know 1) the _fetched_ set contains all regular columns for the table and 2)
- *     _fetched_ == _queried_ for static columns, so we don't have to record this set, we just keep a pointer to the
- *     table metadata. The only set we concretely store is thus the _queried_ one.
+ *   - whenever those sets are different, the _fetched_ columns can contains either all the regular columns and
+ *     the static columns queried by the user or all the regular and static queries. If the query is a partition level
+ *     query (no restrictions on clustering or regular columns) all the static columns will need to be fetched as
+ *     some data will need to be returned to the user if the partition has no row but some static data. For all the
+ *     other scenarios only the all the regular columns are required.
  *   - in the special case of a {@code SELECT *} query, we want to query all columns, and _fetched_ == _queried.
- *     As this is a common case, we special case it by keeping the _queried_ set {@code null} (and we retrieve
- *     the columns through the metadata pointer).
+ *     As this is a common case, we special case it by using a specific subclass for it.
  *
  * For complex columns, this class optionally allows to specify a subset of the cells to query for each column.
  * We can either select individual cells by path name, or a slice of them. Note that this is a sub-selection of
  * _queried_ cells, so if _fetched_ != _queried_, then the cell selected by this sub-selection are considered
  * queried and the other ones are considered fetched (and if a column has some sub-selection, it must be a queried
  * column, which is actually enforced by the Builder below).
  */
-public class ColumnFilter
+public abstract class ColumnFilter
 {
     private final static Logger logger = LoggerFactory.getLogger(ColumnFilter.class);
 
     public static final ColumnFilter NONE = selection(RegularAndStaticColumns.NONE);
 
     public static final Serializer serializer = new Serializer();
 
-    // True if _fetched_ includes all regular columns (and any static in _queried_), in which case metadata must not be
-    // null. If false, then _fetched_ == _queried_ and we only store _queried_.
-    @VisibleForTesting
-    final boolean fetchAllRegulars;
+    /**
+     * The fetching strategy for the different queries.
+     */
+    private enum FetchingStrategy
+    {
+        /**
+         * This strategy will fetch all the regular and static columns.
+         *
+         * <p>According to the CQL semantic a partition exist if has at least one row or one of its static columns not null.
+         * For queries that have no restrictions on the clustering or regular columns, C* returns will return some data for
+         * the partition even if it does not contains any row as long as one of the static columns contains data.
+         * To be able to ensure those queries all columns need to be fetched.</p>
+         *
+         * <p>This strategy is also used, instead of the ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS one, in mixed version clusters
+         * where some nodes have a version lower than 4.0. To ensure backward compatibility with those version that interpret the
+         * _fetchAll_ serialization flag as a true fetch all request.</p>
+         */
+        ALL_COLUMNS
+        {
+            @Override
+            boolean fetchesAllColumns(boolean isStatic)
+            {
+                return true;
+            }
 
-    // This flag can be only set when fetchAllRegulars is set. When fetchAllRegulars is set and queried==null then
-    // it is implied to be true. The flag when set allows for interpreting the column filter in the same way as it was
-    // interpreted by pre 4.0 Cassandra versions (3.4 ~ 4.0), that is, we fetch all columns (both regulars and static)
-    // but we query only some of them. This allows for proper behaviour during upgrades.
-    private final boolean fetchAllStatics;
+            @Override
+            RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried)
+            {
+                return metadata.regularAndStaticColumns();
+            }
+        },
 
-    @VisibleForTesting
-    final RegularAndStaticColumns fetched;
+        /**
+         * This strategy will fetch all the regular and selected static columns.
+         *
+         * <p>According to the CQL semantic a row exist if has at least one of its columns is not null.
+         * To ensure that we need to fetch all regular columns.</p>
+         */
+        ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS
+        {
+            @Override
+            boolean fetchesAllColumns(boolean isStatic)
+            {
+                return !isStatic;
+            }
 
-    private final RegularAndStaticColumns queried; // can be null if fetchAllRegulars, to represent a wildcard query (all
+            @Override
+            RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried)
+            {
+                return new RegularAndStaticColumns(queried.statics, metadata.regularColumns());
+            }
+        },
 
-    // static and regular columns are both _fetched_ and _queried_).
-    private final SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections; // can be null
+        /**
+         * Fetch only the columns that have been selected.
+         *
+         * <p>With this strategy _queried_ == _fetched_. This strategy is only used for internal queries.</p>
+         */
+        ONLY_QUERIED_COLUMNS
+        {
+            @Override
+            boolean fetchesAllColumns(boolean isStatic)
+            {
+                return false;
+            }
 
-    private ColumnFilter(boolean fetchAllRegulars,
-                         boolean fetchAllStatics,
-                         TableMetadata metadata,
-                         RegularAndStaticColumns queried,
-                         SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections)
-    {
-        assert !fetchAllRegulars || metadata != null;
-        assert fetchAllRegulars || queried != null;
-        assert !fetchAllStatics || fetchAllRegulars;
-        this.fetchAllRegulars = fetchAllRegulars;
-        this.fetchAllStatics = fetchAllStatics || fetchAllRegulars && queried == null;
+            @Override
+            boolean areAllFetchedColumnsQueried()
+            {
+                return true;
+            }
 
-        if (fetchAllRegulars)
-        {
-            RegularAndStaticColumns all = metadata.regularAndStaticColumns();
+            @Override
+            RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried)
+            {
+                return queried;
+            }
+        };
 
-            this.fetched = (all.statics.isEmpty() || queried == null || fetchAllStatics)
-                           ? all
-                           : new RegularAndStaticColumns(queried.statics, all.regulars);
-        }
-        else
+        /**
+         * Checks if the strategy fetch all the specified columns
+         *
+         * @param isStatic {@code true} is the check is for static columns, {@code false} otherwise
+         * @return {@code true} if the strategy fetch all the static columns, {@code false} otherwise.
+         */
+        abstract boolean fetchesAllColumns(boolean isStatic);
+
+        /**
+         * Checks if all the fetched columns are guaranteed to be queried
+         *
+         * @return {@code true} if all the fetched columns are guaranteed to be queried, {@code false} otherwise.
+         */
+        boolean areAllFetchedColumnsQueried()
         {
-            this.fetched = queried;
+            return false;
         }
 
-        this.queried = queried;
-        this.subSelections = subSelections;
+        /**
+         * Returns the columns that must be fetched to answer the query.
+         *
+         * @param metadata the table metadata
+         * @param queried the queried columns
+         * @return the columns that must be fetched
+         */
+        abstract RegularAndStaticColumns getFetchedColumns(TableMetadata metadata, RegularAndStaticColumns queried);
     }
 
     /**
-     * Used on replica for deserialisation
+     * Returns {@code true} if there are 4.0 pre-release nodes in the cluster (e.g. 4.0-rc1), {@code false} otherwise.
+     *
+     * <p>ColumnFilters from 4.0 release before RC2 wrongly assumed that fetching all regular columns and not the all

Review comment:
       ```suggestion
        * <p>ColumnFilters from 4.0 releases before RC2 wrongly assumed that fetching all regular columns and not
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -201,40 +260,59 @@ public static ColumnFilter all(TableMetadata metadata)
      */
     public static ColumnFilter selection(RegularAndStaticColumns columns)
     {
-        return new ColumnFilter(false, false, (TableMetadata) null, columns, null);
+        return SelectionColumnFilter.newInstance(FetchingStrategy.ONLY_QUERIED_COLUMNS,
+                                                 (TableMetadata) null,
+                                                 columns,
+                                                 null);
     }
 
     /**
      * A filter that fetches all columns for the provided table, but returns
      * only the queried ones.
      */
-    public static ColumnFilter selection(TableMetadata metadata, RegularAndStaticColumns queried)
+    public static ColumnFilter selection(TableMetadata metadata,
+                                          RegularAndStaticColumns queried,
+                                          boolean returnStaticContentOnPartitionWithNoRows)
     {
-        return new ColumnFilter(true, shouldFetchAllStatics(), metadata, shouldQueriedBeNull() ? null : queried, null);
+        // pre CASSANDRA-10657 (3.4-), when fetchAll is enabled, queried columns are not considered at all, and it
+        // is assumed that all columns are queried.
+        if (isUpgradingFromVersionLowerThan34())
+        {
+            return new WildCardColumnFilter(metadata.regularAndStaticColumns());
+        }
+
+        // pre CASSANDRA-12768 (4.0-) all static columns should be fetched along with all regular columns.
+        if (isUpgradingFromVersionLowerThan40())
+        {
+            return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_COLUMNS, metadata, queried, null);
+        }
+
+        // pre CASSANDRA-16686 (4.0-RC2-) static columns where not fetched unless queried witch lead to some wrong results
+        // for some queries

Review comment:
       ```suggestion
           // pre CASSANDRA-16686 (4.0-RC2-) static columns were not fetched unless queried which led to some wrong
           // results for some queries
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -201,40 +260,59 @@ public static ColumnFilter all(TableMetadata metadata)
      */
     public static ColumnFilter selection(RegularAndStaticColumns columns)
     {
-        return new ColumnFilter(false, false, (TableMetadata) null, columns, null);
+        return SelectionColumnFilter.newInstance(FetchingStrategy.ONLY_QUERIED_COLUMNS,
+                                                 (TableMetadata) null,
+                                                 columns,
+                                                 null);
     }
 
     /**
      * A filter that fetches all columns for the provided table, but returns
      * only the queried ones.
      */
-    public static ColumnFilter selection(TableMetadata metadata, RegularAndStaticColumns queried)
+    public static ColumnFilter selection(TableMetadata metadata,
+                                          RegularAndStaticColumns queried,
+                                          boolean returnStaticContentOnPartitionWithNoRows)

Review comment:
       Nit: alignement
   ```suggestion
       public static ColumnFilter selection(TableMetadata metadata,
                                            RegularAndStaticColumns queried,
                                            boolean returnStaticContentOnPartitionWithNoRows)
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -502,149 +561,473 @@ public ColumnFilter build()
             boolean isFetchAll = metadata != null;
 
             RegularAndStaticColumns queried = queriedBuilder == null ? null : queriedBuilder.build();
+
             // It's only ok to have queried == null in ColumnFilter if isFetchAll. So deal with the case of a selectionBuilder
             // with nothing selected (we can at least happen on some backward compatible queries - CASSANDRA-10471).
             if (!isFetchAll && queried == null)
                 queried = RegularAndStaticColumns.NONE;
 
-            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = null;
-            if (subSelections != null)
+            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = buildSubSelections();
+
+            if (isFetchAll)
             {
-                s = TreeMultimap.create(Comparator.naturalOrder(), Comparator.naturalOrder());
-                for (ColumnSubselection subSelection : subSelections)
+                // When fetchAll is enabled on pre CASSANDRA-10657 (3.4-), queried columns are not considered at all, and it
+                // is assumed that all columns are queried. CASSANDRA-10657 (3.4+) brings back skipping values of columns
+                // which are not in queried set when fetchAll is enabled. That makes exactly the same filter being
+                // interpreted in a different way on 3.4- and 3.4+.
+                //
+                // Moreover, there is no way to convert the filter with fetchAll and queried != null so that it is
+                // interpreted the same way on 3.4- because that Cassandra version does not support such filtering.
+                //
+                // In order to avoid inconsitencies in data read by 3.4- and 3.4+ we need to avoid creation of incompatible
+                // filters when the cluster contains 3.4- nodes. We do that by forcibly setting queried to null.
+                //
+                // see CASSANDRA-10657, CASSANDRA-15833, CASSANDRA-16415
+                if (queried == null || isUpgradingFromVersionLowerThan34())
+                {
+                    return new WildCardColumnFilter(metadata.regularAndStaticColumns());
+                }
+
+                // pre CASSANDRA-12768 (4.0-) all static columns should be fetched along with all regular columns.
+                if (isUpgradingFromVersionLowerThan40())
                 {
-                    if (fullySelectedComplexColumns == null || !fullySelectedComplexColumns.contains(subSelection.column()))
-                        s.put(subSelection.column().name, subSelection);
+                    return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_COLUMNS, metadata, queried, s);
                 }
+
+                // pre CASSANDRA-16686 (4.0-RC2-) static columns where not fetched unless queried witch lead to some wrong results
+                // for some queries
+                if (isUpgradingFromVersionLowerThan40RC2() || !returnStaticContentOnPartitionWithNoRows)
+                {
+                    return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS, metadata, queried, s);
+                }
+
+                return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_COLUMNS, metadata, queried, s);
+            }
+
+            return SelectionColumnFilter.newInstance(FetchingStrategy.ONLY_QUERIED_COLUMNS, (TableMetadata) null, queried, s);
+        }
+
+        private SortedSetMultimap<ColumnIdentifier, ColumnSubselection> buildSubSelections()
+        {
+            if (subSelections == null)
+                return null;
+
+            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = TreeMultimap.create(Comparator.naturalOrder(), Comparator.naturalOrder());
+            for (ColumnSubselection subSelection : subSelections)
+            {
+                if (fullySelectedComplexColumns == null || !fullySelectedComplexColumns.contains(subSelection.column()))
+                    s.put(subSelection.column().name, subSelection);
             }
 
-            // When fetchAll is enabled on pre CASSANDRA-10657 (3.4-), queried columns are not considered at all, and it
-            // is assumed that all columns are queried. CASSANDRA-10657 (3.4+) brings back skipping values of columns
-            // which are not in queried set when fetchAll is enabled. That makes exactly the same filter being
-            // interpreted in a different way on 3.4- and 3.4+.
-            //
-            // Moreover, there is no way to convert the filter with fetchAll and queried != null so that it is
-            // interpreted the same way on 3.4- because that Cassandra version does not support such filtering.
-            //
-            // In order to avoid inconsitencies in data read by 3.4- and 3.4+ we need to avoid creation of incompatible
-            // filters when the cluster contains 3.4- nodes. We do that by forcibly setting queried to null.
-            //
-            // see CASSANDRA-10657, CASSANDRA-15833, CASSANDRA-16415
-            return new ColumnFilter(isFetchAll, isFetchAll && shouldFetchAllStatics(), metadata, isFetchAll && shouldQueriedBeNull() ? null : queried, s);
+            return s;
         }
     }
 
-    @Override
-    public boolean equals(Object other)
+    /**
+     * {@code ColumnFilter} sub-class for wildcard queries.
+     *
+     * <p>The class  does not rely on TableMetadata and expect a fix set of columns to prevent issues
+     * with Schema race propagation. See CASSANDRA-15899.</p>
+     */
+    public static class WildCardColumnFilter extends ColumnFilter
     {
-        if (other == this)
+        /**
+         * The queried and fetched columns.
+         */
+        private final RegularAndStaticColumns fetchedAndQueried;
+
+        /**
+         * Creates a {@code ColumnFilter} for wildcard queries.
+         *
+         * <p>The class  does not rely on TableMetadata and expect a fix set of columns to prevent issues
+         * with Schema race propagation. See CASSANDRA-15899.</p>
+         *
+         * @param fetchedAndQueried the fetched and queried columns
+         */
+        private WildCardColumnFilter(RegularAndStaticColumns fetchedAndQueried)
+        {
+            this.fetchedAndQueried = fetchedAndQueried;
+        }
+
+        @Override
+        public RegularAndStaticColumns fetchedColumns()
+        {
+            return fetchedAndQueried;
+        }
+
+        @Override
+        public RegularAndStaticColumns queriedColumns()
+        {
+            return fetchedAndQueried;
+        }
+
+        @Override
+        public boolean fetchesAllColumns(boolean isStatic)
+        {
             return true;
+        }
 
-        if (!(other instanceof ColumnFilter))
-            return false;
+        @Override
+        public boolean allFetchedColumnsAreQueried()
+        {
+            return true;
+        }
 
-        ColumnFilter otherCf = (ColumnFilter) other;
+        @Override
+        public boolean fetches(ColumnMetadata column)
+        {
+            return true;
+        }
 
-        return otherCf.fetchAllRegulars == this.fetchAllRegulars &&
-               otherCf.fetchAllStatics == this.fetchAllStatics &&
-               Objects.equals(otherCf.fetched, this.fetched) &&
-               Objects.equals(otherCf.queried, this.queried) &&
-               Objects.equals(otherCf.subSelections, this.subSelections);
-    }
+        @Override
+        public boolean fetchedColumnIsQueried(ColumnMetadata column)
+        {
+            return true;
+        }
 
-    @Override
-    public String toString()
-    {
-        String prefix = "";
+        @Override
+        public boolean fetchedCellIsQueried(ColumnMetadata column, CellPath path)
+        {
+            return true;
+        }
 
-        if (fetchAllRegulars && queried == null)
-            return "*/*";
+        @Override
+        public Iterator<Cell<?>> filterComplexCells(ColumnMetadata column, Iterator<Cell<?>> cells)
+        {
+            return cells;
+        }
 
-        if (fetchAllRegulars && fetchAllStatics)
-            prefix = "*/";
+        @Override
+        public Tester newTester(ColumnMetadata column)
+        {
+            return null;
+        }
 
-        if (fetchAllRegulars && !fetchAllStatics)
+        @Override
+        public boolean equals(Object other)
         {
-            prefix = queried.statics.isEmpty()
-                   ? "<all regulars>/"
-                   : String.format("<all regulars>+%s/", toString(queried.statics.selectOrderIterator(), false));
+            if (other == this)
+                return true;
+
+            if (!(other instanceof WildCardColumnFilter))
+                return false;
+
+            WildCardColumnFilter w = (WildCardColumnFilter) other;
+
+            return fetchedAndQueried.equals(w.fetchedAndQueried);
         }
 
-        return prefix + toString(queried.selectOrderIterator(), false);
-    }
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(fetchedAndQueried);
+        }
 
-    public String toCQLString()
-    {
-        if (queried == null || queried.isEmpty())
+        @Override
+        public String toString()
+        {
+            return "*/*";
+        }
+
+        public String toCQLString()
+        {
             return "*";
+        }
+
+        @Override
+        public boolean isWildcard()
+        {
+            return true;
+        }
 
-        return toString(queried.selectOrderIterator(), true);
+        @Override
+        protected SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections()
+        {
+            return null;
+        }
     }
 
-    private String toString(Iterator<ColumnMetadata> columns, boolean cql)
+    /**
+     * {@code ColumnFilter} sub-class for queries with selected columns.
+     *
+     * <p>The class  does not rely on TableMetadata and expect a fix set of fetched columns to prevent issues
+     * with Schema race propagation. See CASSANDRA-15899.</p>
+     */
+    public static class SelectionColumnFilter extends ColumnFilter
     {
-        StringJoiner joiner = cql ? new StringJoiner(", ") : new StringJoiner(", ", "[", "]");
+        public final FetchingStrategy fetchingStrategy;
+
+        /**
+         * The selected columns
+         */
+        private final RegularAndStaticColumns queried;
+
+        /**
+         * The columns that need to be fetched to be able
+         */
+        private final RegularAndStaticColumns fetched;
+
+        private final SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections; // can be null
+
+        public static SelectionColumnFilter newInstance(FetchingStrategy fetchingStrategy,
+                                                        TableMetadata metadata,
+                                                        RegularAndStaticColumns queried,
+                                                        SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections)
+        {
+            assert fetchingStrategy != FetchingStrategy.ONLY_QUERIED_COLUMNS || metadata == null;
+            assert queried != null;
+
+            return new SelectionColumnFilter(fetchingStrategy,
+                                             queried,
+                                             fetchingStrategy.getFetchedColumns(metadata, queried),
+                                             subSelections);
+        }
+
+        /**
+         * Creates a {@code ColumnFilter} for queries with selected columns.
+         *
+         * <p>The class  does not rely on TableMetadata and expect a fix set of columns to prevent issues
+         * with Schema race propagation. See CASSANDRA-15899.</p>
+         *
+         * @param fetchingStrategy the strategy used to select the fetched columns
+         * @param fetched the columns that must be fetched
+         * @param queried the queried columns
+         * @param subSelections the columns sub-selections
+         */
+        public SelectionColumnFilter(FetchingStrategy fetchingStrategy,
+                                     RegularAndStaticColumns queried,
+                                     RegularAndStaticColumns fetched,
+                                     SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections)
+        {
+            assert fetched.includes(queried);
+
+            this.fetchingStrategy = fetchingStrategy;
+            this.queried = queried;
+            this.fetched = fetched;
+            this.subSelections = subSelections;
+        }
+
+        @Override
+        public RegularAndStaticColumns fetchedColumns()
+        {
+            return fetched;
+        }
+
+        @Override
+        public RegularAndStaticColumns queriedColumns()
+        {
+            return queried;
+        }
+
+        @Override
+        public boolean fetchesAllColumns(boolean isStatic)
+        {
+            return fetchingStrategy.fetchesAllColumns(isStatic);
+        }
+
+        @Override
+        public boolean allFetchedColumnsAreQueried()
+        {
+            return fetchingStrategy.areAllFetchedColumnsQueried();
+        }
+
+        @Override
+        public boolean fetches(ColumnMetadata column)
+        {
+            return fetchingStrategy.fetchesAllColumns(column.isStatic()) || queried.contains(column);
+        }
+
+        @Override
+        public boolean fetchedColumnIsQueried(ColumnMetadata column)
+        {
+            return fetchingStrategy.areAllFetchedColumnsQueried() || queried.contains(column);
+        }
+
+        @Override
+        public boolean fetchedCellIsQueried(ColumnMetadata column, CellPath path)
+        {
+            assert path != null;
+
+            // first verify that the column to which the cell belongs is queried
+            if (!fetchedColumnIsQueried(column))
+                return false;
+
+            if (subSelections == null)
+                return true;
+
+            SortedSet<ColumnSubselection> s = subSelections.get(column.name);
+            // No subsection for this column means everything is queried
+            if (s.isEmpty())
+                return true;
+
+            for (ColumnSubselection subSel : s)
+                if (subSel.compareInclusionOf(path) == 0)
+                    return true;
+
+            return false;
+        }
 
-        while (columns.hasNext())
+        @Override
+        public Iterator<Cell<?>> filterComplexCells(ColumnMetadata column, Iterator<Cell<?>> cells)
         {
-            ColumnMetadata column = columns.next();
-            String columnName = cql ? column.name.toCQLString() : String.valueOf(column.name);
+            Tester tester = newTester(column);
+            if (tester == null)
+                return cells;
 
-            SortedSet<ColumnSubselection> s = subSelections != null
-                                            ? subSelections.get(column.name)
-                                            : Collections.emptySortedSet();
+            return Iterators.filter(cells, cell -> tester.fetchedCellIsQueried(cell.path()));

Review comment:
       Maybe we can consider null cells:
   ```suggestion
               return Iterators.filter(cells, cell -> cell != null && tester.fetchedCellIsQueried(cell.path()));
   ```

##########
File path: test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
##########
@@ -1423,6 +1420,51 @@ public void testWithEmptyRange() throws Throwable
         execute("DELETE FROM %s WHERE k = ? AND a >= ? AND a < ?", "a", 0, 2);
     }
 
+    @Test
+    public void testStaticColumnDeletionWithMultipleStaticColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int, ck int, s1 int static, s2 int static, v int, PRIMARY KEY(pk, ck))");
+
+        execute("INSERT INTO %s (pk, s1, s2) VALUES (1, 1, 1) USING TIMESTAMP 1000");
+        flush();
+        execute("INSERT INTO %s (pk, s1) VALUES (1, 2) USING TIMESTAMP 2000");
+        flush();
+        execute("DELETE s1 FROM %s USING TIMESTAMP 3000 WHERE pk = 1");
+        flush();
+
+        assertRows(execute("SELECT * FROM %s WHERE pk=1"), row(1, null, null, 1, null));
+        assertRows(execute("SELECT pk, s1, s2 FROM %s WHERE pk=1"), row(1, (Integer) null, 1));
+        assertRows(execute("SELECT s1, s2 FROM %s WHERE pk=1"), row((Integer) null, 1));
+        assertRows(execute("SELECT pk, s1 FROM %s WHERE pk=1"), row(1, (Integer) null)); 
+        assertRows(execute("SELECT s1 FROM %s WHERE pk=1"), row((Integer) null));
+        assertRows(execute("SELECT pk, s2 FROM %s WHERE pk=1"), row(1, 1));
+        assertRows(execute("SELECT s2 FROM %s WHERE pk=1"), row(1));
+        assertRows(execute("SELECT pk, ck FROM %s WHERE pk=1"), row(1, null));
+        assertRows(execute("SELECT ck FROM %s WHERE pk=1"), row((Integer) null));
+        assertRows(execute("SELECT DISTINCT pk, s1, s2 FROM %s WHERE pk=1"), row(1, (Integer) null, 1));
+        assertRows(execute("SELECT DISTINCT s1, s2 FROM %s WHERE pk=1"), row((Integer) null, 1));
+        assertRows(execute("SELECT DISTINCT pk, s1 FROM %s WHERE pk=1"), row(1, (Integer) null));
+        assertRows(execute("SELECT DISTINCT pk, s2 FROM %s WHERE pk=1"), row(1, 1));
+        assertRows(execute("SELECT DISTINCT s1 FROM %s WHERE pk=1"), row((Integer) null));
+        assertRows(execute("SELECT DISTINCT s2 FROM %s WHERE pk=1"), row(1));
+    }
+
+    @Test
+    public void testStaticColumnDeletionWithMultipleStaticColumnsAndRegularColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int, ck int, s1 int static, s2 int static, v int, PRIMARY KEY(pk, ck))");
+
+        execute("INSERT INTO %s (pk, ck, v, s2) VALUES (1, 1, 1, 1) USING TIMESTAMP 1000");
+        flush();
+        execute("INSERT INTO %s (pk, s1) VALUES (1, 2) USING TIMESTAMP 2000");
+        flush();
+        execute("DELETE s1 FROM %s USING TIMESTAMP 3000 WHERE pk = 1");
+        flush();
+
+        assertRows(execute("SELECT * FROM %s WHERE pk=1"), row(1, 1, null, 1, 1));
+        assertRows(execute("SELECT s1, s2 FROM %s WHERE pk=1"), row((Integer) null, 1));
+        assertRows(execute("SELECT s1 FROM %s WHERE pk=1"), row((Integer) null));

Review comment:
       Maybe we can add some `DISTINCT` queries here:
   ```suggestion
           assertRows(execute("SELECT * FROM %s WHERE pk=1"), row(1, 1, null, 1, 1));
           assertRows(execute("SELECT s1, s2 FROM %s WHERE pk=1"), row(null, 1));
           assertRows(execute("SELECT s1 FROM %s WHERE pk=1"), row((Integer) null));
           assertRows(execute("SELECT DISTINCT s1, s2 FROM %s WHERE pk=1"), row(null, 1));
           assertRows(execute("SELECT DISTINCT s1 FROM %s WHERE pk=1"), row((Integer) null));
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -502,149 +561,473 @@ public ColumnFilter build()
             boolean isFetchAll = metadata != null;
 
             RegularAndStaticColumns queried = queriedBuilder == null ? null : queriedBuilder.build();
+
             // It's only ok to have queried == null in ColumnFilter if isFetchAll. So deal with the case of a selectionBuilder
             // with nothing selected (we can at least happen on some backward compatible queries - CASSANDRA-10471).
             if (!isFetchAll && queried == null)
                 queried = RegularAndStaticColumns.NONE;
 
-            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = null;
-            if (subSelections != null)
+            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = buildSubSelections();
+
+            if (isFetchAll)
             {
-                s = TreeMultimap.create(Comparator.naturalOrder(), Comparator.naturalOrder());
-                for (ColumnSubselection subSelection : subSelections)
+                // When fetchAll is enabled on pre CASSANDRA-10657 (3.4-), queried columns are not considered at all, and it
+                // is assumed that all columns are queried. CASSANDRA-10657 (3.4+) brings back skipping values of columns
+                // which are not in queried set when fetchAll is enabled. That makes exactly the same filter being
+                // interpreted in a different way on 3.4- and 3.4+.
+                //
+                // Moreover, there is no way to convert the filter with fetchAll and queried != null so that it is
+                // interpreted the same way on 3.4- because that Cassandra version does not support such filtering.
+                //
+                // In order to avoid inconsitencies in data read by 3.4- and 3.4+ we need to avoid creation of incompatible
+                // filters when the cluster contains 3.4- nodes. We do that by forcibly setting queried to null.
+                //
+                // see CASSANDRA-10657, CASSANDRA-15833, CASSANDRA-16415
+                if (queried == null || isUpgradingFromVersionLowerThan34())
+                {
+                    return new WildCardColumnFilter(metadata.regularAndStaticColumns());
+                }
+
+                // pre CASSANDRA-12768 (4.0-) all static columns should be fetched along with all regular columns.
+                if (isUpgradingFromVersionLowerThan40())
                 {
-                    if (fullySelectedComplexColumns == null || !fullySelectedComplexColumns.contains(subSelection.column()))
-                        s.put(subSelection.column().name, subSelection);
+                    return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_COLUMNS, metadata, queried, s);
                 }
+
+                // pre CASSANDRA-16686 (4.0-RC2-) static columns where not fetched unless queried witch lead to some wrong results
+                // for some queries
+                if (isUpgradingFromVersionLowerThan40RC2() || !returnStaticContentOnPartitionWithNoRows)
+                {
+                    return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS, metadata, queried, s);
+                }
+
+                return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_COLUMNS, metadata, queried, s);
+            }
+
+            return SelectionColumnFilter.newInstance(FetchingStrategy.ONLY_QUERIED_COLUMNS, (TableMetadata) null, queried, s);
+        }
+
+        private SortedSetMultimap<ColumnIdentifier, ColumnSubselection> buildSubSelections()
+        {
+            if (subSelections == null)
+                return null;
+
+            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = TreeMultimap.create(Comparator.naturalOrder(), Comparator.naturalOrder());
+            for (ColumnSubselection subSelection : subSelections)
+            {
+                if (fullySelectedComplexColumns == null || !fullySelectedComplexColumns.contains(subSelection.column()))
+                    s.put(subSelection.column().name, subSelection);
             }
 
-            // When fetchAll is enabled on pre CASSANDRA-10657 (3.4-), queried columns are not considered at all, and it
-            // is assumed that all columns are queried. CASSANDRA-10657 (3.4+) brings back skipping values of columns
-            // which are not in queried set when fetchAll is enabled. That makes exactly the same filter being
-            // interpreted in a different way on 3.4- and 3.4+.
-            //
-            // Moreover, there is no way to convert the filter with fetchAll and queried != null so that it is
-            // interpreted the same way on 3.4- because that Cassandra version does not support such filtering.
-            //
-            // In order to avoid inconsitencies in data read by 3.4- and 3.4+ we need to avoid creation of incompatible
-            // filters when the cluster contains 3.4- nodes. We do that by forcibly setting queried to null.
-            //
-            // see CASSANDRA-10657, CASSANDRA-15833, CASSANDRA-16415
-            return new ColumnFilter(isFetchAll, isFetchAll && shouldFetchAllStatics(), metadata, isFetchAll && shouldQueriedBeNull() ? null : queried, s);
+            return s;
         }
     }
 
-    @Override
-    public boolean equals(Object other)
+    /**
+     * {@code ColumnFilter} sub-class for wildcard queries.
+     *
+     * <p>The class  does not rely on TableMetadata and expect a fix set of columns to prevent issues
+     * with Schema race propagation. See CASSANDRA-15899.</p>
+     */
+    public static class WildCardColumnFilter extends ColumnFilter
     {
-        if (other == this)
+        /**
+         * The queried and fetched columns.
+         */
+        private final RegularAndStaticColumns fetchedAndQueried;
+
+        /**
+         * Creates a {@code ColumnFilter} for wildcard queries.
+         *
+         * <p>The class  does not rely on TableMetadata and expect a fix set of columns to prevent issues

Review comment:
       ```suggestion
            * <p>The class does not rely on TableMetadata and expects a fix set of columns to prevent issues
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -502,149 +561,473 @@ public ColumnFilter build()
             boolean isFetchAll = metadata != null;
 
             RegularAndStaticColumns queried = queriedBuilder == null ? null : queriedBuilder.build();
+
             // It's only ok to have queried == null in ColumnFilter if isFetchAll. So deal with the case of a selectionBuilder
             // with nothing selected (we can at least happen on some backward compatible queries - CASSANDRA-10471).
             if (!isFetchAll && queried == null)
                 queried = RegularAndStaticColumns.NONE;
 
-            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = null;
-            if (subSelections != null)
+            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = buildSubSelections();
+
+            if (isFetchAll)
             {
-                s = TreeMultimap.create(Comparator.naturalOrder(), Comparator.naturalOrder());
-                for (ColumnSubselection subSelection : subSelections)
+                // When fetchAll is enabled on pre CASSANDRA-10657 (3.4-), queried columns are not considered at all, and it
+                // is assumed that all columns are queried. CASSANDRA-10657 (3.4+) brings back skipping values of columns
+                // which are not in queried set when fetchAll is enabled. That makes exactly the same filter being
+                // interpreted in a different way on 3.4- and 3.4+.
+                //
+                // Moreover, there is no way to convert the filter with fetchAll and queried != null so that it is
+                // interpreted the same way on 3.4- because that Cassandra version does not support such filtering.
+                //
+                // In order to avoid inconsitencies in data read by 3.4- and 3.4+ we need to avoid creation of incompatible
+                // filters when the cluster contains 3.4- nodes. We do that by forcibly setting queried to null.
+                //
+                // see CASSANDRA-10657, CASSANDRA-15833, CASSANDRA-16415
+                if (queried == null || isUpgradingFromVersionLowerThan34())
+                {
+                    return new WildCardColumnFilter(metadata.regularAndStaticColumns());
+                }
+
+                // pre CASSANDRA-12768 (4.0-) all static columns should be fetched along with all regular columns.
+                if (isUpgradingFromVersionLowerThan40())
                 {
-                    if (fullySelectedComplexColumns == null || !fullySelectedComplexColumns.contains(subSelection.column()))
-                        s.put(subSelection.column().name, subSelection);
+                    return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_COLUMNS, metadata, queried, s);
                 }
+
+                // pre CASSANDRA-16686 (4.0-RC2-) static columns where not fetched unless queried witch lead to some wrong results
+                // for some queries
+                if (isUpgradingFromVersionLowerThan40RC2() || !returnStaticContentOnPartitionWithNoRows)
+                {
+                    return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS, metadata, queried, s);
+                }
+
+                return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_COLUMNS, metadata, queried, s);
+            }
+
+            return SelectionColumnFilter.newInstance(FetchingStrategy.ONLY_QUERIED_COLUMNS, (TableMetadata) null, queried, s);
+        }
+
+        private SortedSetMultimap<ColumnIdentifier, ColumnSubselection> buildSubSelections()
+        {
+            if (subSelections == null)
+                return null;
+
+            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = TreeMultimap.create(Comparator.naturalOrder(), Comparator.naturalOrder());
+            for (ColumnSubselection subSelection : subSelections)
+            {
+                if (fullySelectedComplexColumns == null || !fullySelectedComplexColumns.contains(subSelection.column()))
+                    s.put(subSelection.column().name, subSelection);
             }
 
-            // When fetchAll is enabled on pre CASSANDRA-10657 (3.4-), queried columns are not considered at all, and it
-            // is assumed that all columns are queried. CASSANDRA-10657 (3.4+) brings back skipping values of columns
-            // which are not in queried set when fetchAll is enabled. That makes exactly the same filter being
-            // interpreted in a different way on 3.4- and 3.4+.
-            //
-            // Moreover, there is no way to convert the filter with fetchAll and queried != null so that it is
-            // interpreted the same way on 3.4- because that Cassandra version does not support such filtering.
-            //
-            // In order to avoid inconsitencies in data read by 3.4- and 3.4+ we need to avoid creation of incompatible
-            // filters when the cluster contains 3.4- nodes. We do that by forcibly setting queried to null.
-            //
-            // see CASSANDRA-10657, CASSANDRA-15833, CASSANDRA-16415
-            return new ColumnFilter(isFetchAll, isFetchAll && shouldFetchAllStatics(), metadata, isFetchAll && shouldQueriedBeNull() ? null : queried, s);
+            return s;
         }
     }
 
-    @Override
-    public boolean equals(Object other)
+    /**
+     * {@code ColumnFilter} sub-class for wildcard queries.
+     *
+     * <p>The class  does not rely on TableMetadata and expect a fix set of columns to prevent issues

Review comment:
       ```suggestion
        * <p>The class does not rely on TableMetadata and expects a fix set of columns to prevent issues
   ```

##########
File path: test/unit/org/apache/cassandra/cql3/validation/operations/CompactTableTest.java
##########
@@ -112,4 +112,21 @@ public void compactStorageSemanticsTest() throws Throwable
         assertRows(execute("SELECT * FROM %s WHERE pk = ?",2),
                    row(2, 2, null, 2));
     }
+
+    @Test
+    public void testColumnDeletionWithCompactTableWithMultipleColumns() throws Throwable
+    {
+        createTable("CREATE TABLE %s (pk int PRIMARY KEY, v1 int, v2 int) WITH COMPACT STORAGE");
+
+        execute("INSERT INTO %s (pk, v1, v2) VALUES (1, 1, 1) USING TIMESTAMP 1000");
+        flush();
+        execute("INSERT INTO %s (pk, v1) VALUES (1, 2) USING TIMESTAMP 2000");
+        flush();
+        execute("DELETE v1 FROM %s USING TIMESTAMP 3000 WHERE pk = 1");
+        flush();
+
+        assertRows(execute("SELECT * FROM %s WHERE pk=1"), row(1, null, 1));
+        assertRows(execute("SELECT v1, v2 FROM %s WHERE pk=1"), row((Integer) null, 1));
+        assertRows(execute("SELECT v1 FROM %s WHERE pk=1"), row((Integer) null)); // <-fail

Review comment:
       ```suggestion
           assertRows(execute("SELECT v1, v2 FROM %s WHERE pk=1"), row(null, 1));
           assertRows(execute("SELECT v1 FROM %s WHERE pk=1"), row((Integer) null));
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -502,149 +561,473 @@ public ColumnFilter build()
             boolean isFetchAll = metadata != null;
 
             RegularAndStaticColumns queried = queriedBuilder == null ? null : queriedBuilder.build();
+
             // It's only ok to have queried == null in ColumnFilter if isFetchAll. So deal with the case of a selectionBuilder
             // with nothing selected (we can at least happen on some backward compatible queries - CASSANDRA-10471).
             if (!isFetchAll && queried == null)
                 queried = RegularAndStaticColumns.NONE;
 
-            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = null;
-            if (subSelections != null)
+            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = buildSubSelections();
+
+            if (isFetchAll)
             {
-                s = TreeMultimap.create(Comparator.naturalOrder(), Comparator.naturalOrder());
-                for (ColumnSubselection subSelection : subSelections)
+                // When fetchAll is enabled on pre CASSANDRA-10657 (3.4-), queried columns are not considered at all, and it
+                // is assumed that all columns are queried. CASSANDRA-10657 (3.4+) brings back skipping values of columns
+                // which are not in queried set when fetchAll is enabled. That makes exactly the same filter being
+                // interpreted in a different way on 3.4- and 3.4+.
+                //
+                // Moreover, there is no way to convert the filter with fetchAll and queried != null so that it is
+                // interpreted the same way on 3.4- because that Cassandra version does not support such filtering.
+                //
+                // In order to avoid inconsitencies in data read by 3.4- and 3.4+ we need to avoid creation of incompatible
+                // filters when the cluster contains 3.4- nodes. We do that by forcibly setting queried to null.
+                //
+                // see CASSANDRA-10657, CASSANDRA-15833, CASSANDRA-16415
+                if (queried == null || isUpgradingFromVersionLowerThan34())
+                {
+                    return new WildCardColumnFilter(metadata.regularAndStaticColumns());
+                }
+
+                // pre CASSANDRA-12768 (4.0-) all static columns should be fetched along with all regular columns.
+                if (isUpgradingFromVersionLowerThan40())
                 {
-                    if (fullySelectedComplexColumns == null || !fullySelectedComplexColumns.contains(subSelection.column()))
-                        s.put(subSelection.column().name, subSelection);
+                    return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_COLUMNS, metadata, queried, s);
                 }
+
+                // pre CASSANDRA-16686 (4.0-RC2-) static columns where not fetched unless queried witch lead to some wrong results
+                // for some queries
+                if (isUpgradingFromVersionLowerThan40RC2() || !returnStaticContentOnPartitionWithNoRows)
+                {
+                    return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS, metadata, queried, s);
+                }
+
+                return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_COLUMNS, metadata, queried, s);
+            }
+
+            return SelectionColumnFilter.newInstance(FetchingStrategy.ONLY_QUERIED_COLUMNS, (TableMetadata) null, queried, s);
+        }
+
+        private SortedSetMultimap<ColumnIdentifier, ColumnSubselection> buildSubSelections()
+        {
+            if (subSelections == null)
+                return null;
+
+            SortedSetMultimap<ColumnIdentifier, ColumnSubselection> s = TreeMultimap.create(Comparator.naturalOrder(), Comparator.naturalOrder());
+            for (ColumnSubselection subSelection : subSelections)
+            {
+                if (fullySelectedComplexColumns == null || !fullySelectedComplexColumns.contains(subSelection.column()))
+                    s.put(subSelection.column().name, subSelection);
             }
 
-            // When fetchAll is enabled on pre CASSANDRA-10657 (3.4-), queried columns are not considered at all, and it
-            // is assumed that all columns are queried. CASSANDRA-10657 (3.4+) brings back skipping values of columns
-            // which are not in queried set when fetchAll is enabled. That makes exactly the same filter being
-            // interpreted in a different way on 3.4- and 3.4+.
-            //
-            // Moreover, there is no way to convert the filter with fetchAll and queried != null so that it is
-            // interpreted the same way on 3.4- because that Cassandra version does not support such filtering.
-            //
-            // In order to avoid inconsitencies in data read by 3.4- and 3.4+ we need to avoid creation of incompatible
-            // filters when the cluster contains 3.4- nodes. We do that by forcibly setting queried to null.
-            //
-            // see CASSANDRA-10657, CASSANDRA-15833, CASSANDRA-16415
-            return new ColumnFilter(isFetchAll, isFetchAll && shouldFetchAllStatics(), metadata, isFetchAll && shouldQueriedBeNull() ? null : queried, s);
+            return s;
         }
     }
 
-    @Override
-    public boolean equals(Object other)
+    /**
+     * {@code ColumnFilter} sub-class for wildcard queries.
+     *
+     * <p>The class  does not rely on TableMetadata and expect a fix set of columns to prevent issues
+     * with Schema race propagation. See CASSANDRA-15899.</p>
+     */
+    public static class WildCardColumnFilter extends ColumnFilter
     {
-        if (other == this)
+        /**
+         * The queried and fetched columns.
+         */
+        private final RegularAndStaticColumns fetchedAndQueried;
+
+        /**
+         * Creates a {@code ColumnFilter} for wildcard queries.
+         *
+         * <p>The class  does not rely on TableMetadata and expect a fix set of columns to prevent issues
+         * with Schema race propagation. See CASSANDRA-15899.</p>
+         *
+         * @param fetchedAndQueried the fetched and queried columns
+         */
+        private WildCardColumnFilter(RegularAndStaticColumns fetchedAndQueried)
+        {
+            this.fetchedAndQueried = fetchedAndQueried;
+        }
+
+        @Override
+        public RegularAndStaticColumns fetchedColumns()
+        {
+            return fetchedAndQueried;
+        }
+
+        @Override
+        public RegularAndStaticColumns queriedColumns()
+        {
+            return fetchedAndQueried;
+        }
+
+        @Override
+        public boolean fetchesAllColumns(boolean isStatic)
+        {
             return true;
+        }
 
-        if (!(other instanceof ColumnFilter))
-            return false;
+        @Override
+        public boolean allFetchedColumnsAreQueried()
+        {
+            return true;
+        }
 
-        ColumnFilter otherCf = (ColumnFilter) other;
+        @Override
+        public boolean fetches(ColumnMetadata column)
+        {
+            return true;
+        }
 
-        return otherCf.fetchAllRegulars == this.fetchAllRegulars &&
-               otherCf.fetchAllStatics == this.fetchAllStatics &&
-               Objects.equals(otherCf.fetched, this.fetched) &&
-               Objects.equals(otherCf.queried, this.queried) &&
-               Objects.equals(otherCf.subSelections, this.subSelections);
-    }
+        @Override
+        public boolean fetchedColumnIsQueried(ColumnMetadata column)
+        {
+            return true;
+        }
 
-    @Override
-    public String toString()
-    {
-        String prefix = "";
+        @Override
+        public boolean fetchedCellIsQueried(ColumnMetadata column, CellPath path)
+        {
+            return true;
+        }
 
-        if (fetchAllRegulars && queried == null)
-            return "*/*";
+        @Override
+        public Iterator<Cell<?>> filterComplexCells(ColumnMetadata column, Iterator<Cell<?>> cells)
+        {
+            return cells;
+        }
 
-        if (fetchAllRegulars && fetchAllStatics)
-            prefix = "*/";
+        @Override
+        public Tester newTester(ColumnMetadata column)
+        {
+            return null;
+        }
 
-        if (fetchAllRegulars && !fetchAllStatics)
+        @Override
+        public boolean equals(Object other)
         {
-            prefix = queried.statics.isEmpty()
-                   ? "<all regulars>/"
-                   : String.format("<all regulars>+%s/", toString(queried.statics.selectOrderIterator(), false));
+            if (other == this)
+                return true;
+
+            if (!(other instanceof WildCardColumnFilter))
+                return false;
+
+            WildCardColumnFilter w = (WildCardColumnFilter) other;
+
+            return fetchedAndQueried.equals(w.fetchedAndQueried);
         }
 
-        return prefix + toString(queried.selectOrderIterator(), false);
-    }
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(fetchedAndQueried);
+        }
 
-    public String toCQLString()
-    {
-        if (queried == null || queried.isEmpty())
+        @Override
+        public String toString()
+        {
+            return "*/*";
+        }
+
+        public String toCQLString()
+        {
             return "*";
+        }
+
+        @Override
+        public boolean isWildcard()
+        {
+            return true;
+        }
 
-        return toString(queried.selectOrderIterator(), true);
+        @Override
+        protected SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections()
+        {
+            return null;
+        }
     }
 
-    private String toString(Iterator<ColumnMetadata> columns, boolean cql)
+    /**
+     * {@code ColumnFilter} sub-class for queries with selected columns.
+     *
+     * <p>The class  does not rely on TableMetadata and expect a fix set of fetched columns to prevent issues
+     * with Schema race propagation. See CASSANDRA-15899.</p>
+     */
+    public static class SelectionColumnFilter extends ColumnFilter
     {
-        StringJoiner joiner = cql ? new StringJoiner(", ") : new StringJoiner(", ", "[", "]");
+        public final FetchingStrategy fetchingStrategy;
+
+        /**
+         * The selected columns
+         */
+        private final RegularAndStaticColumns queried;
+
+        /**
+         * The columns that need to be fetched to be able
+         */
+        private final RegularAndStaticColumns fetched;
+
+        private final SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections; // can be null
+
+        public static SelectionColumnFilter newInstance(FetchingStrategy fetchingStrategy,
+                                                        TableMetadata metadata,
+                                                        RegularAndStaticColumns queried,
+                                                        SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections)
+        {
+            assert fetchingStrategy != FetchingStrategy.ONLY_QUERIED_COLUMNS || metadata == null;
+            assert queried != null;
+
+            return new SelectionColumnFilter(fetchingStrategy,
+                                             queried,
+                                             fetchingStrategy.getFetchedColumns(metadata, queried),
+                                             subSelections);
+        }
+
+        /**
+         * Creates a {@code ColumnFilter} for queries with selected columns.
+         *
+         * <p>The class  does not rely on TableMetadata and expect a fix set of columns to prevent issues
+         * with Schema race propagation. See CASSANDRA-15899.</p>
+         *
+         * @param fetchingStrategy the strategy used to select the fetched columns
+         * @param fetched the columns that must be fetched
+         * @param queried the queried columns
+         * @param subSelections the columns sub-selections
+         */
+        public SelectionColumnFilter(FetchingStrategy fetchingStrategy,
+                                     RegularAndStaticColumns queried,
+                                     RegularAndStaticColumns fetched,
+                                     SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections)
+        {
+            assert fetched.includes(queried);
+
+            this.fetchingStrategy = fetchingStrategy;
+            this.queried = queried;
+            this.fetched = fetched;
+            this.subSelections = subSelections;
+        }
+
+        @Override
+        public RegularAndStaticColumns fetchedColumns()
+        {
+            return fetched;
+        }
+
+        @Override
+        public RegularAndStaticColumns queriedColumns()
+        {
+            return queried;
+        }
+
+        @Override
+        public boolean fetchesAllColumns(boolean isStatic)
+        {
+            return fetchingStrategy.fetchesAllColumns(isStatic);
+        }
+
+        @Override
+        public boolean allFetchedColumnsAreQueried()
+        {
+            return fetchingStrategy.areAllFetchedColumnsQueried();
+        }
+
+        @Override
+        public boolean fetches(ColumnMetadata column)
+        {
+            return fetchingStrategy.fetchesAllColumns(column.isStatic()) || queried.contains(column);
+        }
+
+        @Override
+        public boolean fetchedColumnIsQueried(ColumnMetadata column)
+        {
+            return fetchingStrategy.areAllFetchedColumnsQueried() || queried.contains(column);
+        }
+
+        @Override
+        public boolean fetchedCellIsQueried(ColumnMetadata column, CellPath path)
+        {
+            assert path != null;
+
+            // first verify that the column to which the cell belongs is queried
+            if (!fetchedColumnIsQueried(column))
+                return false;
+
+            if (subSelections == null)
+                return true;
+
+            SortedSet<ColumnSubselection> s = subSelections.get(column.name);
+            // No subsection for this column means everything is queried
+            if (s.isEmpty())
+                return true;
+
+            for (ColumnSubselection subSel : s)
+                if (subSel.compareInclusionOf(path) == 0)
+                    return true;
+
+            return false;
+        }
 
-        while (columns.hasNext())
+        @Override
+        public Iterator<Cell<?>> filterComplexCells(ColumnMetadata column, Iterator<Cell<?>> cells)
         {
-            ColumnMetadata column = columns.next();
-            String columnName = cql ? column.name.toCQLString() : String.valueOf(column.name);
+            Tester tester = newTester(column);
+            if (tester == null)
+                return cells;
 
-            SortedSet<ColumnSubselection> s = subSelections != null
-                                            ? subSelections.get(column.name)
-                                            : Collections.emptySortedSet();
+            return Iterators.filter(cells, cell -> tester.fetchedCellIsQueried(cell.path()));
+        }
 
+        @Override
+        public Tester newTester(ColumnMetadata column)
+        {
+            if (subSelections == null || !column.isComplex())
+                return null;
+
+            SortedSet<ColumnSubselection> s = subSelections.get(column.name);
             if (s.isEmpty())
-                joiner.add(columnName);
-            else
-                s.forEach(subSel -> joiner.add(String.format("%s%s", columnName, subSel)));
+                return null;
+
+            return new Tester(fetchingStrategy.fetchesAllColumns(column.isStatic()), s.iterator());
+        }
+
+        @Override
+        protected SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections()
+        {
+            return subSelections;
+        }
+
+        @Override
+        public boolean equals(Object other)
+        {
+            if (other == this)
+                return true;
+
+            if (!(other instanceof SelectionColumnFilter))
+                return false;
+
+            SelectionColumnFilter otherCf = (SelectionColumnFilter) other;
+
+            return otherCf.fetchingStrategy == this.fetchingStrategy &&
+                   Objects.equals(otherCf.queried, this.queried) &&
+                   Objects.equals(otherCf.fetched, this.fetched) &&
+                   Objects.equals(otherCf.subSelections, this.subSelections);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Objects.hash(fetchingStrategy, queried, fetched, subSelections);
+        }
+
+        @Override
+        public String toString()
+        {
+            String prefix = "";
+
+            if (fetchingStrategy.fetchesAllColumns(true))
+                prefix = "*/";
+
+            if (fetchingStrategy == FetchingStrategy.ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS)
+            {
+                prefix = queried.statics.isEmpty()
+                       ? "<all regulars>/"
+                       : String.format("<all regulars>+%s/", toString(queried.statics.selectOrderIterator(), false));
+            }
+
+            return prefix + toString(queried.selectOrderIterator(), false);
+        }
+
+        public String toCQLString()

Review comment:
       ```suggestion
           @Override
           public String toCQLString()
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -49,111 +48,166 @@
  * in its request.
  *
  * The reason for distinguishing those 2 sets is that due to the CQL semantic (see #6588 for more details), we
- * often need to internally fetch all regular columns for the queried table, but can still do some optimizations for
- * those columns that are not directly queried by the user (see #10657 for more details).
+ * often need to internally fetch all regular columns or all columns for the queried table, but can still do some
+ * optimizations for those columns that are not directly queried by the user (see #10657 for more details).
  *
  * Note that in practice:
  *   - the _queried_ columns set is always included in the _fetched_ one.
- *   - whenever those sets are different, we know 1) the _fetched_ set contains all regular columns for the table and 2)
- *     _fetched_ == _queried_ for static columns, so we don't have to record this set, we just keep a pointer to the
- *     table metadata. The only set we concretely store is thus the _queried_ one.
+ *   - whenever those sets are different, the _fetched_ columns can contains either all the regular columns and

Review comment:
       ```suggestion
    *   - whenever those sets are different, the _fetched_ columns can contain either all the regular columns and
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -325,50 +375,51 @@ public boolean fetchedCellIsQueried(ColumnMetadata column, CellPath path)
      * @return the created tester or {@code null} if all the cells from the provided column
      * are queried.
      */
-    public Tester newTester(ColumnMetadata column)
-    {
-        if (subSelections == null || !column.isComplex())
-            return null;
-
-        SortedSet<ColumnSubselection> s = subSelections.get(column.name);
-        if (s.isEmpty())
-            return null;
+    public abstract Tester newTester(ColumnMetadata column);

Review comment:
       Can we mark the method as `Nullable`?
   ```suggestion
       @Nullable
       public abstract Tester newTester(ColumnMetadata column);
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -49,111 +48,166 @@
  * in its request.
  *
  * The reason for distinguishing those 2 sets is that due to the CQL semantic (see #6588 for more details), we
- * often need to internally fetch all regular columns for the queried table, but can still do some optimizations for
- * those columns that are not directly queried by the user (see #10657 for more details).
+ * often need to internally fetch all regular columns or all columns for the queried table, but can still do some
+ * optimizations for those columns that are not directly queried by the user (see #10657 for more details).
  *
  * Note that in practice:
  *   - the _queried_ columns set is always included in the _fetched_ one.
- *   - whenever those sets are different, we know 1) the _fetched_ set contains all regular columns for the table and 2)
- *     _fetched_ == _queried_ for static columns, so we don't have to record this set, we just keep a pointer to the
- *     table metadata. The only set we concretely store is thus the _queried_ one.
+ *   - whenever those sets are different, the _fetched_ columns can contains either all the regular columns and
+ *     the static columns queried by the user or all the regular and static queries. If the query is a partition level
+ *     query (no restrictions on clustering or regular columns) all the static columns will need to be fetched as
+ *     some data will need to be returned to the user if the partition has no row but some static data. For all the
+ *     other scenarios only the all the regular columns are required.

Review comment:
       ```suggestion
    *     other scenarios only the regular columns are required.
   ```

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -665,49 +1046,99 @@ public ColumnFilter deserialize(DataInputPlus in, int version, TableMetadata met
 
             if (hasQueried)
             {
-                Columns statics = Columns.serializer.deserializeStatics(in, metadata);
-                Columns regulars = Columns.serializer.deserializeRegulars(in, metadata);
-                queried = new RegularAndStaticColumns(statics, regulars);
+                queried = deserializeRegularAndStaticColumns(in, metadata);
             }
 
             SortedSetMultimap<ColumnIdentifier, ColumnSubselection> subSelections = null;
             if (hasSubSelections)
             {
-                subSelections = TreeMultimap.create(Comparator.naturalOrder(), Comparator.naturalOrder());
-                int size = (int) in.readUnsignedVInt();
-                for (int i = 0; i < size; i++)
+                subSelections = deserializeSubSelection(in, version, metadata);
+            }
+
+            if (isFetchAll)
+            {
+                // pre CASSANDRA-10657 (3.4-), when fetchAll is enabled, queried columns are not considered at all, and it
+                // is assumed that all columns are queried.
+                if (!hasQueried || isUpgradingFromVersionLowerThan34())
+                {
+                    return new WildCardColumnFilter(fetched);
+                }
+
+                // pre CASSANDRA-12768 (4.0-) all static columns should be fetched along with all regular columns.
+                if (isUpgradingFromVersionLowerThan40())
+                {
+                    return new SelectionColumnFilter(FetchingStrategy.ALL_COLUMNS, queried, fetched, subSelections);
+                }
+
+                // pre CASSANDRA-16686 (4.0-RC2-) static columns where not fetched unless queried witch lead to some wrong results
+                // for some queries
+                if (isUpgradingFromVersionLowerThan40RC2() || !isFetchAllStatics)
                 {
-                    ColumnSubselection subSel = ColumnSubselection.serializer.deserialize(in, version, metadata);
-                    subSelections.put(subSel.column().name, subSel);
+                    return new SelectionColumnFilter(FetchingStrategy.ALL_REGULARS_AND_QUERIED_STATICS_COLUMNS, queried, fetched, subSelections);
                 }
+
+                return new SelectionColumnFilter(FetchingStrategy.ALL_COLUMNS, queried, fetched, subSelections);
             }
 
-            return new ColumnFilter(isFetchAll, isFetchAll && shouldFetchAllStatics(), fetched, isFetchAll && shouldQueriedBeNull() ? null : queried, subSelections);
+            return new SelectionColumnFilter(FetchingStrategy.ONLY_QUERIED_COLUMNS, queried, queried, subSelections);

Review comment:
       Maybe we can assert that `queried` is not null to get rid of the warning

##########
File path: src/java/org/apache/cassandra/db/filter/ColumnFilter.java
##########
@@ -201,40 +260,59 @@ public static ColumnFilter all(TableMetadata metadata)
      */
     public static ColumnFilter selection(RegularAndStaticColumns columns)
     {
-        return new ColumnFilter(false, false, (TableMetadata) null, columns, null);
+        return SelectionColumnFilter.newInstance(FetchingStrategy.ONLY_QUERIED_COLUMNS,
+                                                 (TableMetadata) null,
+                                                 columns,
+                                                 null);
     }
 
     /**
      * A filter that fetches all columns for the provided table, but returns
      * only the queried ones.
      */
-    public static ColumnFilter selection(TableMetadata metadata, RegularAndStaticColumns queried)
+    public static ColumnFilter selection(TableMetadata metadata,
+                                          RegularAndStaticColumns queried,
+                                          boolean returnStaticContentOnPartitionWithNoRows)
     {
-        return new ColumnFilter(true, shouldFetchAllStatics(), metadata, shouldQueriedBeNull() ? null : queried, null);
+        // pre CASSANDRA-10657 (3.4-), when fetchAll is enabled, queried columns are not considered at all, and it
+        // is assumed that all columns are queried.
+        if (isUpgradingFromVersionLowerThan34())
+        {
+            return new WildCardColumnFilter(metadata.regularAndStaticColumns());
+        }
+
+        // pre CASSANDRA-12768 (4.0-) all static columns should be fetched along with all regular columns.
+        if (isUpgradingFromVersionLowerThan40())
+        {
+            return SelectionColumnFilter.newInstance(FetchingStrategy.ALL_COLUMNS, metadata, queried, null);
+        }
+
+        // pre CASSANDRA-16686 (4.0-RC2-) static columns where not fetched unless queried witch lead to some wrong results
+        // for some queries
+        if (isUpgradingFromVersionLowerThan40RC2() || !returnStaticContentOnPartitionWithNoRows)

Review comment:
       Maybe we can check the simpler `returnStaticContentOnPartitionWithNoRows` first in the lazy or:
   ```suggestion
           if (!returnStaticContentOnPartitionWithNoRows || isUpgradingFromVersionLowerThan40RC2())
   ```




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

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