You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by pa...@apache.org on 2017/09/05 06:08:30 UTC

[03/16] cassandra git commit: Fix outstanding MV timestamp issues and add documentation about unsupported cases (see CASSANDRA-11500 for a summary of fixes)

Fix outstanding MV timestamp issues and add documentation about unsupported cases (see CASSANDRA-11500 for a summary of fixes)

This patch introduces the following changes to fix MV timestamp issues:
 - Add strict liveness for view with non-key base column in pk
 - Deprecated shadowable tombstone and use expired livenessInfo instead
 - Include partition deletion for existing base row
 - Disallow dropping base column with MV

Patch by Zhao Yang and Paulo Motta; reviewed by Paulo Motta for CASSANDRA-11500


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

Branch: refs/heads/cassandra-3.0
Commit: 1b36740ebe66b8ed4c3d6cb64eb2419a9279dfbf
Parents: b0eba5f
Author: Zhao Yang <zh...@gmail.com>
Authored: Wed Jul 12 17:49:38 2017 +0800
Committer: Paulo Motta <pa...@apache.org>
Committed: Tue Sep 5 01:03:24 2017 -0500

----------------------------------------------------------------------
 NEWS.txt                                        |   18 +
 doc/cql3/CQL.textile                            |    6 +
 .../org/apache/cassandra/config/CFMetaData.java |   13 +
 .../apache/cassandra/cql3/UpdateParameters.java |    2 +-
 .../cql3/statements/AlterTableStatement.java    |   18 +-
 .../org/apache/cassandra/db/LivenessInfo.java   |   17 +-
 .../org/apache/cassandra/db/ReadCommand.java    |    7 +-
 .../db/compaction/CompactionIterator.java       |    7 +-
 .../apache/cassandra/db/filter/RowFilter.java   |    4 +-
 .../cassandra/db/partitions/PurgeFunction.java  |   14 +-
 .../org/apache/cassandra/db/rows/BTreeRow.java  |    6 +-
 src/java/org/apache/cassandra/db/rows/Row.java  |   15 +-
 .../cassandra/db/rows/UnfilteredSerializer.java |    5 +
 .../apache/cassandra/db/transform/Filter.java   |    8 +-
 .../db/transform/FilteredPartitions.java        |    4 +-
 .../cassandra/db/transform/FilteredRows.java    |    2 +-
 .../apache/cassandra/db/view/TableViews.java    |   18 +-
 src/java/org/apache/cassandra/db/view/View.java |   43 +-
 .../apache/cassandra/db/view/ViewManager.java   |    5 +
 .../cassandra/db/view/ViewUpdateGenerator.java  |  163 ++-
 .../apache/cassandra/service/DataResolver.java  |    4 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |    2 +-
 .../apache/cassandra/cql3/ViewComplexTest.java  | 1343 ++++++++++++++++++
 .../cassandra/cql3/ViewFilteringTest.java       |  706 ++++-----
 .../org/apache/cassandra/cql3/ViewTest.java     |   31 +-
 25 files changed, 1973 insertions(+), 488 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index bb5fdfe..7064c5d 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -21,6 +21,24 @@ Upgrading
    - Nothing specific to this release, but please see previous upgrading sections,
      especially if you are upgrading from 2.2.
 
+Materialized Views
+-------------------
+    - Cassandra will no longer allow dropping columns on tables with Materialized Views.
+    - A change was made in the way the Materialized View timestamp is computed, which
+      may cause an old deletion to a base column which is view primary key (PK) column
+      to not be reflected in the view when repairing the base table post-upgrade. This
+      condition is only possible when a column deletion to an MV primary key (PK) column
+      not present in the base table PK (via UPDATE base SET view_pk_col = null or DELETE
+      view_pk_col FROM base) is missed before the upgrade and received by repair after the upgrade.
+      If such column deletions are done on a view PK column which is not a base PK, it's advisable
+      to run repair on the base table of all nodes prior to the upgrade. Alternatively it's possible
+      to fix potential inconsistencies by running repair on the views after upgrade or drop and
+      re-create the views. See CASSANDRA-11500 for more details.
+    - Removal of columns not selected in the Materialized View (via UPDATE base SET unselected_column
+      = null or DELETE unselected_column FROM base) may not be properly reflected in the view in some
+      situations so we advise against doing deletions on base columns not selected in views
+      until this is fixed on CASSANDRA-13826.
+
 3.0.14
 ======
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 1efa6d4..54888b8 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -524,6 +524,12 @@ h4(#createMVWhere). @WHERE@ Clause
 
 The @<where-clause>@ is similar to the "where clause of a @SELECT@ statement":#selectWhere, with a few differences.  First, the where clause must contain an expression that disallows @NULL@ values in columns in the view's primary key.  If no other restriction is desired, this can be accomplished with an @IS NOT NULL@ expression.  Second, only columns which are in the base table's primary key may be restricted with expressions other than @IS NOT NULL@.  (Note that this second restriction may be lifted in the future.)
 
+h4. MV Limitations
+
+__Note:__
+Removal of columns not selected in the Materialized View (via `UPDATE base SET unselected_column = null` or `DELETE unselected_column FROM base`) may shadow missed updates to other columns received by hints or repair.
+For this reason, we advise against doing deletions on base columns not selected in views until this is fixed on CASSANDRA-13826.
+
 h3(#alterMVStmt). ALTER MATERIALIZED VIEW
 
 __Syntax:__

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index 44f3a96..1eb991a 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -1078,6 +1078,19 @@ public final class CFMetaData
         return isView;
     }
 
+    /**
+     * A table with strict liveness filters/ignores rows without PK liveness info,
+     * effectively tying the row liveness to its primary key liveness.
+     *
+     * Currently this is only used by views with normal base column as PK column
+     * so updates to other columns do not make the row live when the base column
+     * is not live. See CASSANDRA-11500.
+     */
+    public boolean enforceStrictLiveness()
+    {
+        return isView && Keyspace.open(ksName).viewManager.getByName(cfName).enforceStrictLiveness();
+    }
+
     public Serializers serializers()
     {
         return serializers;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/cql3/UpdateParameters.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
index 8ff5344..d070f61 100644
--- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java
+++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
@@ -233,6 +233,6 @@ public class UpdateParameters
             return pendingMutations;
 
         return Rows.merge(prefetchedRow, pendingMutations, nowInSec)
-                   .purge(DeletionPurger.PURGE_ALL, nowInSec);
+                   .purge(DeletionPurger.PURGE_ALL, nowInSec, metadata.enforceStrictLiveness());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index 5db4b9f..befdd25 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -233,22 +233,10 @@ public class AlterTableStatement extends SchemaAlteringStatement
                                                                                         .collect(Collectors.joining(","))));
                 }
 
