cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From slebre...@apache.org
Subject [3/6] cassandra git commit: Revert CASSANDRA-12768 (and update fix of CASSANDRA-12694) due to upgrade regression
Date Fri, 27 Jan 2017 08:32:46 GMT
Revert CASSANDRA-12768 (and update fix of CASSANDRA-12694) due to upgrade regression

patch by Sylvain Lebresne; reviewed by Benjamin Lerer for CASSANDRA-13025


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e1da99a1
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e1da99a1
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e1da99a1

Branch: refs/heads/trunk
Commit: e1da99a1d82b5313f022307c2e110afc07e112b6
Parents: 4bbf993
Author: Sylvain Lebresne <sylvain@datastax.com>
Authored: Mon Jan 23 10:08:40 2017 +0100
Committer: Sylvain Lebresne <sylvain@datastax.com>
Committed: Fri Jan 27 09:24:07 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 -
 .../cql3/statements/CQL3CasRequest.java         | 27 ++++----
 .../cql3/statements/SelectStatement.java        | 26 ++------
 .../db/filter/ClusteringIndexNamesFilter.java   | 12 ++--
 .../cassandra/db/filter/ColumnFilter.java       | 67 +++++---------------
 .../org/apache/cassandra/db/rows/BTreeRow.java  |  2 +-
 6 files changed, 38 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e1da99a1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 396fa3f..3796a8d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,7 +17,6 @@
  * Nodetool compactionstats fails with NullPointerException (CASSANDRA-13021)
  * Thread local pools never cleaned up (CASSANDRA-13033)
  * Set RPC_READY to false when draining or if a node is marked as shutdown (CASSANDRA-12781)
- * CQL often queries static columns unnecessarily (CASSANDRA-12768)
  * Make sure sstables only get committed when it's safe to discard commit log records (CASSANDRA-12956)
  * Reject default_time_to_live option when creating or altering MVs (CASSANDRA-12868)
  * Nodetool should use a more sane max heap size (CASSANDRA-12739)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e1da99a1/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
index db8653d..e226a2a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CQL3CasRequest.java
@@ -157,21 +157,16 @@ public class CQL3CasRequest implements CASRequest
 
     private PartitionColumns columnsToRead()
     {
-        // If all our conditions are columns conditions (IF x = ?), then it's enough to query
-        // the columns from the conditions. If we have a IF EXISTS or IF NOT EXISTS however,
-        // we need to query all columns for the row since if the condition fails, we want
to
-        // return everything to the user. Static columns make this a bit more complex, in
that
-        // if an insert only static columns, then the existence condition applies only to
the
-        // static columns themselves, and so we don't want to include regular columns in
that
-        // case.
-        if (hasExists)
-        {
-            PartitionColumns allColumns = cfm.partitionColumns();
-            Columns statics = updatesStaticRow ? allColumns.statics : Columns.NONE;
-            Columns regulars = updatesRegularRows ? allColumns.regulars : Columns.NONE;
-            return new PartitionColumns(statics, regulars);
-        }
-        return conditionColumns;
+        PartitionColumns allColumns = cfm.partitionColumns();
+
+        // If we update static row, we won't have any conditions on regular rows.
+        // If we update regular row, we have to fetch all regular rows (which would satisfy
column condition) and
+        // static rows that take part in column condition.
+        // In both cases, we're fetching enough rows to distinguish between "all conditions
are nulls" and "row does not exist".
+        // We have to do this as we can't rely on row marker for that (see #6623)
+        Columns statics = updatesStaticRow ? allColumns.statics : conditionColumns.statics;
+        Columns regulars = updatesRegularRows ? allColumns.regulars : conditionColumns.regulars;
+        return new PartitionColumns(statics, regulars);
     }
 
     public SinglePartitionReadCommand readCommand(int nowInSec)
@@ -179,7 +174,7 @@ public class CQL3CasRequest implements CASRequest
         assert staticConditions != null || !conditions.isEmpty();
 
         // Fetch all columns, but query only the selected ones
-        ColumnFilter columnFilter = ColumnFilter.selection(cfm, columnsToRead());
+        ColumnFilter columnFilter = ColumnFilter.selection(columnsToRead());
 
         // With only a static condition, we still want to make the distinction between a
non-existing partition and one
         // that exists (has some live data) but has not static content. So we query the first
live row of the partition.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e1da99a1/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index f2aa030..aca6146 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -141,19 +141,13 @@ public class SelectStatement implements CQLStatement
         if (selection.isWildcard())
             return ColumnFilter.all(cfm);
 
-        ColumnFilter.Builder builder = ColumnFilter.allRegularColumnsBuilder(cfm);
+        ColumnFilter.Builder builder = ColumnFilter.allColumnsBuilder(cfm);
         // Adds all selected columns
         for (ColumnDefinition def : selection.getColumns())
             if (!def.isPrimaryKeyColumn())
                 builder.add(def);
         // as well as any restricted column (so we can actually apply the restriction)
         builder.addAll(restrictions.nonPKRestrictedColumns(true));