-                // If a column is dropped which is included in a view, we don't allow the drop to take place.
-                boolean rejectAlter = false;
-                StringBuilder builder = new StringBuilder();
-                for (ViewDefinition view : views)
-                {
-                    if (!view.includes(columnName)) continue;
-                    if (rejectAlter)
-                        builder.append(',');
-                    rejectAlter = true;
-                    builder.append(view.viewName);
-                }
-                if (rejectAlter)
-                    throw new InvalidRequestException(String.format("Cannot drop column %s, depended on by materialized views (%s.{%s})",
+                if (!Iterables.isEmpty(views))
+                    throw new InvalidRequestException(String.format("Cannot drop column %s on base table with materialized views.",
                                                                     columnName.toString(),
-                                                                    keyspace(),
-                                                                    builder.toString()));
+                                                                    keyspace()));
                 break;
             case OPTS:
                 if (attrs == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/LivenessInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/LivenessInfo.java b/src/java/org/apache/cassandra/db/LivenessInfo.java
index 411fb9a..ab61a23 100644
--- a/src/java/org/apache/cassandra/db/LivenessInfo.java
+++ b/src/java/org/apache/cassandra/db/LivenessInfo.java
@@ -176,13 +176,26 @@ public class LivenessInfo
      * Whether this liveness information supersedes another one (that is
      * whether is has a greater timestamp than the other or not).
      *
-     * @param other the {@code LivenessInfo} to compare this info to.
+     * </br>
+     *
+     * If timestamps are the same, livenessInfo with greater TTL supersedes another.
+     *
+     * It also means, if timestamps are the same, ttl superseders no-ttl.
+     *
+     * This is the same rule as {@link Conflicts#resolveRegular}
+     *
+     * @param other
+     *            the {@code LivenessInfo} to compare this info to.
      *
      * @return whether this {@code LivenessInfo} supersedes {@code other}.
      */
     public boolean supersedes(LivenessInfo other)
     {
-        return timestamp > other.timestamp;
+        if (timestamp != other.timestamp)
+            return timestamp > other.timestamp;
+        if (isExpiring() == other.isExpiring())
+            return localExpirationTime() > other.localExpirationTime();
+        return isExpiring();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 6a21bb3..b73cdde 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -541,7 +541,12 @@ public abstract class ReadCommand implements ReadQuery
         {
             public WithoutPurgeableTombstones()
             {
-                super(isForThrift, nowInSec(), cfs.gcBefore(nowInSec()), oldestUnrepairedTombstone(), cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones());
+                super(isForThrift,
+                      nowInSec(),
+                      cfs.gcBefore(nowInSec()),
+                      oldestUnrepairedTombstone(),
+                      cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones(),
+                      cfs.metadata.enforceStrictLiveness());
             }
 
             protected Predicate<Long> getPurgeEvaluator()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
index 9f0984f..bea365c 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterator.java
@@ -266,7 +266,12 @@ public class CompactionIterator extends CompactionInfo.Holder implements Unfilte
 
         private Purger(boolean isForThrift, CompactionController controller, int nowInSec)
         {
-            super(isForThrift, nowInSec, controller.gcBefore, controller.compactingRepaired() ? Integer.MAX_VALUE : Integer.MIN_VALUE, controller.cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones());
+            super(isForThrift,
+                  nowInSec,
+                  controller.gcBefore,
+                  controller.compactingRepaired() ? Integer.MAX_VALUE : Integer.MIN_VALUE,
+                  controller.cfs.getCompactionStrategyManager().onlyPurgeRepairedTombstones(),
+                  controller.cfs.metadata.enforceStrictLiveness());
             this.controller = controller;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/filter/RowFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/RowFilter.java b/src/java/org/apache/cassandra/db/filter/RowFilter.java
index 8d11038..9f3b868 100644
--- a/src/java/org/apache/cassandra/db/filter/RowFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/RowFilter.java
@@ -148,7 +148,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
     public boolean isSatisfiedBy(CFMetaData metadata, DecoratedKey partitionKey, Row row, int nowInSec)
     {
         // We purge all tombstones as the expressions isSatisfiedBy methods expects it
-        Row purged = row.purge(DeletionPurger.PURGE_ALL, nowInSec);
+        Row purged = row.purge(DeletionPurger.PURGE_ALL, nowInSec, metadata.enforceStrictLiveness());
         if (purged == null)
             return expressions.isEmpty();
 
@@ -290,7 +290,7 @@ public abstract class RowFilter implements Iterable<RowFilter.Expression>
 
                 public Row applyToRow(Row row)
                 {
-                    Row purged = row.purge(DeletionPurger.PURGE_ALL, nowInSec);
+                    Row purged = row.purge(DeletionPurger.PURGE_ALL, nowInSec, metadata.enforceStrictLiveness());
                     if (purged == null)
                         return null;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java b/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java
index 6679bdf..5cc9145 100644
--- a/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java
+++ b/src/java/org/apache/cassandra/db/partitions/PurgeFunction.java
@@ -28,9 +28,16 @@ public abstract class PurgeFunction extends Transformation<UnfilteredRowIterator
     private final boolean isForThrift;
     private final DeletionPurger purger;
     private final int nowInSec;
+
+    private final boolean enforceStrictLiveness;
     private boolean isReverseOrder;
 
-    public PurgeFunction(boolean isForThrift, int nowInSec, int gcBefore, int oldestUnrepairedTombstone, boolean onlyPurgeRepairedTombstones)
+    public PurgeFunction(boolean isForThrift,
+                         int nowInSec,
+                         int gcBefore,
+                         int oldestUnrepairedTombstone,
+                         boolean onlyPurgeRepairedTombstones,
+                         boolean enforceStrictLiveness)
     {
         this.isForThrift = isForThrift;
         this.nowInSec = nowInSec;
@@ -38,6 +45,7 @@ public abstract class PurgeFunction extends Transformation<UnfilteredRowIterator
                       !(onlyPurgeRepairedTombstones && localDeletionTime >= oldestUnrepairedTombstone)
                       && localDeletionTime < gcBefore
                       && getPurgeEvaluator().test(timestamp);
+        this.enforceStrictLiveness = enforceStrictLiveness;
     }
 
     protected abstract Predicate<Long> getPurgeEvaluator();
@@ -84,14 +92,14 @@ public abstract class PurgeFunction extends Transformation<UnfilteredRowIterator
     protected Row applyToStatic(Row row)
     {
         updateProgress();
-        return row.purge(purger, nowInSec);
+        return row.purge(purger, nowInSec, enforceStrictLiveness);
     }
 
     @Override
     protected Row applyToRow(Row row)
     {
         updateProgress();
-        return row.purge(purger, nowInSec);
+        return row.purge(purger, nowInSec, enforceStrictLiveness);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/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 41dad0a..b07960c 100644
--- a/src/java/org/apache/cassandra/db/rows/BTreeRow.java
+++ b/src/java/org/apache/cassandra/db/rows/BTreeRow.java
@@ -334,7 +334,7 @@ public class BTreeRow extends AbstractRow
              : new BTreeRow(clustering, primaryKeyLivenessInfo, Deletion.regular(newDeletion), btree, Integer.MIN_VALUE);
     }
 
-    public Row purge(DeletionPurger purger, int nowInSec)
+    public Row purge(DeletionPurger purger, int nowInSec, boolean enforceStrictLiveness)
     {
         if (!hasDeletion(nowInSec))
             return this;
@@ -342,6 +342,10 @@ public class BTreeRow extends AbstractRow
         LivenessInfo newInfo = purger.shouldPurge(primaryKeyLivenessInfo, nowInSec) ? LivenessInfo.EMPTY : primaryKeyLivenessInfo;
         Deletion newDeletion = purger.shouldPurge(deletion.time()) ? Deletion.LIVE : deletion;
 
+        // when enforceStrictLiveness is set, a row is considered dead when it's PK liveness info is not present
+        if (enforceStrictLiveness && newDeletion.isLive() && newInfo.isEmpty())
+            return null;
+
         return transformAndFilter(newInfo, newDeletion, (cd) -> cd.purge(purger, nowInSec));
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/rows/Row.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Row.java b/src/java/org/apache/cassandra/db/rows/Row.java
index 9ab1f09..3bcc220 100644
--- a/src/java/org/apache/cassandra/db/rows/Row.java
+++ b/src/java/org/apache/cassandra/db/rows/Row.java
@@ -200,10 +200,20 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      *
      * @param purger the {@code DeletionPurger} to use to decide what can be purged.
      * @param nowInSec the current time to decide what is deleted and what isn't (in the case of expired cells).
+     * @param enforceStrictLiveness whether the row should be purged if there is no PK liveness info,
+     *                              normally retrieved from {@link CFMetaData#enforceStrictLiveness()}
+     *
+     *        When enforceStrictLiveness is set, rows with empty PK liveness info
+     *        and no row deletion are purged.
+     *
+     *        Currently this is only used by views with normal base column as PK column
+     *        so updates to other base columns do not make the row live when the PK column
+     *        is not live. See CASSANDRA-11500.
+     *
      * @return this row but without any deletion info purged by {@code purger}. If the purged row is empty, returns
      * {@code null}.
      */
-    public Row purge(DeletionPurger purger, int nowInSec);
+    public Row purge(DeletionPurger purger, int nowInSec, boolean enforceStrictLiveness);
 
     /**
      * Returns a copy of this row where all counter cells have they "local" shard marked for clearing.
@@ -215,7 +225,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
      * timestamp by {@code newTimestamp - 1}.
      *
      * @param newTimestamp the timestamp to use for all live data in the returned row.
-     * @param a copy of this row with timestamp updated using {@code newTimestamp}. This can return {@code null} in the
+     * @return a copy of this row with timestamp updated using {@code newTimestamp}. This can return {@code null} in the
      * rare where the row only as a shadowable row deletion and the new timestamp supersedes it.
      *
      * @see Commit for why we need this.
@@ -277,6 +287,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
             return time.isLive() ? LIVE : new Deletion(time, false);
         }
 
+        @Deprecated
         public static Deletion shadowable(DeletionTime time)
         {
             return new Deletion(time, true);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
index bdc8388..c4684e1 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
@@ -88,6 +88,11 @@ public class UnfilteredSerializer
      * Extended flags
      */
     private final static int IS_STATIC               = 0x01; // Whether the encoded row is a static. If there is no extended flag, the row is assumed not static.
+    /**
+     * A shadowable tombstone cannot replace a previous row deletion otherwise it could resurrect a
+     * previously deleted cell not updated by a subsequent update, SEE CASSANDRA-11500
+     */
+    @Deprecated
     private final static int HAS_SHADOWABLE_DELETION = 0x02; // Whether the row deletion is shadowable. If there is no extended flag (or no row deletion), the deletion is assumed not shadowable.
 
     public void serialize(Unfiltered unfiltered, SerializationHeader header, DataOutputPlus out, int version)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/transform/Filter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/transform/Filter.java b/src/java/org/apache/cassandra/db/transform/Filter.java
index 747983f..48a1634 100644
--- a/src/java/org/apache/cassandra/db/transform/Filter.java
+++ b/src/java/org/apache/cassandra/db/transform/Filter.java
@@ -26,10 +26,12 @@ import org.apache.cassandra.db.rows.*;
 public final class Filter extends Transformation
 {
     private final int nowInSec;
+    private final boolean enforceStrictLiveness;
 
-    public Filter(int nowInSec)
+    public Filter(int nowInSec, boolean enforceStrictLiveness)
     {
         this.nowInSec = nowInSec;
+        this.enforceStrictLiveness = enforceStrictLiveness;
     }
 
     @Override
@@ -46,14 +48,14 @@ public final class Filter extends Transformation
         if (row.isEmpty())
             return Rows.EMPTY_STATIC_ROW;
 
-        row = row.purge(DeletionPurger.PURGE_ALL, nowInSec);
+        row = row.purge(DeletionPurger.PURGE_ALL, nowInSec, enforceStrictLiveness);
         return row == null ? Rows.EMPTY_STATIC_ROW : row;
     }
 
     @Override
     protected Row applyToRow(Row row)
     {
-        return row.purge(DeletionPurger.PURGE_ALL, nowInSec);
+        return row.purge(DeletionPurger.PURGE_ALL, nowInSec, enforceStrictLiveness);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/transform/FilteredPartitions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/transform/FilteredPartitions.java b/src/java/org/apache/cassandra/db/transform/FilteredPartitions.java
index ad9446d..b835a6b 100644
--- a/src/java/org/apache/cassandra/db/transform/FilteredPartitions.java
+++ b/src/java/org/apache/cassandra/db/transform/FilteredPartitions.java
@@ -52,7 +52,9 @@ public final class FilteredPartitions extends BasePartitions<RowIterator, BasePa
      */
     public static FilteredPartitions filter(UnfilteredPartitionIterator iterator, int nowInSecs)
     {
-        FilteredPartitions filtered = filter(iterator, new Filter(nowInSecs));
+        FilteredPartitions filtered = filter(iterator,
+                                             new Filter(nowInSecs,
+                                                        iterator.metadata().enforceStrictLiveness()));
 
         return iterator.isForThrift()
              ? filtered

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/transform/FilteredRows.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/transform/FilteredRows.java b/src/java/org/apache/cassandra/db/transform/FilteredRows.java
index 5b635eb..349183c 100644
--- a/src/java/org/apache/cassandra/db/transform/FilteredRows.java
+++ b/src/java/org/apache/cassandra/db/transform/FilteredRows.java
@@ -55,6 +55,6 @@ public final class FilteredRows extends BaseRows<Row, BaseRowIterator<?>> implem
      */
     public static RowIterator filter(UnfilteredRowIterator iterator, int nowInSecs)
     {
-        return new Filter(nowInSecs).applyToPartition(iterator);
+        return new Filter(nowInSecs, iterator.metadata().enforceStrictLiveness()).applyToPartition(iterator);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/view/TableViews.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/TableViews.java b/src/java/org/apache/cassandra/db/view/TableViews.java
index 1a3cbb1..d2d4a45 100644
--- a/src/java/org/apache/cassandra/db/view/TableViews.java
+++ b/src/java/org/apache/cassandra/db/view/TableViews.java
@@ -159,6 +159,7 @@ public class TableViews extends AbstractCollection<View>
      * but has simply some updated values. This will be empty for view building as we want to assume anything we'll pass
      * to {@code updates} is new.
      * @param nowInSec the current time in seconds.
+     * @param separateUpdates, if false, mutation is per partition.
      * @return the mutations to apply to the {@code views}. This can be empty.
      */
     public Iterator<Collection<Mutation>> generateViewUpdates(Collection<View> views,
@@ -282,7 +283,10 @@ public class TableViews extends AbstractCollection<View>
                             continue;
 
                         Row updateRow = (Row) update;
-                        addToViewUpdateGenerators(emptyRow(updateRow.clustering(), DeletionTime.LIVE), updateRow, generators, nowInSec);
+                        addToViewUpdateGenerators(emptyRow(updateRow.clustering(), existingsDeletion.currentDeletion()),
+                                                  updateRow,
+                                                  generators,
+                                                  nowInSec);
 
                         // If the updates have been filtered, then we won't have any mutations; we need to make sure that we
                         // only return if the mutations are empty. Otherwise, we continue to search for an update which is
@@ -321,7 +325,10 @@ public class TableViews extends AbstractCollection<View>
                     continue;
 
                 Row updateRow = (Row) update;
-                addToViewUpdateGenerators(emptyRow(updateRow.clustering(), DeletionTime.LIVE), updateRow, generators, nowInSec);
+                addToViewUpdateGenerators(emptyRow(updateRow.clustering(), existingsDeletion.currentDeletion()),
+                                          updateRow,
+                                          generators,
+                                          nowInSec);
             }
 
             return Iterators.singletonIterator(buildMutations(baseTableMetadata, generators));
@@ -419,11 +426,12 @@ public class TableViews extends AbstractCollection<View>
         ClusteringIndexFilter clusteringFilter = names == null
                                                ? new ClusteringIndexSliceFilter(sliceBuilder.build(), false)
                                                : new ClusteringIndexNamesFilter(names, false);
+        // since unselected columns also affect view liveness, we need to query all base columns if base and view have same key columns.
         // If we have more than one view, we should merge the queried columns by each views but to keep it simple we just
         // include everything. We could change that in the future.
-        ColumnFilter queriedColumns = views.size() == 1
-                                    ? Iterables.getOnlyElement(views).getSelectStatement().queriedColumns()
-                                    : ColumnFilter.all(metadata);
+        ColumnFilter queriedColumns = views.size() == 1 && metadata.enforceStrictLiveness()
+                                   ? Iterables.getOnlyElement(views).getSelectStatement().queriedColumns()
+                                   : ColumnFilter.all(metadata);
         // Note that the views could have restrictions on regular columns, but even if that's the case we shouldn't apply those
         // when we read, because even if an existing row doesn't match the view filter, the update can change that in which
         // case we'll need to know the existing content. There is also no easy way to merge those RowFilter when we have multiple views.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/View.java b/src/java/org/apache/cassandra/db/view/View.java
index e471349..58e2a84 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -60,7 +60,6 @@ public class View
 
     public volatile List<ColumnDefinition> baseNonPKColumnsInViewPK;
 
-    private final boolean includeAllColumns;
     private ViewBuilder builder;
 
     // Only the raw statement can be final, because the statement cannot always be prepared when the MV is initialized.
@@ -75,7 +74,6 @@ public class View
     {
         this.baseCfs = baseCfs;
         this.name = definition.viewName;
-        this.includeAllColumns = definition.includeAllColumns;
         this.rawSelect = definition.select;
 
         updateDefinition(definition);
@@ -95,8 +93,6 @@ public class View
     public void updateDefinition(ViewDefinition definition)
     {
         this.definition = definition;
-
-        CFMetaData viewCfm = definition.metadata;
         List<ColumnDefinition> nonPKDefPartOfViewPK = new ArrayList<>();
         for (ColumnDefinition baseColumn : baseCfs.metadata.allColumns())
         {
@@ -148,21 +144,7 @@ public class View
         //    neither included in the view, nor used by the view filter).
         if (!getReadQuery().selectsClustering(partitionKey, update.clustering()))
             return false;
-
-        // We want to find if the update modify any of the columns that are part of the view (in which case the view is affected).
-        // But if the view include all the base table columns, or the update has either a row deletion or a row liveness (note
-        // that for the liveness, it would be more "precise" to check if it's live, but pushing an update that is already expired
-        // is dump so it's ok not to optimize for it and it saves us from having to pass nowInSec to the method), we know the view
-        // is affected right away.
-        if (includeAllColumns || !update.deletion().isLive() || !update.primaryKeyLivenessInfo().isEmpty())
-            return true;
-
-        for (ColumnData data : update)
-        {
-            if (definition.metadata.getColumnDefinition(data.column().name) != null)
-                return true;
-        }
-        return false;
+        return true;
     }
 
     /**
@@ -293,4 +275,27 @@ public class View
 
         return expressions.stream().collect(Collectors.joining(" AND "));
     }
+
+    public boolean hasSamePrimaryKeyColumnsAsBaseTable()
+    {
+        return baseNonPKColumnsInViewPK.isEmpty();
+    }
+
+    /**
+     * When views contains a primary key column that is not part
+     * of the base table primary key, we use that column liveness
+     * info as the view PK, to ensure that whenever that column
+     * is not live in the base, the row is not live in the view.
+     *
+     * This is done to prevent cells other than the view PK from
+     * making the view row alive when the view PK column is not
+     * live in the base. So in this case we tie the row liveness,
+     * to the primary key liveness.
+     *
+     * See CASSANDRA-11500 for context.
+     */
+    public boolean enforceStrictLiveness()
+    {
+        return !baseNonPKColumnsInViewPK.isEmpty();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/view/ViewManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewManager.java b/src/java/org/apache/cassandra/db/view/ViewManager.java
index 0a0fa7b..d1cfd9e 100644
--- a/src/java/org/apache/cassandra/db/view/ViewManager.java
+++ b/src/java/org/apache/cassandra/db/view/ViewManager.java
@@ -161,6 +161,11 @@ public class ViewManager
         SystemKeyspace.setViewRemoved(keyspace.getName(), view.name);
     }
 
+    public View getByName(String name)
+    {
+        return viewsByName.get(name);
+    }
+
     public void buildAllViews()
     {
         for (View view : allViews())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
index edb88d0..0c8e078 100644
--- a/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
+++ b/src/java/org/apache/cassandra/db/view/ViewUpdateGenerator.java
@@ -70,7 +70,7 @@ public class ViewUpdateGenerator
         UPDATE_EXISTING, // There was an entry and the update modifies it
         SWITCH_ENTRY     // There was an entry and there is still one after update,
                          // but they are not the same one.
-    };
+    }
 
     /**
      * Creates a new {@code ViewUpdateBuilder}.
@@ -121,14 +121,14 @@ public class ViewUpdateGenerator
                 createEntry(mergedBaseRow);
                 return;
             case DELETE_OLD:
-                deleteOldEntry(existingBaseRow);
+                deleteOldEntry(existingBaseRow, mergedBaseRow);
                 return;
             case UPDATE_EXISTING:
                 updateEntry(existingBaseRow, mergedBaseRow);
                 return;
             case SWITCH_ENTRY:
                 createEntry(mergedBaseRow);
-                deleteOldEntry(existingBaseRow);
+                deleteOldEntry(existingBaseRow, mergedBaseRow);
                 return;
         }
     }
@@ -180,6 +180,7 @@ public class ViewUpdateGenerator
         }
 
         assert view.baseNonPKColumnsInViewPK.size() <= 1 : "We currently only support one base non-PK column in the view PK";
+
         if (view.baseNonPKColumnsInViewPK.isEmpty())
         {
             // The view entry is necessarily the same pre and post update.
@@ -204,7 +205,9 @@ public class ViewUpdateGenerator
         if (!isLive(before))
             return isLive(after) ? UpdateAction.NEW_ENTRY : UpdateAction.NONE;
         if (!isLive(after))
+        {
             return UpdateAction.DELETE_OLD;
+        }
 
         return baseColumn.cellValueType().compare(before.value(), after.value()) == 0
              ? UpdateAction.UPDATE_EXISTING
@@ -269,7 +272,7 @@ public class ViewUpdateGenerator
         }
         if (!matchesViewFilter(mergedBaseRow))
         {
-            deleteOldEntryInternal(existingBaseRow);
+            deleteOldEntryInternal(existingBaseRow, mergedBaseRow);
             return;
         }
 
@@ -281,6 +284,12 @@ public class ViewUpdateGenerator
         currentViewEntryBuilder.addPrimaryKeyLivenessInfo(computeLivenessInfoForEntry(mergedBaseRow));
         currentViewEntryBuilder.addRowDeletion(mergedBaseRow.deletion());
 
+        addDifferentCells(existingBaseRow, mergedBaseRow);
+        submitUpdate();
+    }
+
+    private void addDifferentCells(Row existingBaseRow, Row mergedBaseRow)
+    {
         // We only add to the view update the cells from mergedBaseRow that differs from
         // existingBaseRow. For that and for speed we can just cell pointer equality: if the update
         // hasn't touched a cell, we know it will be the same object in existingBaseRow and
@@ -362,8 +371,6 @@ public class ViewUpdateGenerator
                 addCell(viewColumn, (Cell)mergedData);
             }
         }
-
-        submitUpdate();
     }
 
     /**
@@ -371,20 +378,40 @@ public class ViewUpdateGenerator
      * <p>
      * This method checks that the base row does match the view filter before bothering.
      */
-    private void deleteOldEntry(Row existingBaseRow)
+    private void deleteOldEntry(Row existingBaseRow, Row mergedBaseRow)
     {
         // Before deleting an old entry, make sure it was matching the view filter (otherwise there is nothing to delete)
         if (!matchesViewFilter(existingBaseRow))
             return;
 
-        deleteOldEntryInternal(existingBaseRow);
+        deleteOldEntryInternal(existingBaseRow, mergedBaseRow);
     }
 
-    private void deleteOldEntryInternal(Row existingBaseRow)
+    private void deleteOldEntryInternal(Row existingBaseRow, Row mergedBaseRow)
     {
         startNewUpdate(existingBaseRow);
-        DeletionTime dt = new DeletionTime(computeTimestampForEntryDeletion(existingBaseRow), nowInSec);
-        currentViewEntryBuilder.addRowDeletion(Row.Deletion.shadowable(dt));
+        long timestamp = computeTimestampForEntryDeletion(existingBaseRow, mergedBaseRow);
+        long rowDeletion = mergedBaseRow.deletion().time().markedForDeleteAt();
+        assert timestamp >= rowDeletion;
+
+        // If computed deletion timestamp greater than row deletion, it must be coming from
+        //  1. non-pk base column used in view pk, or
+        //  2. unselected base column
+        //  any case, we need to use it as expired livenessInfo
+        // If computed deletion timestamp is from row deletion, we only need row deletion itself
+        if (timestamp > rowDeletion)
+        {
+            /**
+              * TODO: This is a hack and overload of LivenessInfo and we should probably modify
+              * the storage engine to properly support this, but on the meantime this
+              * should be fine because it only happens in some specific scenarios explained above.
+              */
+            LivenessInfo info = LivenessInfo.create(timestamp, Integer.MAX_VALUE, nowInSec);
+            currentViewEntryBuilder.addPrimaryKeyLivenessInfo(info);
+        }
+        currentViewEntryBuilder.addRowDeletion(mergedBaseRow.deletion());
+
+        addDifferentCells(existingBaseRow, mergedBaseRow);
         submitUpdate();
     }
 
@@ -413,78 +440,92 @@ public class ViewUpdateGenerator
 
     private LivenessInfo computeLivenessInfoForEntry(Row baseRow)
     {
-        /*
-         * We need to compute both the timestamp and expiration.
+        /**
+         * There 3 cases:
+         *  1. No extra primary key in view and all base columns are selected in MV. all base row's components(livenessInfo,
+         *     deletion, cells) are same as view row. Simply map base components to view row.
+         *  2. There is a base non-key column used in view pk. This base non-key column determines the liveness of view row. view's row level
+         *     info should based on this column.
+         *  3. Most tricky case is no extra primary key in view and some base columns are not selected in MV. We cannot use 1 livenessInfo or
+         *     row deletion to represent the liveness of unselected column properly, see CASSANDRA-11500.
+         *     We could make some simplification: the unselected columns will be used only when it affects view row liveness. eg. if view row
+         *     already exists and not expiring, there is no need to use unselected columns.
+         *     Note: if the view row is removed due to unselected column removal(ttl or cell tombstone), we will have problem keeping view
+         *     row alive with a smaller or equal timestamp than the max unselected column timestamp.
          *
-         * For the timestamp, it makes sense to use the bigger timestamp for all view PK columns.
-         *
-         * This is more complex for the expiration. We want to maintain consistency between the base and the view, so the
-         * entry should only exist as long as the base row exists _and_ has non-null values for all the columns that are part
-         * of the view PK.
-         * Which means we really have 2 cases:
-         *   1) either the columns for the base and view PKs are exactly the same: in that case, the view entry should live
-         *      as long as the base row lives. This means the view entry should only expire once *everything* in the base row
-         *      has expired. Which means the row TTL should be the max of any other TTL.
-         *   2) or there is a column that is not in the base PK but is in the view PK (we can only have one so far, we'll need
-         *      to slightly adapt if we allow more later): in that case, as long as that column lives the entry does too, but
-         *      as soon as it expires (or is deleted for that matter) the entry also should expire. So the expiration for the
-         *      view is the one of that column, irregarding of any other expiration.
-         *      To take an example of that case, if you have:
-         *        CREATE TABLE t (a int, b int, c int, PRIMARY KEY (a, b))
-         *        CREATE MATERIALIZED VIEW mv AS SELECT * FROM t WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)
-         *        INSERT INTO t(a, b) VALUES (0, 0) USING TTL 3;
-         *        UPDATE t SET c = 0 WHERE a = 0 AND b = 0;
-         *      then even after 3 seconds elapsed, the row will still exist (it just won't have a "row marker" anymore) and so
-         *      the MV should still have a corresponding entry.
          */
         assert view.baseNonPKColumnsInViewPK.size() <= 1; // This may change, but is currently an enforced limitation
 
         LivenessInfo baseLiveness = baseRow.primaryKeyLivenessInfo();
 
-        if (view.baseNonPKColumnsInViewPK.isEmpty())
+        if (view.hasSamePrimaryKeyColumnsAsBaseTable())
         {
-            int ttl = baseLiveness.ttl();
-            int expirationTime = baseLiveness.localExpirationTime();
+            if (view.getDefinition().includeAllColumns)
+                return baseLiveness;
+
+            long timestamp = baseLiveness.timestamp();
+            boolean hasNonExpiringLiveCell = false;
+            Cell biggestExpirationCell = null;
             for (Cell cell : baseRow.cells())
             {
-                if (cell.ttl() > ttl)
+                if (view.getViewColumn(cell.column()) != null)
+                    continue;
+                if (!isLive(cell))
+                    continue;
+                timestamp = Math.max(timestamp, cell.maxTimestamp());
+                if (!cell.isExpiring())
+                    hasNonExpiringLiveCell = true;
+                else
                 {
-                    ttl = cell.ttl();
-                    expirationTime = cell.localDeletionTime();
+                    if (biggestExpirationCell == null)
+                        biggestExpirationCell = cell;
+                    else if (cell.localDeletionTime() > biggestExpirationCell.localDeletionTime())
+                        biggestExpirationCell = cell;
                 }
             }
-            return ttl == baseLiveness.ttl()
-                 ? baseLiveness
-                 : LivenessInfo.create(baseLiveness.timestamp(), ttl, expirationTime);
+            if (baseLiveness.isLive(nowInSec) && !baseLiveness.isExpiring())
+                return LivenessInfo.create(viewMetadata, timestamp, nowInSec);
+            if (hasNonExpiringLiveCell)
+                return LivenessInfo.create(viewMetadata, timestamp, nowInSec);
+            if (biggestExpirationCell == null)
+                return baseLiveness;
+            if (biggestExpirationCell.localDeletionTime() > baseLiveness.localExpirationTime()
+                    || !baseLiveness.isLive(nowInSec))
+                return LivenessInfo.create(timestamp,
+                                                       biggestExpirationCell.ttl(),
+                                                       biggestExpirationCell.localDeletionTime());
+            return baseLiveness;
         }
 
-        ColumnDefinition baseColumn = view.baseNonPKColumnsInViewPK.get(0);
-        Cell cell = baseRow.getCell(baseColumn);
+        Cell cell = baseRow.getCell(view.baseNonPKColumnsInViewPK.get(0));
         assert isLive(cell) : "We shouldn't have got there if the base row had no associated entry";
 
-        long timestamp = Math.max(baseLiveness.timestamp(), cell.timestamp());
-        return LivenessInfo.create(timestamp, cell.ttl(), cell.localDeletionTime());
+        return LivenessInfo.create(cell.timestamp(), cell.ttl(), cell.localDeletionTime());
     }
 
-    private long computeTimestampForEntryDeletion(Row baseRow)
+    private long computeTimestampForEntryDeletion(Row existingBaseRow, Row mergedBaseRow)
     {
-        // We delete the old row with it's row entry timestamp using a shadowable deletion.
-        // We must make sure that the deletion deletes everything in the entry (or the entry will
-        // still show up), so we must use the bigger timestamp found in the existing row (for any
-        // column included in the view at least).
-        // TODO: We have a problem though: if the entry is "resurected" by a later update, we would
-        // need to ensure that the timestamp for then entry then is bigger than the tombstone
-        // we're just inserting, which is not currently guaranteed.
-        // This is a bug for a separate ticket though.
-        long timestamp = baseRow.primaryKeyLivenessInfo().timestamp();
-        for (ColumnData data : baseRow)
+        DeletionTime deletion = mergedBaseRow.deletion().time();
+        if (view.hasSamePrimaryKeyColumnsAsBaseTable())
         {
-            if (!view.getDefinition().includes(data.column().name))
-                continue;
+            long timestamp = Math.max(deletion.markedForDeleteAt(), existingBaseRow.primaryKeyLivenessInfo().timestamp());
+            if (view.getDefinition().includeAllColumns)
+                return timestamp;
 
-            timestamp = Math.max(timestamp, data.maxTimestamp());
+            for (Cell cell : existingBaseRow.cells())
+            {
+                // selected column should not contribute to view deletion, itself is already included in view row
+                if (view.getViewColumn(cell.column()) != null)
+                    continue;
+                // unselected column is used regardless live or dead, because we don't know if it was used for liveness.
+                timestamp = Math.max(timestamp, cell.maxTimestamp());
+            }
+            return timestamp;
         }
-        return timestamp;
+        // has base non-pk column in view pk
+        Cell before = existingBaseRow.getCell(view.baseNonPKColumnsInViewPK.get(0));
+        assert isLive(before) : "We shouldn't have got there if the base row had no associated entry";
+        return deletion.deletes(before) ? deletion.markedForDeleteAt() : before.timestamp();
     }
 
     private void addColumnData(ColumnDefinition viewColumn, ColumnData baseTableData)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/src/java/org/apache/cassandra/service/DataResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/DataResolver.java b/src/java/org/apache/cassandra/service/DataResolver.java
index 72c4950..c59d688 100644
--- a/src/java/org/apache/cassandra/service/DataResolver.java
+++ b/src/java/org/apache/cassandra/service/DataResolver.java
@@ -86,7 +86,9 @@ public class DataResolver extends ResponseResolver
         DataLimits.Counter counter = command.limits().newCounter(command.nowInSec(), true, command.selectsFullPartition());
 
         UnfilteredPartitionIterator merged = mergeWithShortReadProtection(iters, sources, counter);
-        FilteredPartitions filtered = FilteredPartitions.filter(merged, new Filter(command.nowInSec()));
+        FilteredPartitions filtered = FilteredPartitions.filter(merged,
+                                                                new Filter(command.nowInSec(),
+                                                                           command.metadata().enforceStrictLiveness()));
         PartitionIterator counted = counter.applyTo(filtered);
 
         return command.isForThrift()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b36740e/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index 40aec88..3c0cefc 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -1001,7 +1001,7 @@ public abstract class CQLTester
         assert ignoreExtra || expectedRows.size() == actualRows.size();
     }
 
-    private static List<String> makeRowStrings(UntypedResultSet resultSet)
+    protected static List<String> makeRowStrings(UntypedResultSet resultSet)
     {
         List<List<ByteBuffer>> rows = new ArrayList<>();
         for (UntypedResultSet.Row row : resultSet)


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