-
-        // In a number of cases, we want to distinguish between a partition truly empty and
one with only static content
-        // (but no rows). In those cases, we should force querying all static columns (to
make the distinction).
-        if (cfm.hasStaticColumns() && returnStaticContentOnPartitionWithNoRows())
-            builder.addAll(cfm.partitionColumns().statics);
-
         return builder.build();
     }
 
@@ -740,20 +734,6 @@ public class SelectStatement implements CQLStatement
         }
     }
 
-    // Determines whether, when we have a partition result with not rows, we still return
the static content (as a
-    // result set row with null for all other regular columns.)
-    private boolean returnStaticContentOnPartitionWithNoRows()
-    {
-        // The general rational is that if some rows are specifically selected by the query,
we ignore partitions that
-        // are empty outside of static content, but if it's a full partition query, then
we include that content.
-        // In practice, we consider rows are specifically selected if either there is some
restrictions on the
-        // clustering columns or it's a 2ndary index query (the later is debatable but historical).
An exception however
-        // is 'static compact' table, for which 2ndary index indexes full partition (and
so for which we consider 2ndary
-        // indexquery to be full partition query).
-        return !restrictions.hasClusteringColumnsRestriction()
-            && (!restrictions.usesSecondaryIndexing() || cfm.isStaticCompactTable());
-    }
-
     // Used by ModificationStatement for CAS operations
     void processPartition(RowIterator partition, QueryOptions options, Selection.ResultSetBuilder
result, int nowInSec)
     throws InvalidRequestException
@@ -764,10 +744,12 @@ public class SelectStatement implements CQLStatement
 
         Row staticRow = partition.staticRow();
         // If there is no rows, then provided the select was a full partition selection
+        // (i.e. not a 2ndary index search and there was no condition on clustering columns),
         // we want to include static columns and we're done.
         if (!partition.hasNext())
         {
-            if (!staticRow.isEmpty() && returnStaticContentOnPartitionWithNoRows())
+            if (!staticRow.isEmpty() && (!restrictions.usesSecondaryIndexing() ||
cfm.isStaticCompactTable())
+                    && !restrictions.hasClusteringColumnsRestriction())
             {
                 result.newRow(protocolVersion);
                 for (ColumnDefinition def : selection.getColumns())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e1da99a1/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
index ea5cf55..a81a7a6 100644
--- a/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ClusteringIndexNamesFilter.java
@@ -178,12 +178,12 @@ public class ClusteringIndexNamesFilter extends AbstractClusteringIndexFilter
     {
         final SearchIterator<Clustering, Row> searcher = partition.searchIterator(columnFilter,
reversed);
         return new AbstractUnfilteredRowIterator(partition.metadata(),
-                                                 partition.partitionKey(),
-                                                 partition.partitionLevelDeletion(),
-                                                 columnFilter.fetchedColumns(),
-                                                 searcher.next(Clustering.STATIC_CLUSTERING),
-                                                 reversed,
-                                                 partition.stats())
+                                        partition.partitionKey(),
+                                        partition.partitionLevelDeletion(),
+                                        columnFilter.fetchedColumns(),
+                                        searcher.next(Clustering.STATIC_CLUSTERING),
+                                        reversed,
+                                        partition.stats())
         {
             private final Iterator<Clustering> clusteringIter = clusteringsInQueryOrder.iterator();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e1da99a1/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
index 2377ad0..df91781 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
@@ -37,8 +37,8 @@ import org.apache.cassandra.io.util.DataOutputPlus;
  * by a query.
  *
  * In practice, this class cover 2 main cases:
- *   1) most user queries have to internally query all (regular) columns, because the CQL
semantic requires us to know
- *      if a row is live or not even if it has no values for the columns requested by the
user (see #6588 for more
+ *   1) most user queries have to internally query all columns, because the CQL semantic
requires us to know if
+ *      a row is live or not even if it has no values for the columns requested by the user
(see #6588for more
  *      details). However, while we need to know for columns if it has live values, we can
actually save from
  *      sending the values for those columns that will not be returned to the user.
  *   2) for some internal queries (and for queries using #6588 if we introduce it), we're
actually fine only
@@ -51,11 +51,8 @@ public class ColumnFilter
 {
     public static final Serializer serializer = new Serializer();
 
-    // Distinguish between the 2 cases described above: if 'isFetchAll' is true, then all
regular columns will be
-    // retrieved by the query. If selection is also null, then all static columns will be
fetched too. If 'isFetchAll'
-    // is true and selection is not null, then 1) for static columns, only the ones in selection
are read and 2) for
-    // regular columns, while all are fetches, the values for column/cells not selected by
'selection' and
-    // 'subSelections' will be skipped.
+    // Distinguish between the 2 cases described above: if 'isFetchAll' is true, then all
columns will be retrieved
+    // by the query, but the values for column/cells not selected by 'selection' and 'subSelections'
will be skipped.
     // Otherwise, only the column/cells returned by 'selection' and 'subSelections' will
be returned at all.
     private final boolean isFetchAll;
 
@@ -114,21 +111,12 @@ public class ColumnFilter
      */
     public PartitionColumns fetchedColumns()
     {
-        if (!isFetchAll)
-            return selection;
-
-        // We always fetch all regulars, but only fetch the statics in selection. Unless
selection is null, in which
-        // case it's a wildcard and we fetch everything.
-        PartitionColumns all = metadata.partitionColumns();
-        return selection == null || all.statics.isEmpty()
-             ? all
-             : new PartitionColumns(selection.statics, all.regulars);
+        return isFetchAll ? metadata.partitionColumns() : selection;
     }
 
-    public boolean includesAllColumns(boolean isStatic)
+    public boolean includesAllColumns()
     {
-        // Static columns are never all included, unless selection == null
-        return isStatic ? selection == null : isFetchAll;
+        return isFetchAll;
     }
 
     /**
@@ -136,11 +124,6 @@ public class ColumnFilter
      */
     public boolean includes(ColumnDefinition column)
     {
-        // For statics, it is included only if it's part of selection, or if selection is
null (wildcard query).
-        if (column.isStatic())
-            return selection == null || selection.contains(column);
-
-        // For regulars, if 'isFetchAll', then it's included automatically. Otherwise, it
depends on 'selection'.
         return isFetchAll || selection.contains(column);
     }
 
@@ -192,13 +175,8 @@ public class ColumnFilter
     }
 
     /**
-     * Creates a new {@code Tester} to efficiently test the inclusion of cells
-     * of an included complex column.
-     *
-     * @param column the complex column, which *must* be included by this
-     * filter (that is, we must have {@code this.includes(column)}).
-     * @retun the created tester or {@code null} if all the cells from {@code
-     * column} are included.
+     * Creates a new {@code Tester} to efficiently test the inclusion of cells of complex
column
+     * {@code column}.
      */
     public Tester newTester(ColumnDefinition column)
     {
@@ -209,15 +187,14 @@ public class ColumnFilter
         if (s.isEmpty())
             return null;
 
-        // isFetchAll only imply everything if fetches for regular
-        return new Tester(isFetchAll && !column.isStatic(), s.iterator());
+        return new Tester(isFetchAll, s.iterator());
     }
 
     /**
      * Returns a {@code ColumnFilter}} builder that includes all columns (so the selections
      * added to the builder are the columns/cells for which we shouldn't skip the values).
      */
-    public static Builder allRegularColumnsBuilder(CFMetaData metadata)
+    public static Builder allColumnsBuilder(CFMetaData metadata)
     {
         return new Builder(metadata);
     }
@@ -233,36 +210,24 @@ public class ColumnFilter
 
     public static class Tester
     {
-        private final boolean isFetched; // if true, all cells are included
+        private final boolean isFetchAll;
         private ColumnSubselection current;
         private final Iterator<ColumnSubselection> iterator;
 
-        private Tester(boolean isFetched, Iterator<ColumnSubselection> iterator)
+        private Tester(boolean isFetchAll, Iterator<ColumnSubselection> iterator)
         {
-            this.isFetched = isFetched;
+            this.isFetchAll = isFetchAll;
             this.iterator = iterator;
         }
 
         public boolean includes(CellPath path)
         {
-            // It's included if either all cells are fetched (because it's a
-            // regular column and the filter has 'isFetchAll == true'), or if
-            // it's explicitely selected.
-            return isFetched || includedBySubselection(path);
+            return isFetchAll || includedBySubselection(path);
         }
 
-        /**
-         * Must only be called if {@code includes(path) == true}.
-         */
         public boolean canSkipValue(CellPath path)
         {
-            // We can skip the value of an included column only if it's a
-            // regular column included due to the 'isFetchAll' flag, but which
-            // isn't explicitely selected. In practice, it's enough to not have
-            // the path explicitly selected as it implies the column was
-            // included due to 'isFetchAll' (since we require includes(path) to
-            // be called first).
-            return !includedBySubselection(path);
+            return isFetchAll && !includedBySubselection(path);
         }
 
         private boolean includedBySubselection(CellPath path)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e1da99a1/src/java/org/apache/cassandra/db/rows/BTreeRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/BTreeRow.java b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
index 18f3dec..ea1d9e0 100644
--- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java
+++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
@@ -237,7 +237,7 @@ public class BTreeRow extends AbstractRow
     {
         Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns = metadata.getDroppedColumns();
 
-        if (filter.includesAllColumns(isStatic()) && (activeDeletion.isLive() ||
deletion.supersedes(activeDeletion)) && droppedColumns.isEmpty())
+        if (filter.includesAllColumns() && (activeDeletion.isLive() || deletion.supersedes(activeDeletion))
&& droppedColumns.isEmpty())
             return this;
 
         boolean mayHaveShadowed = activeDeletion.supersedes(deletion.time());


Mime
View raw message